HBASE-16084 Cleaned up the stale references in Javadoc

Signed-off-by: tedyu <yuzhihong@gmail.com>
This commit is contained in:
Jan Hentschel 2017-03-19 20:49:28 +01:00 committed by tedyu
parent 4088f822a4
commit 55d6dcaf87
48 changed files with 86 additions and 100 deletions

View File

@ -723,7 +723,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* This sets the class associated with the region split policy which
* determines when a region split should occur. The class used by
* default is defined in {@link org.apache.hadoop.hbase.regionserver.RegionSplitPolicy}
* default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
* @param clazz the class name
*/
public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
@ -734,7 +734,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* This gets the class associated with the region split policy which
* determines when a region split should occur. The class used by
* default is defined in {@link org.apache.hadoop.hbase.regionserver.RegionSplitPolicy}
* default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
*
* @return the class name of the region split policy for this table.
* If this returns null, the default split policy is used.
@ -827,7 +827,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* This sets the class associated with the flush policy which determines determines the stores
* need to be flushed when flushing a region. The class used by default is defined in
* {@link org.apache.hadoop.hbase.regionserver.FlushPolicy}
* org.apache.hadoop.hbase.regionserver.FlushPolicy.
* @param clazz the class name
*/
public HTableDescriptor setFlushPolicyClassName(String clazz) {
@ -838,7 +838,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* This gets the class associated with the flush policy which determines the stores need to be
* flushed when flushing a region. The class used by default is defined in
* {@link org.apache.hadoop.hbase.regionserver.FlushPolicy}
* org.apache.hadoop.hbase.regionserver.FlushPolicy.
* @return the class name of the flush policy for this table. If this returns null, the default
* flush policy is used.
*/
@ -1244,7 +1244,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* Add a table coprocessor to this table. The coprocessor
* type must be {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
* type must be org.apache.hadoop.hbase.coprocessor.RegionObserver
* or Endpoint.
* It won't check if the class can be loaded or not.
* Whether a coprocessor is loadable or not will be determined when
@ -1259,7 +1259,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* Add a table coprocessor to this table. The coprocessor
* type must be {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
* type must be org.apache.hadoop.hbase.coprocessor.RegionObserver
* or Endpoint.
* It won't check if the class can be loaded or not.
* Whether a coprocessor is loadable or not will be determined when
@ -1304,7 +1304,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* Add a table coprocessor to this table. The coprocessor
* type must be {@link org.apache.hadoop.hbase.coprocessor.RegionObserver}
* type must be org.apache.hadoop.hbase.coprocessor.RegionObserver
* or Endpoint.
* It won't check if the class can be loaded or not.
* Whether a coprocessor is loadable or not will be determined when

View File

@ -452,7 +452,7 @@ class AsyncProcess {
/**
* Only used w/useGlobalErrors ctor argument, for HTable backward compat.
* @return Whether there were any errors in any request since the last time
* {@link #waitForAllPreviousOpsAndReset(List, String)} was called, or AP was created.
* {@link #waitForAllPreviousOpsAndReset(List, TableName)} was called, or AP was created.
*/
public boolean hasError() {
return globalErrors != null && globalErrors.hasErrors();
@ -463,9 +463,9 @@ class AsyncProcess {
* Waits for all previous operations to finish, and returns errors and (optionally)
* failed operations themselves.
* @param failedRows an optional list into which the rows that failed since the last time
* {@link #waitForAllPreviousOpsAndReset(List, String)} was called, or AP was created, are saved.
* {@link #waitForAllPreviousOpsAndReset(List, TableName)} was called, or AP was created, are saved.
* @param tableName name of the table
* @return all the errors since the last time {@link #waitForAllPreviousOpsAndReset(List, String)}
* @return all the errors since the last time {@link #waitForAllPreviousOpsAndReset(List, TableName)}
* was called, or AP was created.
*/
public RetriesExhaustedWithDetailsException waitForAllPreviousOpsAndReset(

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.util.ReflectionUtils;
* A non-instantiable class that manages creation of {@link Connection}s. Managing the lifecycle of
* the {@link Connection}s to the cluster is the responsibility of the caller. From a
* {@link Connection}, {@link Table} implementations are retrieved with
* {@link Connection#getTable(TableName)}. Example:
* {@link Connection#getTable(org.apache.hadoop.hbase.TableName)}. Example:
*
* <pre>
* Connection connection = ConnectionFactory.createConnection(config);

View File

@ -80,7 +80,7 @@ abstract class MasterCallable<V> implements RetryingCallable<V>, Closeable {
}
/**
* Override that changes the {@link Callable#call()} Exception from {@link Exception} to
* Override that changes the {@link java.util.concurrent.Callable#call()} Exception from {@link Exception} to
* {@link IOException}. It also does setup of an rpcController and calls through to the rpcCall()
* method which callers are expected to implement. If rpcController is an instance of
* PayloadCarryingRpcController, we will set a timeout on it.

View File

@ -56,7 +56,7 @@ public abstract class Query extends OperationWithAttributes {
/**
* Apply the specified server-side filter when performing the Query.
* Only {@link Filter#filterKeyValue(Cell)} is called AFTER all tests
* Only {@link Filter#filterKeyValue(org.apache.hadoop.hbase.Cell)} is called AFTER all tests
* for ttl, column match, deletes and max versions have been run.
* @param filter filter to run on the server
* @return this for invocation chaining

View File

@ -44,8 +44,8 @@ must:
for more details on defining services.</li>
<li>Generate the Service and Message code using the protoc compiler</li>
<li>Implement the generated Service interface in your coprocessor class and implement the
{@link org.apache.hadoop.hbase.coprocessor.CoprocessorService} interface. The
{@link org.apache.hadoop.hbase.coprocessor.CoprocessorService#getService()}
org.apache.hadoop.hbase.coprocessor.CoprocessorService interface. The
org.apache.hadoop.hbase.coprocessor.CoprocessorService#getService()
method should return a reference to the Endpoint's protocol buffer Service instance.
</ul>
<p>
@ -146,10 +146,10 @@ public static abstract class RowCountService
</pre></blockquote></div>
<p>
Our coprocessor Service will need to implement this interface and the {@link org.apache.hadoop.hbase.coprocessor.CoprocessorService}
Our coprocessor Service will need to implement this interface and the org.apache.hadoop.hbase.coprocessor.CoprocessorService
in order to be registered correctly as an endpoint. For the sake of simplicity the server-side
implementation is omitted. To see the implementing code, please see the
{@link org.apache.hadoop.hbase.coprocessor.example.RowCountEndpoint} class in the HBase source code.
org.apache.hadoop.hbase.coprocessor.example.RowCountEndpoint class in the HBase source code.
</p>
<p>

View File

@ -41,7 +41,7 @@ import org.apache.hadoop.hbase.shaded.com.google.protobuf.UnsafeByteOperations;
* Note : It may emit KVs which do not have the given columns in them, if
* these KVs happen to occur before a KV which does have a match. Given this
* caveat, this filter is only useful for special cases
* like {@link org.apache.hadoop.hbase.mapreduce.RowCounter}.
* like org.apache.hadoop.hbase.mapreduce.RowCounter.
* <p>
* @deprecated Deprecated in 2.0. See HBASE-13347
*/

View File

@ -54,8 +54,8 @@ import org.apache.hadoop.util.StringUtils;
public class ServerRpcController implements RpcController {
/**
* The exception thrown within
* {@link com.google.protobuf.Service#callMethod(
* Descriptors.MethodDescriptor, RpcController, Message, RpcCallback)}
* {@link com.google.protobuf.Service#callMethod(com.google.protobuf.Descriptors.MethodDescriptor, RpcController,
* com.google.protobuf.Message, RpcCallback)}
* if any.
*/
// TODO: it would be good widen this to just Throwable, but IOException is what we allow now

View File

@ -630,7 +630,7 @@ public class RecoverableZooKeeper {
}
}
/**
* Convert Iterable of {@link ZKOp} we got into the ZooKeeper.Op
* Convert Iterable of {@link org.apache.zookeeper.Op} we got into the ZooKeeper.Op
* instances to actually pass to multi (need to do this in order to appendMetaData).
*/
private Iterable<Op> prepareZKMulti(Iterable<Op> ops)

View File

@ -492,7 +492,7 @@ public abstract class ByteBuff {
}
/**
* Similar to {@link WritableUtils#readVLong(DataInput)} but reads from a
* Similar to {@link WritableUtils#readVLong(java.io.DataInput)} but reads from a
* {@link ByteBuff}.
*/
public static long readVLong(ByteBuff in) {

View File

@ -341,7 +341,7 @@ public class OrderedBytes {
/**
* Perform unsigned comparison between two long values. Conforms to the same interface as
* {@link Comparator#compare(Object, Object)}.
* {@link org.apache.hadoop.hbase.CellComparator#COMPARATOR#compare(Object, Object)}.
*/
private static int unsignedCmp(long x1, long x2) {
int cmp;

View File

@ -32,8 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Common helpers for testing HBase that do not depend on specific server/etc. things.
* @see {@link HBaseTestingUtility}
*
* {@see org.apache.hadoop.hbase.HBaseTestingUtility}
*/
@InterfaceAudience.Public
@InterfaceStability.Unstable

View File

@ -71,7 +71,7 @@ public interface CellSearcher extends ReversibleCellScanner {
/**
* <p>
* Note: Added for backwards compatibility with
* {@link org.apache.hadoop.hbase.regionserver.KeyValueScanner#reseek}
* org.apache.hadoop.hbase.regionserver.KeyValueScanner#reseek(Cell)
* </p><p>
* Look for the key, but only look after the current position. Probably not needed for an
* efficient tree implementation, but is important for implementations without random access such

View File

@ -119,8 +119,7 @@ public class ProcedureWALFormatReader {
* purpose. If all procedures updated in a WAL are found to be obsolete, it can be safely deleted.
* (see {@link WALProcedureStore#removeInactiveLogs()}).
* However, we don't need deleted part of a WAL's tracker for this purpose, so we don't bother
* re-building it. (To understand why, take a look at
* {@link ProcedureStoreTracker.BitSetNode#subtract(ProcedureStoreTracker.BitSetNode)}).
* re-building it.
*/
private ProcedureStoreTracker localTracker;

View File

@ -466,8 +466,6 @@ public class HFileArchiver {
* <p>
* A best effort is made to delete each of the files, rather than bailing on the first failure.
* <p>
* This method is preferable to {@link #deleteFilesWithoutArchiving(Collection)} since it consumes
* less resources, but is limited in terms of usefulness
* @param compactedFiles store files to delete from the file system.
* @throws IOException if a file cannot be deleted. All files will be attempted to deleted before
* throwing the exception, rather than failing at the first file.

View File

@ -67,7 +67,8 @@ class HFileArchiveManager {
/**
* Stop retaining HFiles for the given table in the archive. HFiles will be cleaned up on the next
* pass of the {@link HFileCleaner}, if the HFiles are retained by another cleaner.
* pass of the {@link org.apache.hadoop.hbase.master.cleaner.HFileCleaner}, if the HFiles are retained by another
* cleaner.
* @param table name of the table for which to disable hfile retention.
* @return <tt>this</tt> for chaining.
* @throws KeeperException if if we can't reach zookeeper to update the hfile cleaner.

View File

@ -283,7 +283,7 @@ public class RestoreTool {
/**
* Duplicate the backup image if it's on local cluster
* @see HStore#bulkLoadHFile(String, long)
* @see HStore#bulkLoadHFile(org.apache.hadoop.hbase.regionserver.StoreFile)
* @see HRegionFileSystem#bulkLoadStoreFile(String familyName, Path srcPath, long seqNum)
* @param tableArchivePath archive path
* @return the new tableArchivePath
@ -554,7 +554,7 @@ public class RestoreTool {
/**
* Prepare the table for bulkload, most codes copied from
* {@link LoadIncrementalHFiles#createTable(String, String)}
* {@link LoadIncrementalHFiles#createTable(TableName, String, Admin)}
* @param conn connection
* @param tableBackupPath path
* @param tableName table name

View File

@ -27,15 +27,15 @@ import org.apache.hadoop.hbase.client.Put;
* any order.
* <p>
* A {@link Constraint} must be added to a table before the table is loaded via
* {@link Constraints#add(HTableDescriptor, Class...)} or
* {@link Constraints#add(HTableDescriptor,
* {@link Constraints#add(org.apache.hadoop.hbase.HTableDescriptor, Class[])} or
* {@link Constraints#add(org.apache.hadoop.hbase.HTableDescriptor,
* org.apache.hadoop.hbase.util.Pair...)}
* (if you want to add a configuration with the {@link Constraint}). Constraints
* will be run in the order that they are added. Further, a Constraint will be
* configured before it is run (on load).
* <p>
* See {@link Constraints#enableConstraint(HTableDescriptor, Class)} and
* {@link Constraints#disableConstraint(HTableDescriptor, Class)} for
* See {@link Constraints#enableConstraint(org.apache.hadoop.hbase.HTableDescriptor, Class)} and
* {@link Constraints#disableConstraint(org.apache.hadoop.hbase.HTableDescriptor, Class)} for
* enabling/disabling of a given {@link Constraint} after it has been added.
* <p>
* If a {@link Put} is invalid, the Constraint should throw some sort of

View File

@ -248,7 +248,7 @@ public class CompoundBloomFilterWriter extends CompoundBloomFilterBase
}
/**
* This is modeled after {@link BloomFilterChunk.MetaWriter} for simplicity,
* This is modeled after {@link CompoundBloomFilterWriter.MetaWriter} for simplicity,
* although the two metadata formats do not have to be consistent. This
* does have to be consistent with how {@link
* CompoundBloomFilter#CompoundBloomFilter(DataInput,

View File

@ -697,7 +697,7 @@ public class HFileBlock implements Cacheable {
}
/**
* Cannot be {@link #UNSET}. Must be a legitimate value. Used re-making the {@link CacheKey} when
* Cannot be {@link #UNSET}. Must be a legitimate value. Used re-making the {@link BlockCacheKey} when
* block is returned to the cache.
* @return the offset of this block in the file it was read from
*/

View File

@ -61,7 +61,7 @@ import org.apache.hadoop.util.StringUtils;
* {@link org.apache.hadoop.hbase.io.hfile.CompoundBloomFilterWriter} and
* {@link HFileWriterImpl}. Examples of how to use the reader can be
* found in {@link HFileReaderImpl} and
* {@link org.apache.hadoop.hbase.io.hfile.TestHFileBlockIndex}.
* org.apache.hadoop.hbase.io.hfile.TestHFileBlockIndex.
*/
@InterfaceAudience.Private
public class HFileBlockIndex {

View File

@ -76,8 +76,7 @@ public class RowCounter extends Configured implements Tool {
* @param values The columns.
* @param context The current context.
* @throws IOException When something is broken with the data.
* @see org.apache.hadoop.mapreduce.Mapper#map(KEYIN, VALUEIN,
* org.apache.hadoop.mapreduce.Mapper.Context)
* @see org.apache.hadoop.mapreduce.Mapper#map(Object, Object, Context)
*/
@Override
public void map(ImmutableBytesWritable row, Result values,

View File

@ -212,7 +212,7 @@ public class TableNamespaceManager {
/**
* Create Namespace in a blocking manner. Keeps trying until
* {@link ClusterSchema.HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires.
* {@link ClusterSchema#HBASE_MASTER_CLUSTER_SCHEMA_OPERATION_TIMEOUT_KEY} expires.
* Note, by-passes notifying coprocessors and name checks. Use for system namespaces only.
*/
private void blockingCreateNamespace(final NamespaceDescriptor namespaceDescriptor)

View File

@ -980,8 +980,8 @@ public class StochasticLoadBalancer extends BaseLoadBalancer {
abstract double cost();
/**
* Function to compute a scaled cost using {@link DescriptiveStatistics}. It
* assumes that this is a zero sum set of costs. It assumes that the worst case
* Function to compute a scaled cost using {@link org.apache.commons.math3.stat.descriptive.DescriptiveStatistics}.
* It assumes that this is a zero sum set of costs. It assumes that the worst case
* possible is all of the elements in one region server and the rest having 0.
*
* @param stats the costs

View File

@ -25,9 +25,9 @@ import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
* Wraps together the mutations which are applied as a batch to the region and their operation
* status and WALEdits.
* @see org.apache.hadoop.hbase.coprocessor.RegionObserver#preBatchMutate(
* ObserverContext, MiniBatchOperationInProgress)
* org.apache.hadoop.hbase.coprocessor.ObserverContext, MiniBatchOperationInProgress)
* @see org.apache.hadoop.hbase.coprocessor.RegionObserver#postBatchMutate(
* ObserverContext, MiniBatchOperationInProgress)
* org.apache.hadoop.hbase.coprocessor.ObserverContext, MiniBatchOperationInProgress)
* @param T Pair&lt;Mutation, Integer&gt; pair of Mutations and associated rowlock ids .
*/
@InterfaceAudience.LimitedPrivate("Coprocessors")

View File

@ -208,7 +208,7 @@ public class StoreFileReader {
* Checks whether the given scan passes the Bloom filter (if present). Only
* checks Bloom filters for single-row or single-row-column scans. Bloom
* filter checking for multi-gets is implemented as part of the store
* scanner system (see {@link StoreFileScanner#seekExactly}) and uses
* scanner system (see {@link StoreFileScanner#seek(Cell)} and uses
* the lower-level API {@link #passesGeneralRowBloomFilter(byte[], int, int)}
* and {@link #passesGeneralRowColBloomFilter(Cell)}.
*

View File

@ -244,8 +244,8 @@ public class FSHLog extends AbstractFSWAL<Writer> {
/**
* Currently, we need to expose the writer's OutputStream to tests so that they can manipulate the
* default behavior (such as setting the maxRecoveryErrorCount value for example (see
* {@link AbstractTestWALReplay#testReplayEditsWrittenIntoWAL()}). This is done using reflection
* on the underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1 support is
* {@see org.apache.hadoop.hbase.regionserver.wal.AbstractTestWALReplay#testReplayEditsWrittenIntoWAL()}). This is
* done using reflection on the underlying HDFS OutputStream. NOTE: This could be removed once Hadoop1 support is
* removed.
* @return null if underlying stream is not ready.
*/
@ -809,9 +809,9 @@ public class FSHLog extends AbstractFSWAL<Writer> {
* To start up the drama, Thread A creates an instance of this class each time it would do this
* zigzag dance and passes it to Thread B (these classes use Latches so it is one shot only).
* Thread B notices the new instance (via reading a volatile reference or how ever) and it starts
* to work toward the 'safe point'. Thread A calls {@link #waitSafePoint()} when it cannot proceed
* to work toward the 'safe point'. Thread A calls {@link #waitSafePoint(SyncFuture)} when it cannot proceed
* until the Thread B 'safe point' is attained. Thread A will be held inside in
* {@link #waitSafePoint()} until Thread B reaches the 'safe point'. Once there, Thread B frees
* {@link #waitSafePoint(SyncFuture)} until Thread B reaches the 'safe point'. Once there, Thread B frees
* Thread A by calling {@link #safePointAttained()}. Thread A now knows Thread B is at the 'safe
* point' and that it is holding there (When Thread B calls {@link #safePointAttained()} it blocks
* here until Thread A calls {@link #releaseSafePoint()}). Thread A proceeds to do what it needs

View File

@ -61,7 +61,7 @@ class SequenceIdAccounting {
* {@link #flushingSequenceIds}.
*
* <p>The two Maps are tied by this locking object EXCEPT when we go to update the lowest
* entry; see {@link #lowest(byte[], Set, Long)}. In here is a putIfAbsent call on
* entry; see {@link #lowestUnflushedSequenceIds}. In here is a putIfAbsent call on
* {@link #lowestUnflushedSequenceIds}. In this latter case, we will add this lowest
* sequence id if we find that there is no entry for the current column family. There will be no
* entry only if we just came up OR we have moved aside current set of lowest sequence ids
@ -403,8 +403,8 @@ class SequenceIdAccounting {
/**
* Iterates over the given Map and compares sequence ids with corresponding entries in
* {@link #oldestUnflushedRegionSequenceIds}. If a region in
* {@link #oldestUnflushedRegionSequenceIds} has a sequence id less than that passed in
* {@link #lowestUnflushedSequenceIds}. If a region in
* {@link #lowestUnflushedSequenceIds} has a sequence id less than that passed in
* <code>sequenceids</code> then return it.
* @param sequenceids Sequenceids keyed by encoded region name.
* @return regions found in this instance with sequence ids less than those passed in.

View File

@ -77,7 +77,7 @@ class SyncFuture {
* Call this method to clear old usage and get it ready for new deploy.
* @param txid the new transaction id
* @param span current span, detached from caller. Don't forget to attach it when resuming after a
* call to {@link #get()}.
* call to {@link #get(long)}.
* @return this
*/
synchronized SyncFuture reset(final long txid, Span span) {
@ -107,7 +107,7 @@ class SyncFuture {
/**
* Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue
* the span. Thread waiting on this Future musn't call this method until AFTER calling
* {@link #get()} and the future has been released back to the originating thread.
* {@link #get(long)} and the future has been released back to the originating thread.
*/
synchronized Span getSpan() {
return this.span;

View File

@ -75,7 +75,7 @@ public class CoprocessorWhitelistMasterObserver implements MasterObserver {
* 1) a "*" to wildcard all coprocessor paths
* 2) a specific filesystem (e.g. hdfs://my-cluster/)
* 3) a wildcard path to be evaluated by
* {@link FilenameUtils.wildcardMatch}
* {@link FilenameUtils#wildcardMatch(String, String)}
* path can specify scheme or not (e.g.
* "file:///usr/hbase/coprocessors" or for all
* filesystems "/usr/hbase/coprocessors")

View File

@ -85,7 +85,7 @@ public class TableAuthManager implements Closeable {
/**
* Returns a combined map of user and group permissions, with group names
* distinguished according to {@link AuthUtil.isGroupPrincipal}
* distinguished according to {@link AuthUtil#isGroupPrincipal(String)}.
*/
public ListMultimap<String,T> getAllPermissions() {
ListMultimap<String,T> tmp = ArrayListMultimap.create();

View File

@ -2059,7 +2059,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
}
/** A tracker for tracking and validating table rows
* generated with {@link HBaseTestingUtility#loadTable(HTable, byte[])}
* generated with {@link HBaseTestingUtility#loadTable(Table, byte[])}
*/
public static class SeenRowTracker {
int dim = 'z' - 'a' + 1;
@ -2326,7 +2326,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
return digest.toString();
}
/** All the row values for the data loaded by {@link #loadTable(HTable, byte[])} */
/** All the row values for the data loaded by {@link #loadTable(Table, byte[])} */
public static final byte[][] ROWS = new byte[(int) Math.pow('z' - 'a' + 1, 3)][3]; // ~52KB
static {
int i = 0;
@ -3110,7 +3110,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility {
/**
* Waits for a table to be 'enabled'. Enabled means that table is set as 'enabled' and the
* regions have been all assigned.
* @see #waitTableEnabled(Admin, byte[], long)
* @see #waitTableEnabled(TableName, long)
* @param table Table to wait on.
* @param timeoutMillis Time to wait on it being marked enabled.
* @throws InterruptedException

View File

@ -301,12 +301,12 @@ public class TestMetaTableLocator {
* want to pass a mocked HRS; can be null.
* @param client A mocked ClientProtocol instance, can be null
* @return Mock up a connection that returns a {@link Configuration} when
* {@link HConnection#getConfiguration()} is called, a 'location' when
* {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
* {@link org.apache.hadoop.hbase.client.ClusterConnection#getConfiguration()} is called, a 'location' when
* {@link org.apache.hadoop.hbase.client.RegionLocator#getRegionLocation(byte[], boolean)} is called,
* and that returns the passed {@link AdminProtos.AdminService.BlockingInterface} instance when
* {@link HConnection#getAdmin(ServerName)} is called, returns the passed
* {@link org.apache.hadoop.hbase.client.ClusterConnection#getAdmin(ServerName)} is called, returns the passed
* {@link ClientProtos.ClientService.BlockingInterface} instance when
* {@link HConnection#getClient(ServerName)} is called.
* {@link org.apache.hadoop.hbase.client.ClusterConnection#getClient(ServerName)} is called.
* @throws IOException
*/
private ClusterConnection mockConnection(final AdminProtos.AdminService.BlockingInterface admin,

View File

@ -447,7 +447,7 @@ public class TestPartialResultsFromClientSide {
}
/**
* Test the method {@link Result#createCompleteResult(List)}
* Test the method {@link Result#createCompleteResult(Iterable)}
* @throws Exception
*/
@Test

View File

@ -571,7 +571,7 @@ public class TestSerialization {
protected static final byte [][] COLUMNS = {fam1, fam2, fam3};
/**
* Create a table of name <code>name</code> with {@link COLUMNS} for
* Create a table of name <code>name</code> with {@link #COLUMNS} for
* families.
* @param name Name to give table.
* @return Column descriptor.
@ -581,7 +581,7 @@ public class TestSerialization {
}
/**
* Create a table of name <code>name</code> with {@link COLUMNS} for
* Create a table of name <code>name</code> with {@link #COLUMNS} for
* families.
* @param name Name to give table.
* @param versions How many versions to allow per column.

View File

@ -41,7 +41,7 @@ import org.junit.experimental.categories.Category;
import org.junit.rules.TestName;
/**
* Run tests related to {@link TimestampsFilter} using HBase client APIs.
* Run tests related to {@link org.apache.hadoop.hbase.filter.TimestampsFilter} using HBase client APIs.
* Sets up the HBase mini cluster once at start. Each creates a table
* named for the method and does its stuff against that.
*/

View File

@ -40,7 +40,7 @@ public class TestFirstKeyValueMatchingQualifiersFilter extends TestCase {
/**
* Test the functionality of
* {@link FirstKeyValueMatchingQualifiersFilter#filterKeyValue(KeyValue)}
* {@link FirstKeyValueMatchingQualifiersFilter#filterKeyValue(org.apache.hadoop.hbase.Cell)}
*
* @throws Exception
*/

View File

@ -55,7 +55,7 @@ import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
/**
* Test {@link HFileScanner#seekTo(byte[])} and its variants.
* Test {@link HFileScanner#seekTo(Cell)} and its variants.
*/
@Category({IOTests.class, SmallTests.class})
@RunWith(Parameterized.class)

View File

@ -113,7 +113,7 @@ import org.junit.rules.TestRule;
import org.mockito.Mockito;
/**
* Simple test for {@link CellSortReducer} and {@link HFileOutputFormat2}.
* Simple test for {@link HFileOutputFormat2}.
* Sets up and runs a mapreduce job that writes hfile output.
* Creates a few inner classes to implement splits and an inputformat that
* emits keys and values like those of {@link PerformanceEvaluation}.
@ -684,9 +684,8 @@ public class TestHFileOutputFormat2 {
}
/**
* Test for {@link HFileOutputFormat2#configureCompression(org.apache.hadoop.hbase.client.Table,
* Configuration)} and {@link HFileOutputFormat2#createFamilyCompressionMap
* (Configuration)}.
* Test for {@link HFileOutputFormat2#configureCompression(Configuration, HTableDescriptor)} and
* {@link HFileOutputFormat2#createFamilyCompressionMap(Configuration)}.
* Tests that the compression map is correctly serialized into
* and deserialized from configuration
*
@ -754,9 +753,8 @@ public class TestHFileOutputFormat2 {
/**
* Test for {@link HFileOutputFormat2#configureBloomType(org.apache.hadoop.hbase.client.Table,
* Configuration)} and {@link HFileOutputFormat2#createFamilyBloomTypeMap
* (Configuration)}.
* Test for {@link HFileOutputFormat2#configureBloomType(HTableDescriptor, Configuration)} and
* {@link HFileOutputFormat2#createFamilyBloomTypeMap(Configuration)}.
* Tests that the compression map is correctly serialized into
* and deserialized from configuration
*
@ -824,9 +822,8 @@ public class TestHFileOutputFormat2 {
}
/**
* Test for {@link HFileOutputFormat2#configureBlockSize(org.apache.hadoop.hbase.client.Table,
* Configuration)} and {@link HFileOutputFormat2#createFamilyBlockSizeMap
* (Configuration)}.
* Test for {@link HFileOutputFormat2#configureBlockSize(HTableDescriptor, Configuration)} and
* {@link HFileOutputFormat2#createFamilyBlockSizeMap(Configuration)}.
* Tests that the compression map is correctly serialized into
* and deserialized from configuration
*

View File

@ -700,7 +700,7 @@ public class TestImportExport {
}
/**
* This listens to the {@link #visitLogEntryBeforeWrite(HTableDescriptor, WALKey, WALEdit)} to
* This listens to the {@link #visitLogEntryBeforeWrite(HRegionInfo, WALKey, WALEdit)} to
* identify that an entry is written to the Write Ahead Log for the given table.
*/
private static class TableWALActionListener extends WALActionsListener.Base {

View File

@ -139,7 +139,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
/**
* Map of regions to map of rows and {@link Result}. Used as data source when
* {@link MockRegionServer#get(byte[], Get)} is called. Because we have a byte
* {@link #get(RpcController, GetRequest)} is called. Because we have a byte
* key, need to use TreeMap and provide a Comparator. Use
* {@link #setGetResult(byte[], byte[], Result)} filling this map.
*/
@ -190,7 +190,7 @@ ClientProtos.ClientService.BlockingInterface, RegionServerServices {
}
/**
* Use this method filling the backing data source used by {@link #get(byte[], Get)}
* Use this method filling the backing data source used by {@link #get(RpcController, GetRequest)}
* @param regionName
* @param row
* @param r

View File

@ -46,7 +46,7 @@ import org.junit.After;
import org.junit.Test;
/**
* Run tests that use the HBase clients; {@link HTable}.
* Run tests that use the HBase clients; {@link org.apache.hadoop.hbase.client.HTable}.
* Sets up the HBase mini cluster once at start and runs through all client tests.
* Each creates a table named for the method and does its stuff against that.
*/

View File

@ -224,8 +224,8 @@ public class TestProcedureMember {
/**
* Fail correctly if coordinator aborts the procedure. The subprocedure will not interrupt a
* running {@link Subprocedure#prepare} -- prepare needs to finish first, and the the abort
* is checked. Thus, the {@link Subprocedure#prepare} should succeed but later get rolled back
* running {@link Subprocedure#acquireBarrier()} -- prepare needs to finish first, and the the abort
* is checked. Thus, the {@link Subprocedure#acquireBarrier()} should succeed but later get rolled back
* via {@link Subprocedure#cleanup}.
*/
@Test(timeout = 60000)

View File

@ -69,7 +69,7 @@ public class DataBlockEncodingTool {
/**
* How many times to run the benchmark. More times means better data in terms
* of statistics but slower execution. Has to be strictly larger than
* {@link DEFAULT_BENCHMARK_N_OMIT}.
* {@link #DEFAULT_BENCHMARK_N_OMIT}.
*/
private static final int DEFAULT_BENCHMARK_N_TIMES = 12;

View File

@ -24,7 +24,6 @@ import java.util.List;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.client.Durability;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
@ -32,7 +31,7 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.MutateRequ
/**
* A region server that will OOME.
* Everytime {@link #put(regionName, Durability)} is called, we add
* Everytime {@link #put(byte[], Put)} is called, we add
* keep around a reference to the batch. Use this class to test OOME extremes.
* Needs to be started manually as in
* <code>${HBASE_HOME}/bin/hbase ./bin/hbase org.apache.hadoop.hbase.OOMERegionServer start</code>.

View File

@ -88,12 +88,6 @@ import com.google.common.base.Joiner;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.RpcController;
import org.apache.hadoop.hbase.shaded.com.google.protobuf.ServiceException;
/**
* Like {@link TestRegionMergeTransaction} in that we're testing
* {@link RegionMergeTransactionImpl} only the below tests are against a running
* cluster where {@link TestRegionMergeTransaction} is tests against bare
* {@link HRegion}.
*/
@Category({RegionServerTests.class, MediumTests.class})
public class TestRegionMergeTransactionOnCluster {
private static final Log LOG = LogFactory

View File

@ -64,7 +64,7 @@ import org.apache.hadoop.util.ToolRunner;
/**
* A command-line utility that reads, writes, and verifies data. Unlike
* {@link PerformanceEvaluation}, this tool validates the data written,
* {@link org.apache.hadoop.hbase.PerformanceEvaluation}, this tool validates the data written,
* and supports simultaneously writing and reading the same set of keys.
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)

View File

@ -78,8 +78,8 @@ public abstract class MultiThreadedAction {
* Default implementation of LoadTestDataGenerator that uses LoadTestKVGenerator, fixed
* set of column families, and random number of columns in range. The table for it can
* be created manually or, for example, via
* {@link HBaseTestingUtility#createPreSplitLoadTestTable(
* org.apache.hadoop.hbase.Configuration, byte[], byte[], Algorithm, DataBlockEncoding)}
* {@link org.apache.hadoop.hbase.HBaseTestingUtility#createPreSplitLoadTestTable(Configuration, TableName, byte[],
* org.apache.hadoop.hbase.io.compress.Compression.Algorithm, org.apache.hadoop.hbase.io.encoding.DataBlockEncoding)}
*/
public static class DefaultDataGenerator extends LoadTestDataGenerator {
private byte[][] columnFamilies = null;