Fix of a bunch of javadoc warnings -- more to follow

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1033373 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Michael Stack 2010-11-10 06:37:35 +00:00
parent 504a604d9a
commit 42a8477e40
15 changed files with 38 additions and 44 deletions

View File

@ -523,7 +523,7 @@ public class HColumnDescriptor implements WritableComparable<HColumnDescriptor>
}
/**
* @param toggle bloom filter type
* @param bt bloom filter type
*/
public void setBloomFilterType(final StoreFile.BloomType bt) {
setValue(BLOOMFILTER, bt.toString());

View File

@ -268,7 +268,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
* Make a region name of passed parameters.
* @param tableName
* @param startKey Can be null
* @param id Region id (Usually timestamp from when region was created).
* @param regionid Region id (Usually timestamp from when region was created).
* @param newFormat should we create the region name in the new format
* (such that it contains its encoded name?).
* @return Region name made of passed tableName, startKey and id
@ -347,7 +347,7 @@ public class HRegionInfo extends VersionedWritable implements WritableComparable
/**
* Gets the table name from the specified region name.
* @param regionName
* @return
* @return Table name.
*/
public static byte [] getTableName(byte [] regionName) {
int offset = -1;

View File

@ -266,7 +266,7 @@ public class LocalHBaseCluster {
/**
* Wait for the specified region server to stop
* Removes this thread from list of running threads.
* @param serverNumber
* @param rst
* @return Name of region server that just went down.
*/
public String waitOnRegionServer(JVMClusterUtil.RegionServerThread rst) {
@ -360,7 +360,7 @@ public class LocalHBaseCluster {
/**
* Wait for the specified master to stop
* Removes this thread from list of running threads.
* @param serverNumber
* @param masterThread
* @return Name of master that just went down.
*/
public String waitOnMaster(JVMClusterUtil.MasterThread masterThread) {

View File

@ -34,8 +34,7 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
* <p>
* Utilizes {@link ZooKeeperNodeTracker} for zk interactions.
* <p>
* You can get the current master via {@link #getMasterAddress()} or the
* blocking method {@link #waitMasterAddress()}.
* You can get the current master via {@link #getMasterAddress()}
*/
public class MasterAddressTracker extends ZooKeeperNodeTracker {
/**
@ -57,8 +56,6 @@ public class MasterAddressTracker extends ZooKeeperNodeTracker {
* Get the address of the current master if one is available. Returns null
* if no current master.
*
* Use {@link #waitMasterAddress} if you want to block until the master is
* available.
* @return server address of current active master, or null if none available
*/
public HServerAddress getMasterAddress() {

View File

@ -45,7 +45,7 @@ public class MetaEditor {
/**
* Adds a META row for the specified new region.
* @param info region information
* @param regionInfo region information
* @throws IOException if problem connecting or updating meta
*/
public static void addRegionToMeta(CatalogTracker catalogTracker,

View File

@ -369,7 +369,7 @@ public class MetaReader {
/**
* Checks if the specified table exists. Looks at the META table hosted on
* the specified server.
* @param metaServer server hosting meta
* @param catalogTracker
* @param tableName table to check
* @return true if the table exists in meta, false if not
* @throws IOException

View File

@ -439,9 +439,9 @@ public class HBaseAdmin implements Abortable {
* and {@link #isTableEnabled(byte[])} instead.
* @param tableName name of the table
* @throws IOException if a remote or network exception occurs
* @see {@link #isTableEnabled(byte[])}
* @see {@link #disableTable(byte[])}
* @see {@link #enableTableAsync(byte[])}
* @see isTableEnabled(byte[])
* @see disableTable(byte[])
* @see enableTableAsync(byte[])
*/
public void enableTable(final byte [] tableName)
throws IOException {
@ -514,8 +514,8 @@ public class HBaseAdmin implements Abortable {
* If table is taking too long to online, check server logs.
* @param tableName name of table
* @throws IOException if a remote or network exception occurs
* @see {@link #isTableDisabled(byte[])}
* @see {@link #isTableEnabled(byte[])}
* @see isTableDisabled(byte[])
* @see isTableEnabled(byte[])
* @since 0.90.0
*/
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
* that have not be successfully applied.
* @since 0.20.1
* @see {@link #batch(java.util.List, Object[])}
* @see batch(java.util.List, Object[])
*/
@Override
public void delete(final List<Delete> deletes)

View File

@ -2373,7 +2373,7 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
}
}
/** {@inheritDoc} */
@Override
public HRegionInfo[] getRegionsAssignment() throws IOException {
HRegionInfo[] regions = new HRegionInfo[onlineRegions.size()];
Iterator<HRegion> ite = onlineRegions.values().iterator();

View File

@ -37,7 +37,7 @@ import org.apache.hadoop.io.compress.DefaultCodec;
/**
* Implementation of {@link HLog.Writer} that delegates to
* {@link SequenceFile.Writer}.
* SequenceFile.Writer.
*/
public class SequenceFileLogWriter implements HLog.Writer {
private final Log LOG = LogFactory.getLog(this.getClass());

View File

@ -278,7 +278,7 @@ public class ZKAssign {
* of the specified regions transition.
*
* @param zkw zk reference
* @param region opened region to be deleted from zk
* @param regionName opened region to be deleted from zk
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NoNodeException if node does not exist
*/
@ -303,7 +303,7 @@ public class ZKAssign {
* that has died are all set to OFFLINE before being processed.
*
* @param zkw zk reference
* @param region closed region to be deleted from zk
* @param regionName closed region to be deleted from zk
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NoNodeException if node does not exist
*/
@ -329,7 +329,7 @@ public class ZKAssign {
* of the specified regions transition to being closed.
*
* @param zkw zk reference
* @param region closed region to be deleted from zk
* @param regionName closed region to be deleted from zk
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NoNodeException if node does not exist
*/
@ -355,7 +355,7 @@ public class ZKAssign {
* of the specified regions transition to being closed.
*
* @param zkw zk reference
* @param region closing region to be deleted from zk
* @param regionName closing region to be deleted from zk
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NoNodeException if node does not exist
*/
@ -723,10 +723,9 @@ public class ZKAssign {
*
* <p>Sets a watch on the node if the node exists.
*
* @param watcher zk reference
* @param zkw zk reference
* @param pathOrRegionName fully-specified path or region name
* @return data for the unassigned node
* @throws KeeperException
* @throws KeeperException if unexpected zookeeper exception
*/
public static RegionTransitionData getData(ZooKeeperWatcher zkw,
@ -749,11 +748,10 @@ public class ZKAssign {
*
* <p>Does not set a watch.
*
* @param watcher zk reference
* @param zkw zk reference
* @param pathOrRegionName fully-specified path or region name
* @param stat object to store node info into on getData call
* @return data for the unassigned node
* @throws KeeperException
* @throws KeeperException if unexpected zookeeper exception
*/
public static RegionTransitionData getDataNoWatch(ZooKeeperWatcher zkw,
@ -834,7 +832,7 @@ public class ZKAssign {
* <p>
* Method synchronizes() with ZK so will yield an up-to-date result but is
* a slow read.
* @param watcher
* @param zkw
* @param region
* @param expectedState
* @return true if region exists and is in expected state

View File

@ -183,7 +183,7 @@ public class ZKConfig {
* Return the ZK Quorum servers string given zk properties returned by
* makeZKProps
* @param properties
* @return
* @return Quorum servers String
*/
public static String getZKQuorumServersString(Properties properties) {
String clientPort = null;
@ -244,7 +244,7 @@ public class ZKConfig {
/**
* Return the ZK Quorum servers string given the specified configuration.
* @param properties
* @return
* @return Quorum servers
*/
public static String getZKQuorumServersString(Configuration conf) {
return getZKQuorumServersString(makeZKProps(conf));

View File

@ -286,7 +286,7 @@ public class ZKUtil {
*
* @param zkw zk reference
* @param znode path of node to list and watch children of
* @returns list of children of the specified node, an empty list if the node
* @return list of children of the specified node, an empty list if the node
* exists but has no children, and null if the node does not exist
* @throws KeeperException if unexpected zookeeper exception
*/
@ -874,7 +874,6 @@ public class ZKUtil {
* @param data data of node to create
* @param cb
* @param ctx
* @return version of node created
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NodeExistsException if node already exists
*/
@ -970,7 +969,7 @@ public class ZKUtil {
/**
* Deletes the specified node. Fails silent if the node does not exist.
* @param zkw
* @param joinZNode
* @param node
* @throws KeeperException
*/
public static void deleteNodeFailSilent(ZooKeeperWatcher zkw, String node)

View File

@ -63,8 +63,8 @@ public abstract class ZooKeeperNodeTracker extends ZooKeeperListener {
/**
* Starts the tracking of the node in ZooKeeper.
*
* <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.
* <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.
*/
public synchronized void start() {
this.watcher.registerListener(this);

View File

@ -220,7 +220,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable {
/**
* Get the quorum address of this instance.
* @returns quorum string of this zookeeper connection instance
* @return quorum string of this zookeeper connection instance
*/
public String getQuorum() {
return quorum;
@ -349,7 +349,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable {
/**
* Get the set of already watched unassigned nodes.
* @return
* @return Set of Nodes.
*/
public Set<String> getNodes() {
return unassignedNodes;