HBASE-7363 Fix javadocs warnings for hbase-server packages from master to end

git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1422493 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
nkeywal 2012-12-16 09:58:29 +00:00
parent 9e9e1ab97d
commit 07d670920e
40 changed files with 79 additions and 97 deletions

View File

@ -112,7 +112,7 @@ public interface MasterAdminProtocol extends
* @param req ModifyColumnRequest that contains:<br>
* - tableName: table name<br>
* - descriptor: new column descriptor
* @throws IOException e
* @throws ServiceException e
*/
@Override
public ModifyColumnResponse modifyColumn(RpcController controller, ModifyColumnRequest req)

View File

@ -560,7 +560,7 @@ Server {
* Create CatalogTracker.
* In its own method so can intercept and mock it over in tests.
* @param zk If zk is null, we'll create an instance (and shut it down
* when {@link #stop()} is called) else we'll use what is passed.
* when {@link #stop(String)} is called) else we'll use what is passed.
* @param conf
* @param abortable If fatal exception we'll call abort on this. May be null.
* If it is we'll use the Connection associated with the passed
@ -1771,7 +1771,7 @@ Server {
@Override
public void checkTableModifiable(final byte [] tableName)
throws IOException {
throws IOException, TableNotFoundException, TableNotDisabledException {
String tableNameStr = Bytes.toString(tableName);
if (isCatalogTable(tableName)) {
throw new IOException("Can't modify catalog tables");

View File

@ -323,7 +323,7 @@ public class MasterFileSystem {
/**
* Get the rootdir. Make sure its wholesome and exists before returning.
* @param rd
* @param conf
* @param c
* @param fs
* @return hbase.rootdir (after checks for existence and bootstrapping if
* needed populating the directory with necessary bootup files).

View File

@ -25,6 +25,8 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.Server;
import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableNotDisabledException;
import org.apache.hadoop.hbase.TableNotFoundException;
import org.apache.hadoop.hbase.executor.ExecutorService;
/**
@ -56,9 +58,12 @@ public interface MasterServices extends Server {
* Check table is modifiable; i.e. exists and is offline.
* @param tableName Name of table to check.
* @throws TableNotDisabledException
* @throws TableNotFoundException
* @throws TableNotFoundException
* @throws IOException
*/
public void checkTableModifiable(final byte [] tableName) throws IOException;
// We actually throw the exceptions mentioned in the
public void checkTableModifiable(final byte [] tableName)
throws IOException, TableNotFoundException, TableNotDisabledException;
/**
* Create a table using the given table definition.

View File

@ -346,8 +346,8 @@ public class ServerManager {
/**
* Adds the onlineServers list.
* @param hsl
* @param serverName The remote servers name.
* @param sl
*/
void recordNewServer(final ServerName serverName, final ServerLoad sl) {
LOG.info("Registering server=" + serverName);

View File

@ -124,10 +124,10 @@ public class SplitLogManager extends ZooKeeperListener {
private final Object deadWorkersLock = new Object();
/**
* Wrapper around {@link #SplitLogManager(ZooKeeperWatcher, Configuration,
* Stoppable, String, TaskFinisher)} that provides a task finisher for
* copying recovered edits to their final destination. The task finisher
* has to be robust because it can be arbitrarily restarted or called
* Wrapper around {@link #SplitLogManager(ZooKeeperWatcher zkw, Configuration conf,
* Stoppable stopper, MasterServices master, ServerName serverName, TaskFinisher tf)}
* that provides a task finisher for copying recovered edits to their final destination.
* The task finisher has to be robust because it can be arbitrarily restarted or called
* multiple times.
*
* @param zkw

View File

@ -173,7 +173,7 @@ public class DefaultLoadBalancer extends BaseLoadBalancer {
* Does this mean we need HeapSize on HMaster? Or just careful monitor?
* (current thinking is we will hold all assignments in memory)
*
* @param clusterState Map of regionservers and their load/region information to
* @param clusterMap Map of regionservers and their load/region information to
* a list of their most loaded regions
* @return a list of regions to be moved, including source and destination,
* or null if cluster is already balanced

View File

@ -116,7 +116,6 @@ class RegionLocationFinder {
* files on that host, so the first host in the list is the server which holds
* the most bytes of the given region's HFiles.
*
* @param fs the filesystem
* @param region region
* @return ordered list of hosts holding blocks of the specified region
*/

View File

@ -569,6 +569,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
* Compute a cost of a potential cluster configuration based upon where
* {@link org.apache.hadoop.hbase.regionserver.StoreFile}s are located.
*
* @param initialRegionMapping - not used
* @param clusterState The state of the cluster
* @return A cost between 0 and 1. 0 Means all regions are on the sever with
* the most local store files.

View File

@ -398,8 +398,9 @@ public class ServerShutdownHandler extends EventHandler {
/**
* Check that daughter regions are up in .META. and if not, add them.
* @param hris All regions for this server in meta.
* @param result The contents of the parent row in .META.
* @param assignmentManager
* @param catalogTracker
* @return the number of daughters missing and fixed
* @throws IOException
*/
@ -417,8 +418,8 @@ public class ServerShutdownHandler extends EventHandler {
/**
* Check individual daughter is up in .META.; fixup if its not.
* @param result The contents of the parent row in .META.
* @param qualifier Which daughter to check for.
* @param result The contents of the parent row in .META. - not used
* @param daughter Which daughter to check for.
* @return 1 if the daughter is missing and fixed. Otherwise 0
* @throws IOException
*/

View File

@ -346,7 +346,7 @@ public final class ProtobufUtil {
/**
* Convert a protocol buffer Get to a client Get
*
* @param get the protocol buffer Get to convert
* @param proto the protocol buffer Get to convert
* @return the converted client Get
* @throws IOException
*/

View File

@ -74,7 +74,6 @@ public class ExplicitColumnTracker implements ColumnTracker {
* @param maxVersions maximum versions to return per column
* @param oldestUnexpiredTS the oldest timestamp we are interested in,
* based on TTL
* @param ttl The timeToLive to enforce
*/
public ExplicitColumnTracker(NavigableSet<byte[]> columns, int minVersions,
int maxVersions, long oldestUnexpiredTS) {

View File

@ -222,8 +222,7 @@ class GetClosestRowBeforeTracker {
/**
* @param kv Current kv
* @param First on row kv.
* @param state
* @param firstOnRow on row kv.
* @return True if we went too far, past the target key.
*/
boolean isTooFar(final KeyValue kv, final KeyValue firstOnRow) {

View File

@ -289,7 +289,7 @@ public class HRegion implements HeapSize { // , Writable{
private final Path regiondir;
KeyValue.KVComparator comparator;
private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
private final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
/**
* @return The smallest mvcc readPoint across all the scanners in this
@ -442,7 +442,7 @@ public class HRegion implements HeapSize { // , Writable{
* (implying that the HRegion has been written-to before), then read it from
* the supplied path.
* @param fs is the filesystem.
* @param conf is global configuration settings.
* @param confParam is global configuration settings.
* @param regionInfo - HRegionInfo that describes the region
* is new), then read them from the supplied path.
* @param rsServices reference to {@link RegionServerServices} or null
@ -3764,7 +3764,7 @@ public class HRegion implements HeapSize { // , Writable{
* @param conf
* @param hTableDescriptor
* @param hlog shared HLog
* @param boolean initialize - true to initialize the region
* @param initialize - true to initialize the region
* @return new HRegion
*
* @throws IOException
@ -3790,11 +3790,9 @@ public class HRegion implements HeapSize { // , Writable{
* @param conf
* @param hTableDescriptor
* @param hlog shared HLog
* @param boolean initialize - true to initialize the region
* @param boolean ignoreHLog
- true to skip generate new hlog if it is null, mostly for createTable
* @param initialize - true to initialize the region
* @param ignoreHLog - true to skip generate new hlog if it is null, mostly for createTable
* @return new HRegion
*
* @throws IOException
*/
public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,

View File

@ -1410,10 +1410,10 @@ public class HRegionServer implements ClientProtocol,
}
/**
* Called by {@link #instantiateHLog(Path, Path)} setting up WAL instance.
* Called by {@link #instantiateHLog(Path, String)} setting up WAL instance.
* Add any {@link WALActionsListener}s you want inserted before WAL startup.
* @return List of WALActionsListener that will be passed in to
* {@link FSHLog} on construction.
* {@link org.apache.hadoop.hbase.regionserver.wal.FSHLog} on construction.
*/
protected List<WALActionsListener> getWALActionListeners() {
List<WALActionsListener> listeners = new ArrayList<WALActionsListener>();
@ -3139,21 +3139,7 @@ public class HRegionServer implements ClientProtocol,
}
/**
* Executes a single {@link org.apache.hadoop.hbase.ipc.CoprocessorProtocol}
* method using the registered protocol handlers.
* {@link CoprocessorProtocol} implementations must be registered per-region
* via the
* {@link org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)}
* method before they are available.
*
* @param regionName name of the region against which the invocation is executed
* @param call an {@code Exec} instance identifying the protocol, method name,
* and parameters for the method invocation
* @return an {@code ExecResult} instance containing the region name of the
* invocation and the return value
* @throws IOException if no registered protocol handler is found or an error
* occurs during the invocation
* @see org.apache.hadoop.hbase.regionserver.HRegion#registerProtocol(Class, org.apache.hadoop.hbase.ipc.CoprocessorProtocol)
* Executes a single method using protobuff.
*/
@Override
public ExecCoprocessorResponse execCoprocessor(final RpcController controller,

View File

@ -135,7 +135,7 @@ public class HStore implements Store, StoreConfiguration {
*/
private volatile ImmutableList<StoreFile> storefiles = null;
List<StoreFile> filesCompacting = Lists.newArrayList();
final List<StoreFile> filesCompacting = Lists.newArrayList();
// All access must be synchronized.
private final CopyOnWriteArraySet<ChangedReadersObserver> changedReaderObservers =

View File

@ -55,7 +55,7 @@ import java.io.IOException;
public class Leases extends HasThread {
private static final Log LOG = LogFactory.getLog(Leases.class.getName());
private final int leaseCheckFrequency;
private volatile DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
private final DelayQueue<Lease> leaseQueue = new DelayQueue<Lease>();
protected final Map<String, Lease> leases = new HashMap<String, Lease>();
private volatile boolean stopRequested = false;
@ -269,7 +269,7 @@ public class Leases extends HasThread {
if (getClass() != obj.getClass()) {
return false;
}
return this.hashCode() == ((Lease) obj).hashCode();
return this.hashCode() == obj.hashCode();
}
@Override

View File

@ -250,7 +250,8 @@ public class RegionCoprocessorHost
* org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#preOpen()} and
* {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost#postOpen()} are such hooks.
*
* See also {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable()}
* See also
* {@link org.apache.hadoop.hbase.master.MasterCoprocessorHost#handleCoprocessorThrowable(CoprocessorEnvironment, Throwable)}
* @param env The coprocessor that threw the exception.
* @param e The exception that was thrown.
*/
@ -523,7 +524,7 @@ public class RegionCoprocessorHost
/**
* See
* {@link RegionObserver#preFlush(ObserverContext, HStore, KeyValueScanner)}
* {@link RegionObserver#preFlush(ObserverContext<RegionCoprocessorEnvironment>, HStore, KeyValueScanner)}
*/
public InternalScanner preFlushScannerOpen(HStore store, KeyValueScanner memstoreScanner) throws IOException {
ObserverContext<RegionCoprocessorEnvironment> ctx = null;

View File

@ -92,7 +92,7 @@ public abstract class RegionSplitPolicy extends Configured {
/**
* Create the RegionSplitPolicy configured for the given table.
* Each
* @param htd
* @param region
* @param conf
* @return
* @throws IOException

View File

@ -75,7 +75,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
private final ServerName serverName;
private final TaskExecutor splitTaskExecutor;
private Object taskReadyLock = new Object();
private final Object taskReadyLock = new Object();
volatile int taskReadySeq = 0;
private volatile String currentTask = null;
private int currentVersion;
@ -176,7 +176,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
* one at a time. This policy puts an upper-limit on the number of
* simultaneous log splitting that could be happening in a cluster.
* <p>
* Synchronization using {@link #task_ready_signal_seq} ensures that it will
* Synchronization using {@link #taskReadyLock} ensures that it will
* try to grab every task that has been put up
*/
private void taskLoop() {
@ -371,7 +371,7 @@ public class SplitLogWorker extends ZooKeeperListener implements Runnable {
/**
* endTask() can fail and the only way to recover out of it is for the
* {@link SplitLogManager} to timeout the task node.
* @param ts
* @param slt
* @param ctr
*/
private void endTask(SplitLogTask slt, AtomicLong ctr) {

View File

@ -512,7 +512,6 @@ public class SplitTransaction {
/**
* Open daughter regions, add them to online list and update meta.
* @param server
* @param services Can be null when testing.
* @param daughter
* @throws IOException
* @throws KeeperException
@ -692,10 +691,10 @@ public class SplitTransaction {
/**
* @param hri Spec. for daughter region to open.
* @param flusher Flusher this region should use.
* @param rsServices RegionServerServices this region should use.
* @return Created daughter HRegion.
* @throws IOException
* @see #cleanupDaughterRegion(FileSystem, Path, HRegionInfo)
* @see #cleanupDaughterRegion(FileSystem, Path, String)
*/
HRegion createDaughterRegion(final HRegionInfo hri,
final RegionServerServices rsServices)

View File

@ -226,7 +226,6 @@ public class StoreFile {
*
* @param fs The current file system to use.
* @param p The path of the file.
* @param blockcache <code>true</code> if the block cache is enabled.
* @param conf The current configuration.
* @param cacheConf The cache configuration and block cache reference.
* @param cfBloomType The bloom type to use for this store file as specified
@ -524,7 +523,7 @@ public class StoreFile {
* Opens reader on this store file. Called by Constructor.
* @return Reader for the store file.
* @throws IOException
* @see #closeReader()
* @see #closeReader(boolean)
*/
private Reader open() throws IOException {
if (this.reader != null) {
@ -1275,7 +1274,8 @@ public class StoreFile {
writer.appendFileInfo(key, value);
}
/** For use in testing, e.g. {@link CreateRandomStoreFile} */
/** For use in testing, e.g. {@link org.apache.hadoop.hbase.regionserver.CreateRandomStoreFile}
*/
HFile.Writer getHFileWriter() {
return writer;
}

View File

@ -306,8 +306,8 @@ public class OpenRegionHandler extends EventHandler {
}
/**
* @param Region we're working on.
* This is not guaranteed to succeed, we just do our best.
* @param hri Region we're working on.
* @return whether znode is successfully transitioned to FAILED_OPEN state.
*/
private boolean tryTransitionToFailedOpen(final HRegionInfo hri) {

View File

@ -91,8 +91,7 @@ public class Compressor {
*
* @param in the DataInput to read from
* @param dict the dictionary we use for our read.
*
* @param the uncompressed array.
* @return the uncompressed array.
*/
static byte[] readCompressed(DataInput in, Dictionary dict)
throws IOException {

View File

@ -249,7 +249,6 @@ class FSHLog implements HLog, Syncable {
* @param fs filesystem handle
* @param root path for stored and archived hlogs
* @param logName dir where hlogs are stored
* @param oldLogName dir where hlogs are archived
* @param conf configuration to use
* @param listeners Listeners on WAL events. Listeners passed here will
* be registered before we do anything else; e.g. the
@ -275,7 +274,7 @@ class FSHLog implements HLog, Syncable {
*
* @param fs filesystem handle
* @param root path to where logs and oldlogs
* @param oldLogDir path to where hlogs are archived
* @param oldLogName path to where hlogs are archived
* @param conf configuration to use
* @param listeners Listeners on WAL events. Listeners passed here will
* be registered before we do anything else; e.g. the
@ -943,7 +942,7 @@ class FSHLog implements HLog, Syncable {
private final long optionalFlushInterval;
private AtomicBoolean closeLogSyncer = new AtomicBoolean(false);
private final AtomicBoolean closeLogSyncer = new AtomicBoolean(false);
// List of pending writes to the HLog. There corresponds to transactions
// that have not yet returned to the client. We keep them cached here

View File

@ -114,7 +114,7 @@ public class HLogSplitter {
// Wait/notify for when data has been produced by the reader thread,
// consumed by the reader thread, or an exception occurred
Object dataAvailable = new Object();
final Object dataAvailable = new Object();
private MonitoredTask status;
@ -692,7 +692,7 @@ public class HLogSplitter {
* @param conf
* @return A new Reader instance
* @throws IOException
* @throws CorruptedLogFile
* @throws CorruptedLogFileException
*/
protected Reader getReader(FileSystem fs, FileStatus file, Configuration conf,
boolean skipErrors)

View File

@ -51,6 +51,7 @@ public class WALCoprocessorHost
/**
* Constructor
* @param implClass - not used
* @param impl the coprocessor instance
* @param priority chaining priority
* @param seq load sequence

View File

@ -64,7 +64,7 @@ public class RESTServlet implements Constants {
/**
* Constructor with existing configuration
* @param conf existing configuration
* @throws IOException.
* @throws IOException
*/
RESTServlet(Configuration conf) throws IOException {
this.conf = conf;

View File

@ -146,13 +146,13 @@ public class ZKPermissionWatcher extends ZooKeeperListener {
* @param tableName
* @param permsData
*/
public void writeToZookeeper(byte[] tableName, byte[] parmsData) {
public void writeToZookeeper(byte[] tableName, byte[] permsData) {
String zkNode = ZKUtil.joinZNode(watcher.baseZNode, ACL_NODE);
zkNode = ZKUtil.joinZNode(zkNode, Bytes.toString(tableName));
try {
ZKUtil.createWithParents(watcher, zkNode);
ZKUtil.updateExistingNodeData(watcher, zkNode, parmsData, -1);
ZKUtil.updateExistingNodeData(watcher, zkNode, permsData, -1);
} catch (KeeperException e) {
LOG.error("Failed updating permissions for table '" +
Bytes.toString(tableName) + "'", e);

View File

@ -49,9 +49,7 @@ public class ChecksumFactory {
/**
* Returns a Constructor that can be used to create a Checksum object.
* @return The Constructor that can be used to create a
* new Checksum object.
* @param theClass classname for which an constructor is created
* @param className classname for which an constructor is created
* @return a new Constructor object
*/
static public Constructor<?> newConstructor(String className)

View File

@ -166,7 +166,7 @@ public enum ChecksumType {
/**
* Map a checksum name to a specific type.
* Do our own names.
* @param b
* @param name
* @return Type associated with passed code.
*/
public static ChecksumType nameToType(final String name) {

View File

@ -127,8 +127,7 @@ public abstract class FSUtils {
* <ol>
* <li>overwrite the file if it exists</li>
* <li>apply the umask in the configuration (if it is enabled)</li>
* <li>use the fs configured buffer size (or {@value DEFAULT_BUFFER_SIZE} if
* not set)</li>
* <li>use the fs configured buffer size (or 4096 if not set)</li>
* <li>use the default replication</li>
* <li>use the default block size</li>
* <li>not track progress</li>
@ -148,8 +147,7 @@ public abstract class FSUtils {
* Create the specified file on the filesystem. By default, this will:
* <ol>
* <li>apply the umask in the configuration (if it is enabled)</li>
* <li>use the fs configured buffer size (or {@value DEFAULT_BUFFER_SIZE} if
* not set)</li>
* <li>use the fs configured buffer size (or 4096 if not set)</li>
* <li>use the default replication</li>
* <li>use the default block size</li>
* <li>not track progress</li>

View File

@ -83,7 +83,7 @@ public class HBaseFsckRepair {
*
* This assumes that info is in META.
*
* @param conf
* @param admin
* @param region
* @throws IOException
* @throws KeeperException

View File

@ -42,7 +42,7 @@ public class HFileArchiveUtil {
* @param conf {@link Configuration} to read for the archive directory name
* @param tableName table name under which the store currently lives
* @param regionName region encoded name under which the store currently lives
* @param family name of the family in the store
* @param familyName name of the family in the store
* @return {@link Path} to the directory to archive the given store or
* <tt>null</tt> if it should not be archived
*/

View File

@ -145,7 +145,7 @@ public class RegionSplitter {
* {@link HexStringSplit} to partition their table and set it as default, but
* provided this for your custom algorithm. To use, create a new derived class
* from this interface and call {@link RegionSplitter#createPresplitTable} or
* {@link RegionSplitter#rollingSplit(String, String, Configuration)} with the
* {@link RegionSplitter#rollingSplit(String, SplitAlgorithm, Configuration)} with the
* argument splitClassName giving the name of your class.
*/
public static interface SplitAlgorithm {

View File

@ -48,7 +48,7 @@ import org.apache.hadoop.classification.InterfaceStability;
public class SoftValueSortedMap<K,V> implements SortedMap<K,V> {
private final SortedMap<K, SoftValue<K,V>> internalMap;
private final ReferenceQueue<V> rq = new ReferenceQueue<V>();
private Object sync;
private final Object sync;
/** Constructor */
public SoftValueSortedMap() {

View File

@ -66,7 +66,6 @@ public class Writables {
* @return The bytes of <code>w</code> gotten by running its
* {@link Writable#write(java.io.DataOutput)} method.
* @throws IOException e
* @see #getHRegionInfos(byte[], int, int)
*/
public static byte [] getBytes(final Writable... ws) throws IOException {
List<byte []> bytes = new ArrayList<byte []>();

View File

@ -49,7 +49,7 @@ public class DrainingServerTracker extends ZooKeeperListener {
private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class);
private ServerManager serverManager;
private NavigableSet<ServerName> drainingServers = new TreeSet<ServerName>();
private final NavigableSet<ServerName> drainingServers = new TreeSet<ServerName>();
private Abortable abortable;
public DrainingServerTracker(ZooKeeperWatcher watcher,

View File

@ -451,8 +451,6 @@ public class ZKAssign {
ZKUtil.deleteChildrenRecursively(zkw, zkw.assignmentZNode);
}
// RegionServer methods
/**
* Creates a new unassigned node in the CLOSING state for the specified
* region.
@ -463,10 +461,10 @@ public class ZKAssign {
* <p>If creation is successful, returns the version number of the CLOSING
* node created.
*
* <p>Does not set any watches.
* <p>Set a watch.
*
* <p>This method should only be used by a RegionServer when initiating a
* close of a region after receiving a CLOSE RPC from the Master.
* <p>This method should only be used by a Master when initiating a
* close of a region before sending a close request to the region server.
*
* @param zkw zk reference
* @param region region to be created as closing
@ -486,6 +484,8 @@ public class ZKAssign {
return ZKUtil.createAndWatch(zkw, node, rt.toByteArray());
}
// RegionServer methods
/**
* Transitions an existing unassigned node for the specified region which is
* currently in the CLOSING state to be in the CLOSED state.

View File

@ -43,8 +43,8 @@ public class ZKTableReadOnly {
private ZKTableReadOnly() {}
/**
* Go to zookeeper and see if state of table is {@link TableState#DISABLED}.
* This method does not use cache as {@link #isDisabledTable(String)} does.
* Go to zookeeper and see if state of table is {@link ZooKeeperProtos.Table.State#DISABLED}.
* This method does not use cache.
* This method is for clients other than {@link AssignmentManager}
* @param zkw
* @param tableName
@ -59,8 +59,8 @@ public class ZKTableReadOnly {
}
/**
* Go to zookeeper and see if state of table is {@link TableState#ENABLED}.
* This method does not use cache as {@link #isEnabledTable(String)} does.
* Go to zookeeper and see if state of table is {@link ZooKeeperProtos.Table.State#ENABLED}.
* This method does not use cache.
* This method is for clients other than {@link AssignmentManager}
* @param zkw
* @param tableName
@ -74,9 +74,9 @@ public class ZKTableReadOnly {
}
/**
* Go to zookeeper and see if state of table is {@link TableState#DISABLING}
* of {@link TableState#DISABLED}.
* This method does not use cache as {@link #isEnabledTable(String)} does.
* Go to zookeeper and see if state of table is {@link ZooKeeperProtos.Table.State#DISABLING}
* of {@link ZooKeeperProtos.Table.State#DISABLED}.
* This method does not use cache.
* This method is for clients other than {@link AssignmentManager}.
* @param zkw
* @param tableName
@ -135,7 +135,7 @@ public class ZKTableReadOnly {
/**
* @param zkw
* @param child
* @return Null or {@link TableState} found in znode.
* @return Null or {@link ZooKeeperProtos.Table.State} found in znode.
* @throws KeeperException
*/
static ZooKeeperProtos.Table.State getTableState(final ZooKeeperWatcher zkw,