HBASE-13569 Correct Javadoc (for Java8)

Signed-off-by: Sean Busbey <busbey@apache.org>
This commit is contained in:
Gábor Lipták 2015-04-26 21:07:45 -04:00 committed by Sean Busbey
parent 293506c7cd
commit 682b8ab8a5
158 changed files with 510 additions and 458 deletions

View File

@ -85,8 +85,8 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
/**
* Key for cache data into L1 if cache is set up with more than one tier.
* To set in the shell, do something like this:
* <code>hbase(main):003:0> create 't',
* {NAME => 't', CONFIGURATION => {CACHE_DATA_IN_L1 => 'true'}}</code>
* <code>hbase(main):003:0&gt; create 't',
* {NAME =&gt; 't', CONFIGURATION =&gt; {CACHE_DATA_IN_L1 =&gt; 'true'}}</code>
*/
public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
@ -115,7 +115,7 @@ public class HColumnDescriptor implements Comparable<HColumnDescriptor> {
/**
* Retain all cells across flushes and compactions even if they fall behind
* a delete tombstone. To see all retained cells, do a 'raw' scan; see
* Scan#setRaw or pass RAW => true attribute in the shell.
* Scan#setRaw or pass RAW =&gt; true attribute in the shell.
*/
public static final String KEEP_DELETED_CELLS = "KEEP_DELETED_CELLS";
public static final String COMPRESS_TAGS = "COMPRESS_TAGS";

View File

@ -54,17 +54,21 @@ import com.google.protobuf.InvalidProtocolBufferException;
* about the region.
*
* The region has a unique name which consists of the following fields:
* <ul>
* <li> tableName : The name of the table </li>
* <li> startKey : The startKey for the region. </li>
* <li> regionId : A timestamp when the region is created. </li>
* <li> replicaId : An id starting from 0 to differentiate replicas of the same region range
* but hosted in separated servers. The same region range can be hosted in multiple locations.</li>
* <li> encodedName : An MD5 encoded string for the region name.</li>
* </ul>
*
* <br> Other than the fields in the region name, region info contains:
* <ul>
* <li> endKey : the endKey for the region (exclusive) </li>
* <li> split : Whether the region is split </li>
* <li> offline : Whether the region is offline </li>
* </ul>
*
* In 0.98 or before, a list of table's regions would fully cover the total keyspace, and at any
* point in time, a row key always belongs to a single region, which is hosted in a single server.
@ -647,7 +651,7 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* by this region. For example, if the region is foo,a,g and this is
* passed ["b","c"] or ["a","c"] it will return true, but if this is passed
* ["b","z"] it will return false.
* @throws IllegalArgumentException if the range passed is invalid (ie end < start)
* @throws IllegalArgumentException if the range passed is invalid (ie. end &lt; start)
*/
public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
@ -1098,7 +1102,6 @@ public class HRegionInfo implements Comparable<HRegionInfo> {
* @param r Result to pull from
* @return A pair of the {@link HRegionInfo} and the {@link ServerName}
* (or null for server address if no address set in hbase:meta).
* @throws IOException
* @deprecated use MetaTableAccessor methods for interacting with meta layouts
*/
@Deprecated

View File

@ -179,7 +179,7 @@ public class HTableDescriptor implements Comparable<HTableDescriptor> {
/**
* <em>INTERNAL</em> flag to indicate whether or not the memstore should be replicated
* for read-replicas (CONSISTENCY => TIMELINE).
* for read-replicas (CONSISTENCY =&gt; TIMELINE).
*/
public static final String REGION_MEMSTORE_REPLICATION = "REGION_MEMSTORE_REPLICATION";
private static final Bytes REGION_MEMSTORE_REPLICATION_KEY =

View File

@ -180,7 +180,7 @@ public class MetaTableAccessor {
}
}
/** The delimiter for meta columns for replicaIds > 0 */
/** The delimiter for meta columns for replicaIds &gt; 0 */
protected static final char META_REPLICA_ID_DELIMITER = '_';
/** A regex for parsing server columns from meta. See above javadoc for meta layout */
@ -1080,7 +1080,7 @@ public class MetaTableAccessor {
/**
* Fetch table states from META table
* @param conn connection to use
* @return map {tableName -> state}
* @return map {tableName -&gt; state}
* @throws IOException
*/
public static Map<TableName, TableState> getTableStates(Connection conn)

View File

@ -54,11 +54,11 @@ public abstract class AbstractClientScanner implements ResultScanner {
}
/**
* Get <param>nbRows</param> rows.
* Get nbRows rows.
* How many RPCs are made is determined by the {@link Scan#setCaching(int)}
* setting (or hbase.client.scanner.caching in hbase-site.xml).
* @param nbRows number of rows to return
* @return Between zero and <param>nbRows</param> RowResults. Scan is done
* @return Between zero and nbRows rowResults. Scan is done
* if returned array is of zero-length (We never return null).
* @throws IOException
*/

View File

@ -199,7 +199,7 @@ public interface Admin extends Abortable, Closeable {
*
* @param desc table descriptor for table
* @throws IllegalArgumentException if the table name is reserved
* @throws MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
* threads, the table may have been created between test-for-existence and attempt-at-creation).
* @throws IOException if a remote or network exception occurs
@ -218,7 +218,7 @@ public interface Admin extends Abortable, Closeable {
* @param endKey end of key range
* @param numRegions the total number of regions to create
* @throws IllegalArgumentException if the table name is reserved
* @throws MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
* threads, the table may have been created between test-for-existence and attempt-at-creation).
* @throws IOException
@ -235,7 +235,7 @@ public interface Admin extends Abortable, Closeable {
* @param splitKeys array of split keys for the initial regions of the table
* @throws IllegalArgumentException if the table name is reserved, if the split keys are repeated
* and if the split key has empty byte array.
* @throws MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running
* @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent
* threads, the table may have been created between test-for-existence and attempt-at-creation).
* @throws IOException
@ -248,11 +248,11 @@ public interface Admin extends Abortable, Closeable {
* It may throw ExecutionException if there was an error while executing the operation
* or TimeoutException in case the wait timeout was not long enough to allow the
* operation to complete.
* Throws IllegalArgumentException Bad table name, if the split keys
* are repeated and if the split key has empty byte array.
*
* @param desc table descriptor for table
* @param splitKeys keys to check if the table has been created with all split keys
* @throws IllegalArgumentException Bad table name, if the split keys
* are repeated and if the split key has empty byte array.
* @throws IOException if a remote or network exception occurs
* @return the result of the async creation. You can use Future.get(long, TimeUnit)
* to wait on the operation to complete.
@ -727,7 +727,7 @@ public interface Admin extends Abortable, Closeable {
* @param destServerName The servername of the destination regionserver. If passed the empty byte
* array we'll assign to a random server. A server name is made of host, port and startcode.
* Here is an example: <code> host187.example.com,60020,1289493121758</code>
* @throws UnknownRegionException Thrown if we can't find a region named
* @throws IOException if we can't find a region named
* <code>encodedRegionName</code>
*/
void move(final byte[] encodedRegionName, final byte[] destServerName)

View File

@ -39,10 +39,11 @@ import org.apache.hadoop.hbase.util.Bytes;
import com.google.common.annotations.VisibleForTesting;
/**
* <p>
* Client scanner for small reversed scan. Generally, only one RPC is called to fetch the
* scan results, unless the results cross multiple regions or the row count of
* results exceed the caching.
* <p/>
* </p>
* For small scan, it will get better performance than {@link ReversedClientScanner}
*/
@InterfaceAudience.Private

View File

@ -47,13 +47,13 @@ public enum Durability {
* Write the Mutation to the WAL synchronously.
* The data is flushed to the filesystem implementation, but not necessarily to disk.
* For HDFS this will flush the data to the designated number of DataNodes.
* See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313<a/>
* See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313</a>
*/
SYNC_WAL,
/**
* Write the Mutation to the WAL synchronously and force the entries to disk.
* (Note: this is currently not supported and will behave identical to {@link #SYNC_WAL})
* See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313<a/>
* See <a href="https://issues.apache.org/jira/browse/HADOOP-6313">HADOOP-6313</a>
*/
FSYNC_WAL
}

View File

@ -231,7 +231,8 @@ public class HBaseAdmin implements Admin {
* The connection to master will be created when required by admin functions.
*
* @param connection The Connection instance to use
* @throws MasterNotRunningException, ZooKeeperConnectionException are not
* @throws MasterNotRunningException
* @throws ZooKeeperConnectionException are not
* thrown anymore but kept into the interface for backward api compatibility
* @deprecated Constructing HBaseAdmin objects manually has been deprecated.
* Use {@link Connection#getAdmin()} to obtain an instance of {@link Admin} instead.

View File

@ -126,7 +126,7 @@ public interface HTableInterface extends Table {
* Executes all the buffered {@link Put} operations.
* <p>
* This method gets called once automatically for every {@link Put} or batch
* of {@link Put}s (when <code>put(List<Put>)</code> is used) when
* of {@link Put}s (when <code>put(List&lt;Put&gt;)</code> is used) when
* {@link #isAutoFlush} is {@code true}.
* @throws IOException if a remote or network exception occurs.
* @deprecated as of 1.0.0. Replaced by {@link BufferedMutator#flush()}

View File

@ -121,7 +121,6 @@ public class HTableMultiplexer {
* @param tableName
* @param put
* @return true if the request can be accepted by its corresponding buffer queue.
* @throws IOException
*/
public boolean put(TableName tableName, final Put put) {
return put(tableName, put, this.retryNum);
@ -133,7 +132,6 @@ public class HTableMultiplexer {
* @param tableName
* @param puts
* @return the list of puts which could not be queued
* @throws IOException
*/
public List<Put> put(TableName tableName, final List<Put> puts) {
if (puts == null)
@ -169,7 +167,6 @@ public class HTableMultiplexer {
* retried before dropping the request.
* Return false if the queue is already full.
* @return true if the request can be accepted by its corresponding buffer queue.
* @throws IOException
*/
public boolean put(final TableName tableName, final Put put, int retry) {
if (retry <= 0) {

View File

@ -328,7 +328,7 @@ public class Put extends Mutation implements HeapSize, Comparable<Row> {
/**
* A convenience method to determine if this object's familyMap contains
* a value assigned to the given family & qualifier.
* a value assigned to the given family &amp; qualifier.
* Both given arguments must match the KeyValue object to return true.
*
* @param family column family

View File

@ -204,7 +204,7 @@ public class Result implements CellScannable, CellScanner {
/**
* Return the array of Cells backing this Result instance.
*
* The array is sorted from smallest -> largest using the
* The array is sorted from smallest -&gt; largest using the
* {@link CellComparator#COMPARATOR}.
*
* The array only contains what your Get or Scan specifies and no more.
@ -601,7 +601,7 @@ public class Result implements CellScannable, CellScanner {
* Map of families to all versions of its qualifiers and values.
* <p>
* Returns a three level Map of the form:
* <code>Map&amp;family,Map&lt;qualifier,Map&lt;timestamp,value>>></code>
* <code>Map&amp;family,Map&lt;qualifier,Map&lt;timestamp,value&gt;&gt;&gt;</code>
* <p>
* Note: All other map returning methods make use of this map internally.
* @return map from families to qualifiers to versions
@ -643,7 +643,7 @@ public class Result implements CellScannable, CellScanner {
/**
* Map of families to their most recent qualifiers and values.
* <p>
* Returns a two level Map of the form: <code>Map&amp;family,Map&lt;qualifier,value>></code>
* Returns a two level Map of the form: <code>Map&amp;family,Map&lt;qualifier,value&gt;&gt;</code>
* <p>
* The most recent version of each qualifier will be used.
* @return map from families to qualifiers and value
@ -675,7 +675,7 @@ public class Result implements CellScannable, CellScanner {
/**
* Map of qualifiers to values.
* <p>
* Returns a Map of the form: <code>Map&lt;qualifier,value></code>
* Returns a Map of the form: <code>Map&lt;qualifier,value&gt;</code>
* @param family column family to get
* @return map of qualifiers to values
*/
@ -945,4 +945,4 @@ public class Result implements CellScannable, CellScanner {
throw new UnsupportedOperationException("Attempting to modify readonly EMPTY_RESULT!");
}
}
}
}

View File

@ -42,7 +42,7 @@ public interface ResultScanner extends Closeable, Iterable<Result> {
/**
* @param nbRows number of rows to return
* @return Between zero and <param>nbRows</param> Results
* @return Between zero and nbRows results
* @throws IOException e
*/
Result [] next(int nbRows) throws IOException;

View File

@ -24,7 +24,7 @@ import java.io.IOException;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* A Callable<T> that will be retried. If {@link #call(int)} invocation throws exceptions,
* A Callable&lt;T&gt; that will be retried. If {@link #call(int)} invocation throws exceptions,
* we will call {@link #throwable(Throwable, boolean)} with whatever the exception was.
* @param <T>
*/
@ -42,7 +42,7 @@ public interface RetryingCallable<T> {
* make it so we succeed on next call (clear caches, do relookup of locations, etc.).
* @param t
* @param retrying True if we are in retrying mode (we are not in retrying mode when max
* retries == 1; we ARE in retrying mode if retries > 1 even when we are the last attempt)
* retries == 1; we ARE in retrying mode if retries &gt; 1 even when we are the last attempt)
*/
void throwable(final Throwable t, boolean retrying);

View File

@ -174,6 +174,7 @@ public class RpcRetryingCallerWithReadReplicas {
}
/**
* <p>
* Algo:
* - we put the query into the execution pool.
* - after x ms, if we don't have a result, we add the queries for the secondary replicas
@ -186,7 +187,7 @@ public class RpcRetryingCallerWithReadReplicas {
* - a call is a thread. Let's not multiply the number of thread by the number of replicas.
* Server side, if we can cancel when it's still in the handler pool, it's much better, as a call
* can take some i/o.
* <p/>
* </p>
* Globally, the number of retries, timeout and so on still applies, but it's per replica,
* not global. We continue until all retries are done, or all timeouts are exceeded.
*/

View File

@ -727,10 +727,10 @@ public class Scan extends Query {
* this can deliver huge perf gains when there's a cf with lots of data; however, it can
* also lead to some inconsistent results, as follows:
* - if someone does a concurrent update to both column families in question you may get a row
* that never existed, e.g. for { rowKey = 5, { cat_videos => 1 }, { video => "my cat" } }
* someone puts rowKey 5 with { cat_videos => 0 }, { video => "my dog" }, concurrent scan
* filtering on "cat_videos == 1" can get { rowKey = 5, { cat_videos => 1 },
* { video => "my dog" } }.
* that never existed, e.g. for { rowKey = 5, { cat_videos =&gt; 1 }, { video =&gt; "my cat" } }
* someone puts rowKey 5 with { cat_videos =&gt; 0 }, { video =&gt; "my dog" }, concurrent scan
* filtering on "cat_videos == 1" can get { rowKey = 5, { cat_videos =&gt; 1 },
* { video =&gt; "my dog" } }.
* - if there's a concurrent split and you have more than 2 column families, some rows may be
* missing some column families.
*/
@ -982,7 +982,6 @@ public class Scan extends Query {
return ProtobufUtil.toScanMetrics(bytes);
}
public Boolean isAsyncPrefetch() {
return asyncPrefetch;
}
@ -991,6 +990,4 @@ public class Scan extends Query {
this.asyncPrefetch = asyncPrefetch;
return this;
}
}

View File

@ -64,16 +64,17 @@ import com.google.protobuf.Message;
* <p>
* This will serve as the client side handler for invoking the aggregate
* functions.
* <ul>
* For all aggregate functions,
* <li>start row < end row is an essential condition (if they are not
* <ul>
* <li>start row &lt; end row is an essential condition (if they are not
* {@link HConstants#EMPTY_BYTE_ARRAY})
* <li>Column family can't be null. In case where multiple families are
* provided, an IOException will be thrown. An optional column qualifier can
* also be defined.
* also be defined.</li>
* <li>For methods to find maximum, minimum, sum, rowcount, it returns the
* parameter type. For average and std, it returns a double value. For row
* count, it returns a long value.
* count, it returns a long value.</li>
* </ul>
* <p>Call {@link #close()} when done.
*/
@InterfaceAudience.Private
@ -109,10 +110,10 @@ public class AggregationClient implements Closeable {
* @param tableName
* @param ci
* @param scan
* @return max val <R>
* @return max val &lt;R&gt;
* @throws Throwable
* The caller is supposed to handle the exception as they are thrown
* & propagated to it.
* &amp; propagated to it.
*/
public <R, S, P extends Message, Q extends Message, T extends Message> R max(
final TableName tableName, final ColumnInterpreter<R, S, P, Q, T> ci, final Scan scan)
@ -129,10 +130,10 @@ public class AggregationClient implements Closeable {
* @param table
* @param ci
* @param scan
* @return max val <R>
* @return max val &lt;&gt;
* @throws Throwable
* The caller is supposed to handle the exception as they are thrown
* & propagated to it.
* &amp; propagated to it.
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
R max(final Table table, final ColumnInterpreter<R, S, P, Q, T> ci,
@ -199,7 +200,7 @@ public class AggregationClient implements Closeable {
* @param tableName
* @param ci
* @param scan
* @return min val <R>
* @return min val &lt;R&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> R min(
@ -217,7 +218,7 @@ public class AggregationClient implements Closeable {
* @param table
* @param ci
* @param scan
* @return min val <R>
* @return min val &lt;R&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
@ -269,11 +270,11 @@ public class AggregationClient implements Closeable {
* optimised the operation. In case qualifier is provided, I can't use the
* filter as it may set the flag to skip to next row, but the value read is
* not of the given filter: in this case, this particular row will not be
* counted ==> an error.
* counted ==&gt; an error.
* @param tableName
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> long rowCount(
@ -290,11 +291,11 @@ public class AggregationClient implements Closeable {
* optimised the operation. In case qualifier is provided, I can't use the
* filter as it may set the flag to skip to next row, but the value read is
* not of the given filter: in this case, this particular row will not be
* counted ==> an error.
* counted ==&gt; an error.
* @param table
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
@ -341,7 +342,7 @@ public class AggregationClient implements Closeable {
* @param tableName
* @param ci
* @param scan
* @return sum <S>
* @return sum &lt;S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> S sum(
@ -358,7 +359,7 @@ public class AggregationClient implements Closeable {
* @param table
* @param ci
* @param scan
* @return sum <S>
* @return sum &lt;S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
@ -485,7 +486,7 @@ public class AggregationClient implements Closeable {
* @param tableName
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
@ -504,7 +505,7 @@ public class AggregationClient implements Closeable {
* @param table
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> double avg(
@ -593,7 +594,7 @@ public class AggregationClient implements Closeable {
* @param tableName
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message>
@ -613,7 +614,7 @@ public class AggregationClient implements Closeable {
* @param table
* @param ci
* @param scan
* @return <R, S>
* @return &lt;R, S&gt;
* @throws Throwable
*/
public <R, S, P extends Message, Q extends Message, T extends Message> double std(

View File

@ -203,14 +203,9 @@ public class MyLittleHBaseClient {
<li><a href="http://hbase.org">HBase Home Page</a>
<li><a href="http://hadoop.apache.org/">Hadoop Home Page</a>
</ul>
</pre></code>
</div>
<p>See also the section in the HBase Reference Guide where it discusses
<a href="http://hbase.apache.org/book.html#client">HBase Client</a>. It
has section on how to access HBase from inside your multithreaded environment
how to control resources consumed client-side, etc.</p>
</body>
</html>
*/
package org.apache.hadoop.hbase.client;

View File

@ -35,23 +35,23 @@ import com.google.protobuf.Message;
* for an example.
* <p>
* Takes two generic parameters and three Message parameters.
* The cell value type of the interpreter is <T>.
* The cell value type of the interpreter is &lt;T&gt;.
* During some computations like sum, average, the return type can be different
* than the cell value data type, for eg, sum of int cell values might overflow
* in case of a int result, we should use Long for its result. Therefore, this
* class mandates to use a different (promoted) data type for result of these
* computations <S>. All computations are performed on the promoted data type
* <S>. There is a conversion method
* {@link ColumnInterpreter#castToReturnType(Object)} which takes a <T> type and
* returns a <S> type.
* The AggregateImplementation uses PB messages to initialize the
* computations &lt;S&gt;. All computations are performed on the promoted data type
* &lt;S&gt;. There is a conversion method
* {@link ColumnInterpreter#castToReturnType(Object)} which takes a &lt;T&gt; type and
* returns a &lt;S&gt; type.
* The AggregateIm&gt;lementation uses PB messages to initialize the
* user's ColumnInterpreter implementation, and for sending the responses
* back to AggregationClient.
* @param <T> Cell value data type
* @param <S> Promoted data type
* @param <P> PB message that is used to transport initializer specific bytes
* @param <Q> PB message that is used to transport Cell (<T>) instance
* @param <R> PB message that is used to transport Promoted (<S>) instance
* @param T Cell value data type
* @param S Promoted data type
* @param P PB message that is used to transport initializer specific bytes
* @param Q PB message that is used to transport Cell (&lt;T&gt;) instance
* @param R PB message that is used to transport Promoted (&lt;S&gt;) instance
*/
@InterfaceAudience.Private
public abstract class ColumnInterpreter<T, S, P extends Message,
@ -109,14 +109,15 @@ Q extends Message, R extends Message> {
* This takes care if either of arguments are null. returns 0 if they are
* equal or both are null;
* <ul>
* <li>>0 if l1 > l2 or l1 is not null and l2 is null.
* <li>< 0 if l1 < l2 or l1 is null and l2 is not null.
* <li>&gt; 0 if l1 &gt; l2 or l1 is not null and l2 is null.</li>
* <li>&lt; 0 if l1 &lt; l2 or l1 is null and l2 is not null.</li>
* </ul>
*/
public abstract int compare(final T l1, final T l2);
/**
* used for computing average of <S> data values. Not providing the divide
* method that takes two <S> values as it is not needed as of now.
* used for computing average of &lt;S&gt; data values. Not providing the divide
* method that takes two &lt;S&gt; values as it is not needed as of now.
* @param o
* @param l
* @return Average

View File

@ -32,15 +32,16 @@ import org.apache.hadoop.hbase.protobuf.generated.FilterProtos;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* <p>
* This filter is used to filter based on the column family. It takes an
* operator (equal, greater, not equal, etc) and a byte [] comparator for the
* column family portion of a key.
* <p/>
* </p><p>
* This filter can be wrapped with {@link org.apache.hadoop.hbase.filter.WhileMatchFilter} and {@link org.apache.hadoop.hbase.filter.SkipFilter}
* to add more control.
* <p/>
* </p><p>
* Multiple filters can be combined using {@link org.apache.hadoop.hbase.filter.FilterList}.
* <p/>
* </p>
* If an already known column family is looked for, use {@link org.apache.hadoop.hbase.client.Get#addFamily(byte[])}
* directly rather than a filter.
*/

View File

@ -261,7 +261,6 @@ public abstract class Filter {
* @param pbBytes A pb serialized {@link Filter} instance
* @return An instance of {@link Filter} made from <code>bytes</code>
* @throws DeserializationException
* @throws IOException in case an I/O or an filter specific failure needs to be signaled.
* @see #toByteArray
*/
public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException {

View File

@ -41,7 +41,7 @@ public abstract class FilterBase extends Filter {
* Filters that are purely stateless and do nothing in their reset() methods can inherit
* this null/empty implementation.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public void reset() throws IOException {
@ -51,7 +51,7 @@ public abstract class FilterBase extends Filter {
* Filters that do not filter by row key can inherit this implementation that
* never filters anything. (ie: returns false).
*
* @inheritDoc
* {@inheritDoc}
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Instead use {@link #filterRowKey(Cell)}
*/
@ -72,7 +72,7 @@ public abstract class FilterBase extends Filter {
* Filters that never filter all remaining can inherit this implementation that
* never stops the filter early.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public boolean filterAllRemaining() throws IOException {
@ -82,7 +82,7 @@ public abstract class FilterBase extends Filter {
/**
* By default no transformation takes place
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public Cell transformCell(Cell v) throws IOException {
@ -93,7 +93,7 @@ public abstract class FilterBase extends Filter {
* Filters that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public void filterRowCells(List<Cell> ignored) throws IOException {
@ -103,7 +103,7 @@ public abstract class FilterBase extends Filter {
* Fitlers that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public boolean hasFilterRow() {
@ -115,7 +115,7 @@ public abstract class FilterBase extends Filter {
* {@link #filterKeyValue(Cell)} can inherit this implementation that
* never filters a row.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public boolean filterRow() throws IOException {
@ -126,7 +126,7 @@ public abstract class FilterBase extends Filter {
* Filters that are not sure which key must be next seeked to, can inherit
* this implementation that, by default, returns a null Cell.
*
* @inheritDoc
* {@inheritDoc}
*/
public Cell getNextCellHint(Cell currentCell) throws IOException {
return null;
@ -136,7 +136,7 @@ public abstract class FilterBase extends Filter {
* By default, we require all scan's column families to be present. Our
* subclasses may be more precise.
*
* @inheritDoc
* {@inheritDoc}
*/
public boolean isFamilyEssential(byte[] name) throws IOException {
return true;

View File

@ -41,14 +41,14 @@ import com.google.protobuf.InvalidProtocolBufferException;
* Since you can use Filter Lists as children of Filter Lists, you can create a
* hierarchy of filters to be evaluated.
*
* <br/>
* <br>
* {@link Operator#MUST_PASS_ALL} evaluates lazily: evaluation stops as soon as one filter does
* not include the KeyValue.
*
* <br/>
* <br>
* {@link Operator#MUST_PASS_ONE} evaluates non-lazily: all filters are always evaluated.
*
* <br/>
* <br>
* Defaults to {@link Operator#MUST_PASS_ALL}.
*/
@InterfaceAudience.Public
@ -315,7 +315,7 @@ final public class FilterList extends Filter {
* Filters that never filter by modifying the returned List of Cells can
* inherit this implementation that does nothing.
*
* @inheritDoc
* {@inheritDoc}
*/
@Override
public void filterRowCells(List<Cell> cells) throws IOException {

View File

@ -44,7 +44,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
* phoenix etc. However, both solutions are inefficient. Both of them can't utilize the range info
* to perform fast forwarding during scan which is quite time consuming. If the number of ranges
* are quite big (e.g. millions), join is a proper solution though it is slow. However, there are
* cases that user wants to specify a small number of ranges to scan (e.g. <1000 ranges). Both
* cases that user wants to specify a small number of ranges to scan (e.g. &lt;1000 ranges). Both
* solutions can't provide satisfactory performance in such case. MultiRowRangeFilter is to support
* such usec ase (scan multiple row key ranges), which can construct the row key ranges from user
* specified list and perform fast-forwarding during scan. Thus, the scan will be quite efficient.

View File

@ -31,11 +31,11 @@ import com.google.common.base.Preconditions;
import com.google.protobuf.InvalidProtocolBufferException;
/**
* Implementation of Filter interface that limits results to a specific page
* size. It terminates scanning once the number of filter-passed rows is >
* size. It terminates scanning once the number of filter-passed rows is &gt;
* the given page size.
* <p>
* Note that this filter cannot guarantee that the number of results returned
* to a client are <= page size. This is because the filter is applied
* to a client are &lt;= page size. This is because the filter is applied
* separately on different region servers. It does however optimize the scan of
* individual HRegions by making sure that the page size is never exceeded
* locally.

View File

@ -39,8 +39,9 @@ import com.google.protobuf.InvalidProtocolBufferException;
* entire row if any of its weights are zero. In this case, we want to prevent
* rows from being emitted if a single key is filtered. Combine this filter
* with a {@link ValueFilter}:
* </p>
* <p>
* <pre>
* <code>
* scan.setFilter(new SkipFilter(new ValueFilter(CompareOp.NOT_EQUAL,
* new BinaryComparator(Bytes.toBytes(0))));
* </code>
@ -48,6 +49,7 @@ import com.google.protobuf.InvalidProtocolBufferException;
* (since ValueFilter will not pass that Cell).
* Without this filter, the other non-zero valued columns in the row would still
* be emitted.
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Stable

View File

@ -422,7 +422,6 @@ public class AsyncRpcClient extends AbstractRpcClient {
* @param rpcTimeout default rpc operation timeout
*
* @return A rpc channel that goes via this rpc client instance.
* @throws IOException when channel could not be created
*/
public RpcChannel createRpcChannel(final ServerName sn, final User user, int rpcTimeout) {
return new RpcChannelImplementation(this, sn, user, rpcTimeout);

View File

@ -24,7 +24,7 @@ import java.net.InetSocketAddress;
/**
* This class holds the address and the user ticket, etc. The client connections
* to servers are uniquely identified by <remoteAddress, ticket, serviceName>
* to servers are uniquely identified by &lt;remoteAddress, ticket, serviceName&gt;
*/
@InterfaceAudience.Private
public class ConnectionId {

View File

@ -35,7 +35,8 @@ import com.google.protobuf.RpcController;
* When implementing {@link com.google.protobuf.Service} defined methods,
* coprocessor endpoints can use the following pattern to pass exceptions back to the RPC client:
* <code>
* public void myMethod(RpcController controller, MyRequest request, RpcCallback<MyResponse> done) {
* public void myMethod(RpcController controller, MyRequest request,
* RpcCallback&lt;MyResponse&gt; done) {
* MyResponse response = null;
* try {
* // do processing

View File

@ -252,7 +252,7 @@ public final class ProtobufUtil {
* to flag what follows as a protobuf in hbase. Prepend these bytes to all content written to
* znodes, etc.
* @param bytes Bytes to decorate
* @return The passed <code>bytes</codes> with magic prepended (Creates a new
* @return The passed <code>bytes</code> with magic prepended (Creates a new
* byte array that is <code>bytes.length</code> plus {@link ProtobufMagic#PB_MAGIC}.length.
*/
public static byte [] prependPBMagic(final byte [] bytes) {
@ -2120,7 +2120,7 @@ public final class ProtobufUtil {
}
/**
* Convert a ListMultimap<String, TablePermission> where key is username
* Convert a ListMultimap&lt;String, TablePermission&gt; where key is username
* to a protobuf UserPermission
*
* @param perm the list of user and table permissions
@ -2374,7 +2374,7 @@ public final class ProtobufUtil {
/**
* Convert a protobuf UserTablePermissions to a
* ListMultimap<String, TablePermission> where key is username.
* ListMultimap&lt;String, TablePermission&gt; where key is username.
*
* @param proto the protobuf UserPermission
* @return the converted UserPermission

View File

@ -51,11 +51,11 @@ import org.apache.hadoop.hbase.util.Strings;
* Helper class to interact with the quota table.
* <pre>
* ROW-KEY FAM/QUAL DATA
* n.<namespace> q:s <global-quotas>
* t.<table> q:s <global-quotas>
* u.<user> q:s <global-quotas>
* u.<user> q:s.<table> <table-quotas>
* u.<user> q:s.<ns>: <namespace-quotas>
* n.&lt;namespace&gt; q:s &lt;global-quotas&gt;
* t.&lt;table&gt; q:s &lt;global-quotas&gt;
* u.&lt;user&gt; q:s &lt;global-quotas&gt;
* u.&lt;user&gt; q:s.&lt;table&gt; &lt;table-quotas&gt;
* u.&lt;user&gt; q:s.&lt;ns&gt;: &lt;namespace-quotas&gt;
* </pre>
*/
@InterfaceAudience.Private

View File

@ -34,7 +34,7 @@ public enum BloomType {
*/
ROW,
/**
* Bloom enabled with Table row & column (family+qualifier) as Key
* Bloom enabled with Table row &amp; column (family+qualifier) as Key
*/
ROWCOL
}

View File

@ -125,7 +125,7 @@ public interface ReplicationPeers {
* have to be connected. The state is read directly from the backing store.
* @param peerId a short that identifies the cluster
* @return true if replication is enabled, false otherwise.
* @throws IOException Throws if there's an error contacting the store
* @throws ReplicationException thrown if there's an error contacting the store
*/
boolean getStatusOfPeerFromBackingStore(String peerId) throws ReplicationException;

View File

@ -24,7 +24,8 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* This contains a visibility expression which can be associated with a cell. When it is set with a
* Mutation, all the cells in that mutation will get associated with this expression. A visibility
* expression can contain visibility labels combined with logical operators AND(&), OR(|) and NOT(!)
* expression can contain visibility labels combined with logical
* operators AND(&amp;), OR(|) and NOT(!)
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@ -51,7 +52,7 @@ public class CellVisibility {
/**
* Helps in quoting authentication Strings. Use this if unicode characters to
* be used in expression or special characters like '(', ')',
* '"','\','&','|','!'
* '"','\','&amp;','|','!'
*/
public static String quote(String auth) {
return quote(Bytes.toBytes(auth));
@ -60,7 +61,7 @@ public class CellVisibility {
/**
* Helps in quoting authentication Strings. Use this if unicode characters to
* be used in expression or special characters like '(', ')',
* '"','\','&','|','!'
* '"','\','&amp;','|','!'
*/
public static String quote(byte[] auth) {
int escapeChars = 0;

View File

@ -1206,8 +1206,6 @@ public class ZKUtil {
* @param data data of node to create
* @param cb
* @param ctx
* @throws KeeperException if unexpected zookeeper exception
* @throws KeeperException.NodeExistsException if node already exists
*/
public static void asyncCreate(ZooKeeperWatcher zkw,
String znode, byte [] data, final AsyncCallback.StringCallback cb,
@ -2027,7 +2025,6 @@ public class ZKUtil {
/**
* Recursively print the current state of ZK (non-transactional)
* @param root name of the root directory in zk to print
* @throws KeeperException
*/
public static void logZKTree(ZooKeeperWatcher zkw, String root) {
if (!LOG.isDebugEnabled()) return;

View File

@ -656,7 +656,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable {
/**
* Close the connection to ZooKeeper.
*
* @throws InterruptedException
*/
@Override
public void close() {

View File

@ -23,7 +23,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* The unit of storage in HBase consisting of the following fields:<br/>
* The unit of storage in HBase consisting of the following fields:
* <br>
* <pre>
* 1) row
* 2) column family
@ -33,30 +34,36 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* 6) MVCC version
* 7) value
* </pre>
* <p/>
* <p>
* Uniqueness is determined by the combination of row, column family, column qualifier,
* timestamp, and type.
* <p/>
* </p>
* <p>
* The natural comparator will perform a bitwise comparison on row, column family, and column
* qualifier. Less intuitively, it will then treat the greater timestamp as the lesser value with
* the goal of sorting newer cells first.
* <p/>
* </p>
* <p>
* This interface should not include methods that allocate new byte[]'s such as those used in client
* or debugging code. These users should use the methods found in the {@link CellUtil} class.
* Currently for to minimize the impact of existing applications moving between 0.94 and 0.96, we
* include the costly helper methods marked as deprecated.
* <p/>
* Cell implements Comparable<Cell> which is only meaningful when comparing to other keys in the
* </p>
* <p>
* Cell implements Comparable&lt;Cell&gt; which is only meaningful when
* comparing to other keys in the
* same table. It uses CellComparator which does not work on the -ROOT- and hbase:meta tables.
* <p/>
* </p>
* <p>
* In the future, we may consider adding a boolean isOnHeap() method and a getValueBuffer() method
* that can be used to pass a value directly from an off-heap ByteBuffer to the network without
* copying into an on-heap byte[].
* <p/>
* </p>
* <p>
* Historic note: the original Cell implementation (KeyValue) requires that all fields be encoded as
* consecutive bytes in the same byte[], whereas this interface allows fields to reside in separate
* byte[]'s.
* <p/>
* </p>
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@ -77,7 +84,7 @@ public interface Cell {
int getRowOffset();
/**
* @return Number of row bytes. Must be < rowArray.length - offset.
* @return Number of row bytes. Must be &lt; rowArray.length - offset.
*/
short getRowLength();
@ -97,7 +104,7 @@ public interface Cell {
int getFamilyOffset();
/**
* @return Number of family bytes. Must be < familyArray.length - offset.
* @return Number of family bytes. Must be &lt; familyArray.length - offset.
*/
byte getFamilyLength();
@ -117,7 +124,7 @@ public interface Cell {
int getQualifierOffset();
/**
* @return Number of qualifier bytes. Must be < qualifierArray.length - offset.
* @return Number of qualifier bytes. Must be &lt; qualifierArray.length - offset.
*/
int getQualifierLength();
@ -148,7 +155,7 @@ public interface Cell {
* cells in the memstore but is not retained forever. It may survive several flushes, but
* generally becomes irrelevant after the cell's row is no longer involved in any operations that
* require strict consistency.
* @return mvccVersion (always >= 0 if exists), or 0 if it no longer exists
* @return mvccVersion (always &gt;= 0 if exists), or 0 if it no longer exists
*/
@Deprecated
long getMvccVersion();
@ -158,7 +165,7 @@ public interface Cell {
* exists for cells in the memstore but is not retained forever. It will be kept for
* {@link HConstants#KEEP_SEQID_PERIOD} days, but generally becomes irrelevant after the cell's
* row is no longer involved in any operations that require strict consistency.
* @return seqId (always > 0 if exists), or 0 if it no longer exists
* @return seqId (always &gt; 0 if exists), or 0 if it no longer exists
*/
long getSequenceId();
@ -177,7 +184,7 @@ public interface Cell {
int getValueOffset();
/**
* @return Number of value bytes. Must be < valueArray.length - offset.
* @return Number of value bytes. Must be &lt; valueArray.length - offset.
*/
int getValueLength();
@ -199,7 +206,7 @@ public interface Cell {
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's value.
*
* Added to ease transition from 0.94 -> 0.96.
* Added to ease transition from 0.94 -&gt; 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#cloneValue(Cell)}
*/
@ -209,7 +216,7 @@ public interface Cell {
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's family.
*
* Added to ease transition from 0.94 -> 0.96.
* Added to ease transition from 0.94 -&gt; 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#cloneFamily(Cell)}
*/
@ -219,7 +226,7 @@ public interface Cell {
/**
* WARNING do not use, expensive. This gets an arraycopy of the cell's qualifier.
*
* Added to ease transition from 0.94 -> 0.96.
* Added to ease transition from 0.94 -&gt; 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#cloneQualifier(Cell)}
*/
@ -229,10 +236,10 @@ public interface Cell {
/**
* WARNING do not use, expensive. this gets an arraycopy of the cell's row.
*
* Added to ease transition from 0.94 -> 0.96.
* Added to ease transition from 0.94 -&gt; 0.96.
*
* @deprecated as of 0.96, use {@link CellUtil#getRowByte(Cell, int)}
*/
@Deprecated
byte[] getRow();
}
}

View File

@ -111,7 +111,7 @@ public class CellComparator implements Comparator<Cell>, Serializable {
* @param b
* @param ignoreSequenceid True if we are to compare the key portion only and ignore
* the sequenceid. Set to false to compare key and consider sequenceid.
* @return 0 if equal, -1 if a < b, and +1 if a > b.
* @return 0 if equal, -1 if a &lt; b, and +1 if a &gt; b.
*/
private final int compare(final Cell a, final Cell b, boolean ignoreSequenceid) {
// row
@ -682,4 +682,4 @@ public class CellComparator implements Comparator<Cell>, Serializable {
return result;
}
}
}
}

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* Implementer can return a CellScanner over its Cell content.
* Class name is ugly but mimicing java.util.Iterable only we are about the dumber
* CellScanner rather than say Iterator<Cell>. See CellScanner class comment for why we go
* CellScanner rather than say Iterator&lt;Cell&gt;. See CellScanner class comment for why we go
* dumber than java.util.Iterator.
*/
@InterfaceAudience.Private

View File

@ -35,7 +35,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* may or may not point to a reusable cell implementation, so users of the CellScanner should not,
* for example, accumulate a List of Cells. All of the references may point to the same object,
* which would be the latest state of the underlying Cell. In short, the Cell is mutable.
* <p/>
* </p>
* Typical usage:
*
* <pre>

View File

@ -1133,7 +1133,8 @@ public final class HConstants {
/**
* When using bucket cache, this is a float that EITHER represents a percentage of total heap
* memory size to give to the cache (if < 1.0) OR, it is the capacity in megabytes of the cache.
* memory size to give to the cache (if &lt; 1.0) OR, it is the capacity in
* megabytes of the cache.
*/
public static final String BUCKET_CACHE_SIZE_KEY = "hbase.bucketcache.size";

View File

@ -64,18 +64,21 @@ import com.google.common.annotations.VisibleForTesting;
* <p>
* KeyValue wraps a byte array and takes offsets and lengths into passed array at where to start
* interpreting the content as KeyValue. The KeyValue format inside a byte array is:
* <code>&lt;keylength> &lt;valuelength> &lt;key> &lt;value></code> Key is further decomposed as:
* <code>&lt;rowlength> &lt;row> &lt;columnfamilylength> &lt;columnfamily> &lt;columnqualifier>
* &lt;timestamp> &lt;keytype></code>
* <code>&lt;keylength&gt; &lt;valuelength&gt; &lt;key&gt; &lt;value&gt;</code>
* Key is further decomposed as:
* <code>&lt;rowlength&gt; &lt;row&gt; &lt;columnfamilylength&gt;
* &lt;columnfamily&gt; &lt;columnqualifier&gt;
* &lt;timestamp&gt; &lt;keytype&gt;</code>
* The <code>rowlength</code> maximum is <code>Short.MAX_SIZE</code>, column family length maximum
* is <code>Byte.MAX_SIZE</code>, and column qualifier + key length must be <
* is <code>Byte.MAX_SIZE</code>, and column qualifier + key length must be &lt;
* <code>Integer.MAX_SIZE</code>. The column does not contain the family/qualifier delimiter,
* {@link #COLUMN_FAMILY_DELIMITER}<br>
* KeyValue can optionally contain Tags. When it contains tags, it is added in the byte array after
* the value part. The format for this part is: <code>&lt;tagslength>&lt;tagsbytes></code>.
* the value part. The format for this part is: <code>&lt;tagslength&gt;&lt;tagsbytes&gt;</code>.
* <code>tagslength</code> maximum is <code>Short.MAX_SIZE</code>. The <code>tagsbytes</code>
* contain one or more tags where as each tag is of the form
* <code>&lt;taglength>&lt;tagtype>&lt;tagbytes></code>. <code>tagtype</code> is one byte and
* <code>&lt;taglength&gt;&lt;tagtype&gt;&lt;tagbytes&gt;</code>.
* <code>tagtype</code> is one byte and
* <code>taglength</code> maximum is <code>Short.MAX_SIZE</code> and it includes 1 byte type length
* and actual tag bytes length.
*/
@ -1163,7 +1166,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* as JSON. Values are left out due to their tendency to be large. If needed,
* they can be added manually.
*
* @return the Map<String,?> containing data from this key
* @return the Map&lt;String,?&gt; containing data from this key
*/
public Map<String, Object> toStringMap() {
Map<String, Object> stringMap = new HashMap<String, Object>();
@ -1878,7 +1881,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* Compares the only the user specified portion of a Key. This is overridden by MetaComparator.
* @param left
* @param right
* @return 0 if equal, <0 if left smaller, >0 if right smaller
* @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
*/
protected int compareRowKey(final Cell left, final Cell right) {
return CellComparator.COMPARATOR.compareRows(left, right);
@ -1893,7 +1896,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* @param right
* @param roffset
* @param rlength
* @return 0 if equal, <0 if left smaller, >0 if right smaller
* @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
*/
public int compareFlatKey(byte[] left, int loffset, int llength,
byte[] right, int roffset, int rlength) {
@ -2005,7 +2008,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* @param right
* @param roffset
* @param rlength
* @return 0 if equal, <0 if left smaller, >0 if right smaller
* @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
*/
public int compareRows(byte [] left, int loffset, int llength,
byte [] right, int roffset, int rlength) {
@ -2054,7 +2057,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* @param right
* @param roffset
* @param rlength
* @return 0 if equal, <0 if left smaller, >0 if right smaller
* @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
*/
@Override // SamePrefixComparator
public int compareIgnoringPrefix(int commonPrefix, byte[] left,
@ -2292,7 +2295,7 @@ public class KeyValue implements Cell, HeapSize, Cloneable, SettableSequenceId,
* This is a HFile block index key optimization.
* @param leftKey
* @param rightKey
* @return 0 if equal, <0 if left smaller, >0 if right smaller
* @return 0 if equal, &lt;0 if left smaller, &gt;0 if right smaller
* @deprecated Since 0.99.2;
*/
@Deprecated

View File

@ -526,7 +526,7 @@ public class KeyValueUtil {
/*************** misc **********************************/
/**
* @param cell
* @return <code>cell<code> if it is an instance of {@link KeyValue} else we will return a
* @return <code>cell</code> if it is an instance of {@link KeyValue} else we will return a
* new {@link KeyValue} instance made from <code>cell</code>
* @deprecated without any replacement.
*/

View File

@ -41,7 +41,8 @@ import com.google.protobuf.InvalidProtocolBufferException;
* servers on same hostname and port (startcode is usually timestamp of server startup). The
* {@link #toString()} format of ServerName is safe to use in the filesystem and as znode name
* up in ZooKeeper. Its format is:
* <code>&lt;hostname> '{@link #SERVERNAME_SEPARATOR}' &lt;port> '{@link #SERVERNAME_SEPARATOR}' &lt;startcode></code>.
* <code>&lt;hostname&gt; '{@link #SERVERNAME_SEPARATOR}' &lt;port&gt;
* '{@link #SERVERNAME_SEPARATOR}' &lt;startcode&gt;</code>.
* For example, if hostname is <code>www.example.org</code>, port is <code>1234</code>,
* and the startcode for the regionserver is <code>1212121212</code>, then
* the {@link #toString()} would be <code>www.example.org,1234,1212121212</code>.
@ -224,7 +225,7 @@ public class ServerName implements Comparable<ServerName>, Serializable {
* @param port
* @param startcode
* @return Server name made of the concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
* startcode formatted as <code>&lt;hostname&gt; ',' &lt;port&gt; ',' &lt;startcode&gt;</code>
*/
static String getServerName(String hostName, int port, long startcode) {
final StringBuilder name = new StringBuilder(hostName.length() + 1 + 5 + 1 + 13);
@ -237,10 +238,10 @@ public class ServerName implements Comparable<ServerName>, Serializable {
}
/**
* @param hostAndPort String in form of &lt;hostname> ':' &lt;port>
* @param hostAndPort String in form of &lt;hostname&gt; ':' &lt;port&gt;
* @param startcode
* @return Server name made of the concatenation of hostname, port and
* startcode formatted as <code>&lt;hostname> ',' &lt;port> ',' &lt;startcode></code>
* startcode formatted as <code>&lt;hostname&gt; ',' &lt;port&gt; ',' &lt;startcode&gt;</code>
*/
public static String getServerName(final String hostAndPort,
final long startcode) {
@ -339,7 +340,7 @@ public class ServerName implements Comparable<ServerName>, Serializable {
/**
* @param str Either an instance of {@link ServerName#toString()} or a
* "'<hostname>' ':' '<port>'".
* "'&lt;hostname&gt;' ':' '&lt;port&gt;'".
* @return A ServerName instance.
*/
public static ServerName parseServerName(final String str) {

View File

@ -126,8 +126,8 @@ public final class TableName implements Comparable<TableName> {
* The name may not start with '.' or '-'.
*
* Valid fully qualified table names:
* foo:bar, namespace=>foo, table=>bar
* org:foo.bar, namespace=org, table=>foo.bar
* foo:bar, namespace=&gt;foo, table=&gt;bar
* org:foo.bar, namespace=org, table=&gt;foo.bar
*/
public static byte [] isLegalFullyQualifiedTableName(final byte[] tableName) {
if (tableName == null || tableName.length <= 0) {

View File

@ -27,7 +27,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* Accepts a stream of Cells. This can be used to build a block of cells during compactions
* and flushes, or to build a byte[] to send to the client. This could be backed by a
* List<KeyValue>, but more efficient implementations will append results to a
* List&lt;KeyValue&gt;, but more efficient implementations will append results to a
* byte[] to eliminate overhead, and possibly encode the cells further.
* <p>To read Cells, use {@link org.apache.hadoop.hbase.CellScanner}
* @see org.apache.hadoop.hbase.CellScanner

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.util.Bytes;
/**
* Represents an interval of version timestamps.
* <p>
* Evaluated according to minStamp <= timestamp < maxStamp
* Evaluated according to minStamp &lt;= timestamp &lt; maxStamp
* or [minStamp,maxStamp) in interval notation.
* <p>
* Only used internally; should not be accessed directly by clients.

View File

@ -40,7 +40,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* on the local filesystem. It is configured with a URI passed in as a String
* to init(). The URI should have the form:
* <p>
* <pre> scheme://path?option1=value1&option2=value2</pre>
* <pre> scheme://path?option1=value1&amp;option2=value2</pre>
* <p>
* <i>scheme</i> can be either "jks" or "jceks", specifying the file based
* providers shipped with every JRE. The latter is the certificate store for

View File

@ -61,7 +61,7 @@ public class ThrottledInputStream extends InputStream {
rawStream.close();
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read() throws IOException {
throttle();
@ -72,7 +72,7 @@ public class ThrottledInputStream extends InputStream {
return data;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read(byte[] b) throws IOException {
throttle();
@ -83,7 +83,7 @@ public class ThrottledInputStream extends InputStream {
return readLen;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public int read(byte[] b, int off, int len) throws IOException {
throttle();
@ -159,7 +159,7 @@ public class ThrottledInputStream extends InputStream {
return totalSleepTime;
}
/** @inheritDoc */
/** {@inheritDoc} */
@Override
public String toString() {
return "ThrottledInputStream{" +

View File

@ -45,7 +45,7 @@ public interface Dictionary {
*
* @param data the byte array that we're looking up
* @param offset Offset into <code>data</code> to add to Dictionary.
* @param length Length beyond <code>offset</code> that comprises entry; must be > 0.
* @param length Length beyond <code>offset</code> that comprises entry; must be &gt; 0.
* @return the index of the entry, or {@link #NOT_IN_DICTIONARY} if not found
*/
short findEntry(byte[] data, int offset, int length);
@ -59,7 +59,7 @@ public interface Dictionary {
*
* @param data the entry to add
* @param offset Offset into <code>data</code> to add to Dictionary.
* @param length Length beyond <code>offset</code> that comprises entry; must be > 0.
* @param length Length beyond <code>offset</code> that comprises entry; must be &gt; 0.
* @return the index of the entry
*/

View File

@ -194,7 +194,6 @@ public abstract class User {
* @param action
* @return the result of the action
* @throws IOException
* @throws InterruptedException
*/
@SuppressWarnings({ "rawtypes", "unchecked" })
public static <T> T runAsLoginUser(PrivilegedExceptionAction<T> action) throws IOException {

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
* scenario where the end of the buffer has been reached but there are still
* nullable fields remaining in the {@code Struct} definition. When this
* happens, it will produce null entries for the remaining values. For example:
* </p>
* <pre>
* StructBuilder builder = new StructBuilder()
* .add(OrderedNumeric.ASCENDING) // nullable
@ -57,11 +58,10 @@ import org.apache.hadoop.hbase.util.PositionedByteRange;
* Object[] val = new Object[] { BigDecimal.ONE, "foo" };
* shorter.encode(buf1, val); // write short value with short Struct
* buf1.setPosition(0); // reset position marker, prepare for read
* longer.decode(buf1); // => { BigDecimal.ONE, "foo", null } ; long Struct reads implied null
* longer.decode(buf1); // =&gt; { BigDecimal.ONE, "foo", null } ; long Struct reads implied null
* longer.encode(buf2, val); // write short value with long struct
* Bytes.equals(buf1.getBytes(), buf2.getBytes()); // => true; long Struct skips writing null
* Bytes.equals(buf1.getBytes(), buf2.getBytes()); // =&gt; true; long Struct skips writing null
* </pre>
* </p>
* <h3>Sort Order</h3>
* <p>
* {@code Struct} instances sort according to the composite order of their

View File

@ -33,7 +33,7 @@ public abstract class AbstractByteRange implements ByteRange {
// reuse objects of this class
/**
* The array containing the bytes in this range. It will be >= length.
* The array containing the bytes in this range. It will be &gt;= length.
*/
protected byte[] bytes;
@ -44,7 +44,7 @@ public abstract class AbstractByteRange implements ByteRange {
protected int offset;
/**
* The number of bytes in the range. Offset + length must be <= bytes.length
* The number of bytes in the range. Offset + length must be &lt;= bytes.length
*/
protected int length;

View File

@ -37,7 +37,7 @@ public class Addressing {
public static final String HOSTNAME_PORT_SEPARATOR = ":";
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @param hostAndPort Formatted as <code>&lt;hostname&gt; ':' &lt;port&gt;</code>
* @return An InetSocketInstance
*/
public static InetSocketAddress createInetSocketAddressFromHostAndPortStr(
@ -50,7 +50,7 @@ public class Addressing {
* @param port Server port
* @return Returns a concatenation of <code>hostname</code> and
* <code>port</code> in following
* form: <code>&lt;hostname> ':' &lt;port></code>. For example, if hostname
* form: <code>&lt;hostname&gt; ':' &lt;port&gt;</code>. For example, if hostname
* is <code>example.org</code> and port is 1234, this method will return
* <code>example.org:1234</code>
*/
@ -59,7 +59,7 @@ public class Addressing {
}
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @param hostAndPort Formatted as <code>&lt;hostname&gt; ':' &lt;port&gt;</code>
* @return The hostname portion of <code>hostAndPort</code>
*/
public static String parseHostname(final String hostAndPort) {
@ -71,7 +71,7 @@ public class Addressing {
}
/**
* @param hostAndPort Formatted as <code>&lt;hostname> ':' &lt;port></code>
* @param hostAndPort Formatted as <code>&lt;hostname&gt; ':' &lt;port&gt;</code>
* @return The port portion of <code>hostAndPort</code>
*/
public static int parsePort(final String hostAndPort) {

View File

@ -55,7 +55,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* </p>
* <ul>
* <li>v2.2.1 - Fixed bug using URL_SAFE and ORDERED encodings. Fixed bug
* when using very small files (~< 40 bytes).</li>
* when using very small files (~&lt; 40 bytes).</li>
* <li>v2.2 - Added some helper methods for encoding/decoding directly from
* one file to the next. Also added a main() method to support command
* line encoding/decoding from one file to the next. Also added these

View File

@ -35,6 +35,8 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* </p>
* <p>
* This interface differs from ByteBuffer:
* </p>
* <ul>
* <li>On-heap bytes only</li>
* <li>Raw {@code byte} access only; does not encode other primitives.</li>
* <li>Implements {@code equals(Object)}, {@code #hashCode()}, and
@ -46,7 +48,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
* <li>Can be reused in tight loops like a major compaction which can save
* significant amounts of garbage. (Without reuse, we throw off garbage like
* <a href="http://www.youtube.com/watch?v=lkmBH-MjZF4">this thing</a>.)</li>
* </p>
* </ul>
* <p>
* Mutable, and always evaluates {@code #equals(Object)}, {@code #hashCode()},
* and {@code #compareTo(ByteRange)} based on the current contents.

View File

@ -1377,7 +1377,7 @@ public class Bytes implements Comparable<Bytes> {
* @param offset Offset into array at which vint begins.
* @throws java.io.IOException e
* @return deserialized long from buffer.
* @deprecated Use {@link #readAsVLong(byte[], int)} instead.
* @deprecated Use {@link #readAsVLong(byte[],int)} instead.
*/
@Deprecated
public static long readVLong(final byte [] buffer, final int offset)
@ -1409,7 +1409,7 @@ public class Bytes implements Comparable<Bytes> {
/**
* @param left left operand
* @param right right operand
* @return 0 if equal, < 0 if left is less than right, etc.
* @return 0 if equal, &lt; 0 if left is less than right, etc.
*/
public static int compareTo(final byte [] left, final byte [] right) {
return LexicographicalComparerHolder.BEST_COMPARER.
@ -1425,7 +1425,7 @@ public class Bytes implements Comparable<Bytes> {
* @param offset2 Where to start comparing in the right buffer
* @param length1 How much to compare from the left buffer
* @param length2 How much to compare from the right buffer
* @return 0 if equal, < 0 if left is less than right, etc.
* @return 0 if equal, &lt; 0 if left is less than right, etc.
*/
public static int compareTo(byte[] buffer1, int offset1, int length1,
byte[] buffer2, int offset2, int length2) {
@ -2213,7 +2213,7 @@ public class Bytes implements Comparable<Bytes> {
* Bytewise binary increment/deincrement of long contained in byte array
* on given amount.
*
* @param value - array of bytes containing long (length <= SIZEOF_LONG)
* @param value - array of bytes containing long (length &lt;= SIZEOF_LONG)
* @param amount value will be incremented on (deincremented if negative)
* @return array of bytes containing incremented long (length == SIZEOF_LONG)
*/

View File

@ -305,7 +305,7 @@ public class ClassSize {
/**
* Aligns a number to 8.
* @param num number to align to 8
* @return smallest number >= input that is a multiple of 8
* @return smallest number &gt;= input that is a multiple of 8
*/
public static int align(int num) {
return (int)(align((long)num));
@ -314,7 +314,7 @@ public class ClassSize {
/**
* Aligns a number to 8.
* @param num number to align to 8
* @return smallest number >= input that is a multiple of 8
* @return smallest number &gt;= input that is a multiple of 8
*/
public static long align(long num) {
//The 7 comes from that the alignSize is 8 which is the number of bytes

View File

@ -27,8 +27,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
public class DefaultEnvironmentEdge implements EnvironmentEdge {
/**
* {@inheritDoc}
* <p/>
* <p>
* This implementation returns {@link System#currentTimeMillis()}
* </p>
*/
@Override
public long currentTime() {

View File

@ -45,9 +45,10 @@ public class IncrementingEnvironmentEdge implements EnvironmentEdge {
/**
* {@inheritDoc}
* <p/>
* <p>
* This method increments a known value for the current time each time this
* method is called. The first value is 1.
* </p>
*/
@Override
public synchronized long currentTime() {

View File

@ -66,11 +66,11 @@ public class JenkinsHash extends Hash {
* <p>The best hash table sizes are powers of 2. There is no need to do mod
* a prime (mod is sooo slow!). If you need less than 32 bits, use a bitmask.
* For example, if you need only 10 bits, do
* <code>h = (h & hashmask(10));</code>
* <code>h = (h &amp; hashmask(10));</code>
* In which case, the hash table should have hashsize(10) elements.
*
* <p>If you are hashing n strings byte[][] k, do it like this:
* for (int i = 0, h = 0; i < n; ++i) h = hash( k[i], h);
* for (int i = 0, h = 0; i &lt; n; ++i) h = hash( k[i], h);
*
* <p>By Bob Jenkins, 2006. bob_jenkins@burtleburtle.net. You may use this
* code any way you wish, private, educational, or commercial. It's free.

View File

@ -36,8 +36,9 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
* A utility class to manage a set of locks. Each lock is identified by a String which serves
* as a key. Typical usage is: <p>
* class Example{
* private final static KeyLocker<String> locker = new Locker<String>();
* <p/>
* private final static KeyLocker&lt;String&gt; locker = new Locker&lt;String&gt;();
* </p>
* <p>
* public void foo(String s){
* Lock lock = locker.acquireLock(s);
* try {

View File

@ -40,7 +40,8 @@ import com.google.common.annotations.VisibleForTesting;
* Each value is encoded as one or more bytes. The first byte of the encoding,
* its meaning, and a terse description of the bytes that follow is given by
* the following table:
* <table>
* </p>
* <table summary="Encodings">
* <tr><th>Content Type</th><th>Encoding</th></tr>
* <tr><td>NULL</td><td>0x05</td></tr>
* <tr><td>negative infinity</td><td>0x07</td></tr>
@ -63,7 +64,6 @@ import com.google.common.annotations.VisibleForTesting;
* <tr><td>variable length BLOB</td><td>0x35, B</td></tr>
* <tr><td>byte-for-byte BLOB</td><td>0x36, X</td></tr>
* </table>
* </p>
*
* <h3>Null Encoding</h3>
* <p>
@ -258,8 +258,8 @@ import com.google.common.annotations.VisibleForTesting;
* values are 5 bytes in length.
* </p>
* <p>
* {@code OrderedBytes} encodings are heavily influenced by the <a href="
* http://sqlite.org/src4/doc/trunk/www/key_encoding.wiki">SQLite4 Key
* {@code OrderedBytes} encodings are heavily influenced by the
* <a href="http://sqlite.org/src4/doc/trunk/www/key_encoding.wiki">SQLite4 Key
* Encoding</a>. Slight deviations are make in the interest of order
* correctness and user extensibility. Fixed-width {@code Long} and
* {@link Double} encodings are based on implementations from the now defunct
@ -1408,6 +1408,7 @@ public class OrderedBytes {
* -Double.MIN_VALUE &lt; -0.0 &lt; +0.0; &lt; Double.MIN_VALUE &lt; ...
* &lt; Double.MAX_VALUE &lt; Double.POSITIVE_INFINITY &lt; Double.NaN
* </p>
* <p>
* Floating point numbers are encoded as specified in IEEE 754. A 64-bit
* double precision float consists of a sign bit, 11-bit unsigned exponent
* encoded in offset-1023 notation, and a 52-bit significand. The format is

View File

@ -68,9 +68,9 @@ public class Sleeper {
}
/**
* Sleep for period adjusted by passed <code>startTime<code>
* Sleep for period adjusted by passed <code>startTime</code>
* @param startTime Time some task started previous to now. Time to sleep
* will be docked current time minus passed <code>startTime<code>.
* will be docked current time minus passed <code>startTime</code>.
*/
public void sleep(final long startTime) {
if (this.stopper.isStopped()) {

View File

@ -102,7 +102,7 @@ public interface BaseSource {
/**
* Get the name of the context in JMX that this source will be exposed through.
* This is in ObjectName format. With the default context being Hadoop -> HBase
* This is in ObjectName format. With the default context being Hadoop -&gt; HBase
*/
String getMetricsJmxContext();

View File

@ -47,9 +47,10 @@ import org.apache.hadoop.hbase.util.ByteBufferUtils;
import org.apache.hadoop.io.WritableUtils;
/**
* <p>
* This class is created via reflection in DataBlockEncoding enum. Update the enum if class name or
* package changes.
* <p/>
* </p>
* PrefixTreeDataBlockEncoder implementation of DataBlockEncoder. This is the primary entry point
* for PrefixTree encoding and decoding. Encoding is delegated to instances of
* {@link PrefixTreeEncoder}, and decoding is delegated to instances of

View File

@ -63,8 +63,9 @@ public class PrefixTreeSeeker implements EncodedSeeker {
}
/**
* <p>
* Currently unused.
* <p/>
* </p>
* TODO performance leak. should reuse the searchers. hbase does not currently have a hook where
* this can be called
*/
@ -110,12 +111,13 @@ public class PrefixTreeSeeker implements EncodedSeeker {
}
/**
* <p>
* Currently unused.
* <p/>
* </p><p>
* A nice, lightweight reference, though the underlying cell is transient. This method may return
* the same reference to the backing PrefixTreeCell repeatedly, while other implementations may
* return a different reference for each Cell.
* <p/>
* </p>
* The goal will be to transition the upper layers of HBase, like Filters and KeyValueHeap, to
* use this method instead of the getKeyValue() methods above.
*/

View File

@ -25,10 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
/**
* <p>
* Pools PrefixTreeArraySearcher objects. Each Searcher can consist of hundreds or thousands of
* objects and 1 is needed for each HFile during a Get operation. With tens of thousands of
* Gets/second, reusing these searchers may save a lot of young gen collections.
* <p/>
* </p>
* Alternative implementation would be a ByteBufferSearcherPool (not implemented yet).
*/
@InterfaceAudience.Private

View File

@ -28,10 +28,11 @@ import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
import com.google.common.primitives.UnsignedBytes;
/**
* <p>
* Searcher extends the capabilities of the Scanner + ReversibleScanner to add the ability to
* position itself on a requested Cell without scanning through cells before it. The PrefixTree is
* set up to be a Trie of rows, so finding a particular row is extremely cheap.
* <p/>
* </p>
* Once it finds the row, it does a binary search through the cells inside the row, which is not as
* fast as the trie search, but faster than iterating through every cell like existing block
* formats
@ -309,8 +310,8 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
/****************** complete seek when token mismatch ******************/
/**
* @param searcherIsAfterInputKey <0: input key is before the searcher's position<br/>
* >0: input key is after the searcher's position
* @param searcherIsAfterInputKey &lt;0: input key is before the searcher's position<br/>
* &gt;0: input key is after the searcher's position
*/
protected CellScannerPosition fixRowTokenMissReverse(int searcherIsAfterInputKey) {
if (searcherIsAfterInputKey < 0) {//searcher position is after the input key, so back up
@ -337,8 +338,8 @@ public class PrefixTreeArraySearcher extends PrefixTreeArrayReversibleScanner im
}
/**
* @param searcherIsAfterInputKey <0: input key is before the searcher's position<br/>
* >0: input key is after the searcher's position
* @param searcherIsAfterInputKey &lt;0: input key is before the searcher's position<br>
* &gt;0: input key is after the searcher's position
*/
protected CellScannerPosition fixRowTokenMissForward(int searcherIsAfterInputKey) {
if (searcherIsAfterInputKey < 0) {//searcher position is after the input key

View File

@ -46,12 +46,12 @@ import org.apache.hadoop.io.WritableUtils;
/**
* This is the primary class for converting a CellOutputStream into an encoded byte[]. As Cells are
* added they are completely copied into the various encoding structures. This is important because
* usually the cells being fed in during compactions will be transient.<br/>
* <br/>
* Usage:<br/>
* 1) constructor<br/>
* 4) append cells in sorted order: write(Cell cell)<br/>
* 5) flush()<br/>
* usually the cells being fed in during compactions will be transient.<br>
* <br>
* Usage:<br>
* 1) constructor<br>
* 4) append cells in sorted order: write(Cell cell)<br>
* 5) flush()<br>
*/
@InterfaceAudience.Private
public class PrefixTreeEncoder implements CellOutputStream {
@ -391,10 +391,11 @@ public class PrefixTreeEncoder implements CellOutputStream {
}
/**
* <p>
* The following "compile" methods do any intermediate work necessary to transform the cell
* fragments collected during the writing phase into structures that are ready to write to the
* outputStream.
* <p/>
* </p>
* The family and qualifier treatment is almost identical, as is timestamp and mvccVersion.
*/

View File

@ -32,14 +32,17 @@ import org.apache.hadoop.hbase.util.vint.UFIntTool;
import org.apache.hadoop.hbase.util.vint.UVIntTool;
/**
* <p>
* Column nodes can be either family nodes or qualifier nodes, as both sections encode similarly.
* The family and qualifier sections of the data block are made of 1 or more of these nodes.
* <p/>
* Each node is composed of 3 sections:<br/>
* </p>
* Each node is composed of 3 sections:<br>
* <ul>
* <li>tokenLength: UVInt (normally 1 byte) indicating the number of token bytes
* <li>token[]: the actual token bytes
* <li>parentStartPosition: the offset of the next node from the start of the family or qualifier
* section
* </ul>
*/
@InterfaceAudience.Private
public class ColumnNodeWriter{

View File

@ -34,9 +34,10 @@ import org.apache.hadoop.hbase.util.vint.UFIntTool;
import com.google.common.collect.Lists;
/**
* <p>
* Takes the tokenized family or qualifier data and flattens it into a stream of bytes. The family
* section is written after the row section, and qualifier section after family section.
* <p/>
* </p>
* The family and qualifier tries, or "column tries", are structured differently than the row trie.
* The trie cannot be reassembled without external data about the offsets of the leaf nodes, and
* these external pointers are stored in the nubs and leaves of the row trie. For each cell in a
@ -45,12 +46,13 @@ import com.google.common.collect.Lists;
* comprises the column name. To assemble the column name, the trie is traversed in reverse (right
* to left), with the rightmost tokens pointing to the start of their "parent" node which is the
* node to the left.
* <p/>
* <p>
* This choice was made to reduce the size of the column trie by storing the minimum amount of
* offset data. As a result, to find a specific qualifier within a row, you must do a binary search
* of the column nodes, reassembling each one as you search. Future versions of the PrefixTree might
* encode the columns in both a forward and reverse trie, which would convert binary searches into
* more efficient trie searches which would be beneficial for wide rows.
* </p>
*/
@InterfaceAudience.Private
public class ColumnSectionWriter {

View File

@ -188,8 +188,9 @@ public class RowNodeWriter{
* offsets into the timestamp/column data structures that are written in the middle of the block.
* We use {@link UFIntTool} to encode these indexes/offsets to allow random access during a binary
* search of a particular column/timestamp combination.
* <p/>
* <p>
* Branch nodes will not have any data in these sections.
* </p>
*/
protected void writeFamilyNodeOffsets(OutputStream os) throws IOException {

View File

@ -31,10 +31,12 @@ import com.google.common.collect.Lists;
/**
* Data structure used in the first stage of PrefixTree encoding:
* <ul>
* <li>accepts a sorted stream of ByteRanges
* <li>splits them into a set of tokens, each held by a {@link TokenizerNode}
* <li>connects the TokenizerNodes via standard java references
* <li>keeps a pool of TokenizerNodes and a reusable byte[] for holding all token content
* </ul>
* <p><br>
* Mainly used for turning Cell rowKeys into a trie, but also used for family and qualifier
* encoding.

View File

@ -35,12 +35,12 @@ import com.google.common.collect.Lists;
* Individual node in a Trie structure. Each node is one of 3 types:
* <li>Branch: an internal trie node that may have a token and must have multiple children, but does
* not represent an actual input byte[], hence its numOccurrences is 0
* <li>Leaf: a node with no children and where numOccurrences is >= 1. It's token represents the
* <li>Leaf: a node with no children and where numOccurrences is &gt;= 1. It's token represents the
* last bytes in the input byte[]s.
* <li>Nub: a combination of a branch and leaf. Its token represents the last bytes of input
* byte[]s and has numOccurrences >= 1, but it also has child nodes which represent input byte[]s
* byte[]s and has numOccurrences &gt;= 1, but it also has child nodes which represent input byte[]s
* that add bytes to this nodes input byte[].
* <br/><br/>
* <br><br>
* Example inputs (numInputs=7):
* 0: AAA
* 1: AAA
@ -49,13 +49,13 @@ import com.google.common.collect.Lists;
* 4: AAB
* 5: AABQQ
* 6: AABQQ
* <br/><br/>
* <br><br>
* Resulting TokenizerNodes:
* AA <- branch, numOccurrences=0, tokenStartOffset=0, token.length=2
* A <- leaf, numOccurrences=2, tokenStartOffset=2, token.length=1
* B <- nub, numOccurrences=3, tokenStartOffset=2, token.length=1
* QQ <- leaf, numOccurrences=2, tokenStartOffset=3, token.length=2
* <br/><br/>
* AA &lt;- branch, numOccurrences=0, tokenStartOffset=0, token.length=2
* A &lt;- leaf, numOccurrences=2, tokenStartOffset=2, token.length=1
* B &lt;- nub, numOccurrences=3, tokenStartOffset=2, token.length=1
* QQ &lt;- leaf, numOccurrences=2, tokenStartOffset=3, token.length=2
* <br><br>
* numInputs == 7 == sum(numOccurrences) == 0 + 2 + 3 + 2
*/
@InterfaceAudience.Private
@ -236,13 +236,15 @@ public class TokenizerNode{
/**
* Called when we need to convert a leaf node into a branch with 2 leaves. Comments inside the
* method assume we have token BAA starting at tokenStartOffset=0 and are adding BOO. The output
* will be 3 nodes:<br/>
* <li>1: B <- branch
* <li>2: AA <- leaf
* <li>3: OO <- leaf
* will be 3 nodes:<br>
* <ul>
* <li>1: B &lt;- branch
* <li>2: AA &lt;- leaf
* <li>3: OO &lt;- leaf
* </ul>
*
* @param numTokenBytesToRetain => 1 (the B)
* @param bytes => BOO
* @param numTokenBytesToRetain =&gt; 1 (the B)
* @param bytes =&gt; BOO
*/
protected void split(int numTokenBytesToRetain, final ByteRange bytes) {
int childNodeDepth = nodeDepth;

View File

@ -33,19 +33,22 @@ public interface CellSearcher extends ReversibleCellScanner {
void resetToBeforeFirstEntry();
/**
* <p>
* Do everything within this scanner's power to find the key. Look forward and backwards.
* <p/>
* </p>
* <p>
* Abort as soon as we know it can't be found, possibly leaving the Searcher in an invalid state.
* <p/>
* </p>
* @param key position the CellScanner exactly on this key
* @return true if the cell existed and getCurrentCell() holds a valid cell
*/
boolean positionAt(Cell key);
/**
* <p>
* Same as positionAt(..), but go to the extra effort of finding the previous key if there's no
* exact match.
* <p/>
* </p>
* @param key position the CellScanner on this key or the closest cell before
* @return AT if exact match<br/>
* BEFORE if on last cell before key<br/>
@ -54,9 +57,10 @@ public interface CellSearcher extends ReversibleCellScanner {
CellScannerPosition positionAtOrBefore(Cell key);
/**
* <p>
* Same as positionAt(..), but go to the extra effort of finding the next key if there's no exact
* match.
* <p/>
* </p>
* @param key position the CellScanner on this key or the closest cell after
* @return AT if exact match<br/>
* AFTER if on first cell after key<br/>
@ -65,43 +69,47 @@ public interface CellSearcher extends ReversibleCellScanner {
CellScannerPosition positionAtOrAfter(Cell key);
/**
* <p>
* Note: Added for backwards compatibility with
* {@link org.apache.hadoop.hbase.regionserver.KeyValueScanner#reseek}
* <p/>
* </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
* as unencoded KeyValue blocks.
* <p/>
* </p>
* @param key position the CellScanner exactly on this key
* @return true if getCurrent() holds a valid cell
*/
boolean seekForwardTo(Cell key);
/**
* <p>
* Same as seekForwardTo(..), but go to the extra effort of finding the next key if there's no
* exact match.
* <p/>
* </p>
* @param key
* @return AT if exact match<br/>
* AFTER if on first cell after key<br/>
* @return AT if exact match<br>
* AFTER if on first cell after key<br>
* AFTER_LAST if key was after the last cell in this scanner's scope
*/
CellScannerPosition seekForwardToOrBefore(Cell key);
/**
* <p>
* Same as seekForwardTo(..), but go to the extra effort of finding the next key if there's no
* exact match.
* <p/>
* </p>
* @param key
* @return AT if exact match<br/>
* AFTER if on first cell after key<br/>
* @return AT if exact match<br>
* AFTER if on first cell after key<br>
* AFTER_LAST if key was after the last cell in this scanner's scope
*/
CellScannerPosition seekForwardToOrAfter(Cell key);
/**
* <p>
* Note: This may not be appropriate to have in the interface. Need to investigate.
* <p/>
* </p>
* Position the scanner in an invalid state after the last cell: CellScannerPosition.AFTER_LAST.
* This is used by tests and for handling certain edge cases.
*/

View File

@ -35,7 +35,7 @@ public interface ReversibleCellScanner extends CellScanner {
/**
* Try to position the scanner one Cell before the current position.
* @return true if the operation was successful, meaning getCurrentCell() will return a valid
* Cell.<br/>
* Cell.<br>
* false if there were no previous cells, meaning getCurrentCell() will return null.
* Scanner position will be
* {@link org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition#BEFORE_FIRST}
@ -46,7 +46,7 @@ public interface ReversibleCellScanner extends CellScanner {
* Try to position the scanner in the row before the current row.
* @param endOfRow true for the last cell in the previous row; false for the first cell
* @return true if the operation was successful, meaning getCurrentCell() will return a valid
* Cell.<br/>
* Cell.<br>
* false if there were no previous cells, meaning getCurrentCell() will return null.
* Scanner position will be
* {@link org.apache.hadoop.hbase.codec.prefixtree.scanner.CellScannerPosition#BEFORE_FIRST}

View File

@ -29,10 +29,10 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
* This class converts between positive ints and 1-4 bytes that represent the int. All input ints
* must be positive. Max values stored in N bytes are:
*
* N=1: 2^8 => 256
* N=2: 2^16 => 65,536
* N=3: 2^24 => 16,777,216
* N=4: 2^31 => 2,147,483,648 (Integer.MAX_VALUE)
* N=1: 2^8 =&gt; 256
* N=2: 2^16 =&gt; 65,536
* N=3: 2^24 =&gt; 16,777,216
* N=4: 2^31 =&gt; 2,147,483,648 (Integer.MAX_VALUE)
*
* This was created to get most of the memory savings of a variable length integer when encoding
* an array of input integers, but to fix the number of bytes for each integer to the number needed

View File

@ -42,7 +42,7 @@ public class UVIntTool {
public static final byte[]
MAX_VALUE_BYTES = new byte[] { -1, -1, -1, -1, 7 };
/********************* int -> bytes **************************/
/********************* int -&gt; bytes **************************/
public static int numBytes(int in) {
if (in == 0) {
@ -79,7 +79,7 @@ public class UVIntTool {
return numBytes;
}
/******************** bytes -> int **************************/
/******************** bytes -&gt; int **************************/
public static int getInt(byte[] bytes) {
return getInt(bytes, 0);

View File

@ -43,7 +43,7 @@ public class UVLongTool{
MAX_VALUE_BYTES = new byte[] { -1, -1, -1, -1, -1, -1, -1, -1, 127 };
/********************* long -> bytes **************************/
/********************* long -&gt; bytes **************************/
public static int numBytes(long in) {// do a check for illegal arguments if not protected
if (in == 0) {
@ -77,7 +77,7 @@ public class UVLongTool{
return numBytes;
}
/******************** bytes -> long **************************/
/******************** bytes -&gt; long **************************/
public static long getLong(byte[] bytes) {
return getLong(bytes, 0);

View File

@ -50,9 +50,9 @@ import com.google.protobuf.ByteString;
* the return is a set of sub-procedures or null in case the procedure doesn't
* have sub-procedures. Once the sub-procedures are successfully completed
* the execute() method is called again, you should think at it as a stack:
* -> step 1
* ---> step 2
* -> step 1
* -&gt; step 1
* ---&gt; step 2
* -&gt; step 1
*
* rollback() is called when the procedure or one of the sub-procedures is failed.
* the rollback step is supposed to cleanup the resources created during the

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ProcedureProtos.SequentialProc
/**
* A SequentialProcedure describes one step in a procedure chain.
* -> Step 1 -> Step 2 -> Step 3
* -&gt; Step 1 -&gt; Step 2 -&gt; Step 3
*
* The main difference from a base Procedure is that the execute() of a
* SequentialProcedure will be called only once, there will be no second
@ -79,4 +79,4 @@ public abstract class SequentialProcedure<TEnvironment> extends Procedure<TEnvir
SequentialProcedureData data = SequentialProcedureData.parseDelimitedFrom(stream);
executed = data.getExecuted();
}
}
}

View File

@ -37,6 +37,8 @@ public final class HBaseZeroCopyByteString extends LiteralByteString {
/**
* Wraps a byte array in a {@link ByteString} without copying it.
* @param array array to be wrapped
* @return wrapped array
*/
public static ByteString wrap(final byte[] array) {
return new LiteralByteString(array);
@ -44,6 +46,10 @@ public final class HBaseZeroCopyByteString extends LiteralByteString {
/**
* Wraps a subset of a byte array in a {@link ByteString} without copying it.
* @param array array to be wrapped
* @param offset from
* @param length length
* @return wrapped array
*/
public static ByteString wrap(final byte[] array, int offset, int length) {
return new BoundedByteString(array, offset, length);

View File

@ -99,7 +99,7 @@ public interface InterProcessLock {
/**
* Visits the locks (both held and attempted) of this type with the given
* {@link MetadataHandler}.
* @throws InterruptedException If there is an unrecoverable error
* @throws IOException If there is an unrecoverable error
*/
void visitLocks(MetadataHandler handler) throws IOException;
}

View File

@ -38,7 +38,7 @@ public class HFileArchiveTableMonitor {
* Set the tables to be archived. Internally adds each table and attempts to
* register it.
* <p>
* <b>Note: All previous tables will be removed in favor of these tables.<b>
* <b>Note: All previous tables will be removed in favor of these tables.</b>
* @param tables add each of the tables to be archived.
*/
public synchronized void setArchiveTables(List<String> tables) {

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
* {@link org.apache.hadoop.hbase.client.Put}.
* <p>Does <b>NOT</b> attempt the
* {@link org.apache.hadoop.hbase.client.Put} multiple times,
* since the constraint <it>should</it> fail every time for
* since the constraint <b>should</b> fail every time for
* the same {@link org.apache.hadoop.hbase.client.Put} (it should be
* idempotent).
*/

View File

@ -19,7 +19,6 @@
/**
* Restrict the domain of a data attribute, often times to fulfill business rules/requirements.
*
<p>
<h2> Table of Contents</h2>
<ul>
<li><a href="#overview">Overview</a></li>
@ -27,7 +26,6 @@
<li><a href="#caveats">Caveats</a></li>
<li><a href="#usage">Example Usage</a></li>
</ul>
</p>
<h2><a name="overview">Overview</a></h2>
Constraints are used to enforce business rules in a database.
@ -127,9 +125,9 @@
public class IntegerConstraint extends BaseConstraint {
public void check(Put p) throws ConstraintException {
Map&ltbyte[], List&ltKeyValue&gt&gt familyMap = p.getFamilyMap();
Map&lt;byte[], List&lt;KeyValue&gt;&gt; familyMap = p.getFamilyMap();
for (List &ltKeyValue&gt kvs : familyMap.values()) {
for (List &lt;KeyValue&gt; kvs : familyMap.values()) {
for (KeyValue kv : kvs) {
// just make sure that we can actually pull out an int

View File

@ -137,7 +137,7 @@ public interface SplitLogManagerCoordination {
* It removes recovering regions from Coordination
* @param serverNames servers which are just recovered
* @param isMetaRecovery whether current recovery is for the meta region on
* <code>serverNames<code>
* <code>serverNames</code>
*/
void removeRecoveringRegions(Set<String> serverNames, Boolean isMetaRecovery) throws IOException;

View File

@ -290,7 +290,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
* region server hosting the region can allow reads to the recovered region
* @param recoveredServerNameSet servers which are just recovered
* @param isMetaRecovery whether current recovery is for the meta region on
* <code>serverNames<code>
* <code>serverNames</code>
*/
@Override
public void removeRecoveringRegions(final Set<String> recoveredServerNameSet,
@ -684,8 +684,7 @@ public class ZKSplitLogManagerCoordination extends ZooKeeperListener implements
/**
* ZooKeeper implementation of
* {@link org.apache.hadoop.hbase.coordination.
* SplitLogManagerCoordination#removeStaleRecoveringRegions(Set)}
* {@link SplitLogManagerCoordination#removeStaleRecoveringRegions(Set)}
*/
@Override
public void removeStaleRecoveringRegions(final Set<String> knownFailedServers)

View File

@ -51,11 +51,11 @@ import com.google.protobuf.Service;
* {@link ColumnInterpreter} is used to interpret column value. This class is
* parameterized with the following (these are the types with which the {@link ColumnInterpreter}
* is parameterized, and for more description on these, refer to {@link ColumnInterpreter}):
* @param <T> Cell value data type
* @param <S> Promoted data type
* @param <P> PB message that is used to transport initializer specific bytes
* @param <Q> PB message that is used to transport Cell (<T>) instance
* @param <R> PB message that is used to transport Promoted (<S>) instance
* @param T Cell value data type
* @param S Promoted data type
* @param P PB message that is used to transport initializer specific bytes
* @param Q PB message that is used to transport Cell (&lt;T&gt;) instance
* @param R PB message that is used to transport Promoted (&lt;S&gt;) instance
*/
@InterfaceAudience.Private
public class AggregateImplementation<T, S, P extends Message, Q extends Message, R extends Message>
@ -229,7 +229,6 @@ extends AggregateService implements CoprocessorService, Coprocessor {
/**
* Gives the row count for the given column family and column qualifier, in
* the given row range as defined in the Scan object.
* @throws IOException
*/
@Override
public void getRowNum(RpcController controller, AggregateRequest request,

View File

@ -52,14 +52,14 @@ import com.google.protobuf.Service;
*
* Defines a protocol to perform multi row transactions.
* See {@link MultiRowMutationEndpoint} for the implementation.
* </br>
* <br>
* See
* {@link HRegion#mutateRowsWithLocks(java.util.Collection, java.util.Collection)}
* for details and limitations.
* </br>
* <br>
* Example:
* <code><pre>
* List<Mutation> mutations = ...;
* <code>
* List&lt;Mutation&gt; mutations = ...;
* Put p1 = new Put(row1);
* Put p2 = new Put(row2);
* ...
@ -73,7 +73,7 @@ import com.google.protobuf.Service;
* MultiRowMutationService.newBlockingStub(channel);
* MutateRowsRequest mrm = mrmBuilder.build();
* service.mutateRows(null, mrm);
* </pre></code>
* </code>
*/
@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC)
@InterfaceStability.Evolving

View File

@ -344,7 +344,7 @@ public interface RegionObserver extends Coprocessor {
* (e.getRegion() returns the parent region)
* @throws IOException if an error occurred on the coprocessor
* @deprecated Use preSplit(
* final ObserverContext<RegionCoprocessorEnvironment> c, byte[] splitRow)
* final ObserverContext&lt;RegionCoprocessorEnvironment&gt; c, byte[] splitRow)
*/
@Deprecated
void preSplit(final ObserverContext<RegionCoprocessorEnvironment> c) throws IOException;
@ -1068,7 +1068,8 @@ public interface RegionObserver extends Coprocessor {
* <li>
* <code>boolean filterRow()</code> returning true</li>
* <li>
* <code>void filterRow(List<KeyValue> kvs)</code> removing all the kvs from the passed List</li>
* <code>void filterRow(List&lt;KeyValue&gt; kvs)</code> removing all the kvs
* from the passed List</li>
* </ol>
* @param c the environment provided by the region server
* @param s the scanner
@ -1095,7 +1096,8 @@ public interface RegionObserver extends Coprocessor {
* <li>
* <code>boolean filterRow()</code> returning true</li>
* <li>
* <code>void filterRow(List<KeyValue> kvs)</code> removing all the kvs from the passed List</li>
* <code>void filterRow(List<KeyValue> kvs)</code> removing all the kvs from
* the passed List</li>
* </ol>
* @param c the environment provided by the region server
* @param s the scanner

View File

@ -62,7 +62,7 @@ when the corresponding events happen. The master transitions regions
through the following states:
<p>
&nbsp;&nbsp;&nbsp;
unassigned -> pendingOpen -> open -> pendingClose -> closed.
unassigned -&gt; pendingOpen -&gt; open -&gt; pendingClose -7gt; closed.
<p>
Coprocessors have opportunity to intercept and handle events in
pendingOpen, open, and pendingClose states.
@ -75,7 +75,7 @@ can piggyback or fail this process.
<p>
<ul>
<li>preOpen, postOpen: Called before and after the region is reported as
online to the master.</li><p>
online to the master.</li>
</ul>
<p>
<h3>Open</h3>
@ -85,9 +85,9 @@ split, etc.). Coprocessors can piggyback administrative actions via:
<p>
<ul>
<li>preFlush, postFlush: Called before and after the memstore is flushed
into a new store file.</li><p>
<li>preCompact, postCompact: Called before and after compaction.</li><p>
<li>preSplit, postSplit: Called after the region is split.</li><p>
into a new store file.</li>
<li>preCompact, postCompact: Called before and after compaction.</li>
<li>preSplit, postSplit: Called after the region is split.</li>
</ul>
<p>
<h3>PendingClose</h3>
@ -99,7 +99,7 @@ an indication to this effect will be passed as an argument.
<p>
<ul>
<li>preClose and postClose: Called before and after the region is
reported as closed to the master.</li><p>
reported as closed to the master.</li>
</ul>
<p>
@ -109,23 +109,23 @@ observe and mediate client actions on the region:
<p>
<ul>
<li>preGet, postGet: Called before and after a client makes a Get
request.</li><p>
request.</li>
<li>preExists, postExists: Called before and after the client tests
for existence using a Get.</li><p>
for existence using a Get.</li>
<li>prePut and postPut: Called before and after the client stores a value.
</li><p>
</li>
<li>preDelete and postDelete: Called before and after the client
deletes a value.</li><p>
deletes a value.</li>
<li>preScannerOpen postScannerOpen: Called before and after the client
opens a new scanner.</li><p>
opens a new scanner.</li>
<li>preScannerNext, postScannerNext: Called before and after the client
asks for the next row on a scanner.</li><p>
asks for the next row on a scanner.</li>
<li>preScannerClose, postScannerClose: Called before and after the client
closes a scanner.</li><p>
closes a scanner.</li>
<li>preCheckAndPut, postCheckAndPut: Called before and after the client
calls checkAndPut().</li><p>
calls checkAndPut().</li>
<li>preCheckAndDelete, postCheckAndDelete: Called before and after the client
calls checkAndDelete().</li><p>
calls checkAndDelete().</li>
</ul>
You can also extend abstract class <code>BaseRegionObserverCoprocessor</code>
which
@ -245,7 +245,7 @@ recognize and load it.
</div>
<p>
&lt;path&gt; must point to a jar, can be on any filesystem supported by the
Hadoop </code>FileSystem</code> object.
Hadoop <code>FileSystem</code> object.
<p>
&lt;class&gt; is the coprocessor implementation class. A jar can contain
more than one coprocessor implementation, but only one can be specified
@ -270,7 +270,7 @@ policy implementations, perhaps) ahead of observers.
":" + Coprocessor.Priority.USER);
HBaseAdmin admin = new HBaseAdmin(this.conf);
admin.createTable(htd);
</pre></blockquote>
<h3>Chain of RegionObservers</h3>
As described above, multiple coprocessors can be loaded at one region at the
same time. In case of RegionObserver, you can have more than one
@ -278,8 +278,6 @@ RegionObservers register to one same hook point, i.e, preGet(), etc.
When a region reach the
hook point, the framework will invoke each registered RegionObserver by the
order of assigned priority.
</pre></blockquote>
</div>
*/

View File

@ -38,7 +38,8 @@ public class TimeoutException extends Exception {
* Exception indicating that an operation attempt has timed out
* @param start time the operation started (ms since epoch)
* @param end time the timeout was triggered (ms since epoch)
* @param expected expected amount of time for the operation to complete (ms) (ideally, expected <= end-start)
* @param expected expected amount of time for the operation to complete (ms)
* (ideally, expected &lt;= end-start)
*/
public TimeoutException(String sourceName, long start, long end, long expected) {
super("Timeout elapsed! Source:" + sourceName + " Start:" + start + ", End:" + end

View File

@ -94,9 +94,9 @@ import com.sun.jersey.spi.container.servlet.ServletContainer;
* Create a Jetty embedded server to answer http requests. The primary goal
* is to serve up status information for the server.
* There are three contexts:
* "/logs/" -> points to the log directory
* "/static/" -> points to common static files (src/webapps/static)
* "/" -> the jsp server code from (src/webapps/<name>)
* "/logs/" -&gt; points to the log directory
* "/static/" -&gt; points to common static files (src/webapps/static)
* "/" -&gt; the jsp server code from (src/webapps/<name>)
*/
@InterfaceAudience.Private
@InterfaceStability.Evolving
@ -425,7 +425,7 @@ public class HttpServer implements FilterContainer {
/**
* Create a status server on the given port.
* The jsp scripts are taken from src/webapps/<name>.
* The jsp scripts are taken from src/webapps/&lt;name&gt;.
* @param name The name of the server
* @param port The port to use on the server
* @param findPort whether the server should start at the given port and
@ -1108,13 +1108,14 @@ public class HttpServer implements FilterContainer {
/**
* Checks the user has privileges to access to instrumentation servlets.
* <p/>
* <p>
* If <code>hadoop.security.instrumentation.requires.admin</code> is set to FALSE
* (default value) it always returns TRUE.
* <p/>
* </p><p>
* If <code>hadoop.security.instrumentation.requires.admin</code> is set to TRUE
* it will check that if the current user is in the admin ACLS. If the user is
* in the admin ACLs it returns TRUE, otherwise it returns FALSE.
* </p>
*
* @param servletContext the servlet context.
* @param request the servlet request.

View File

@ -16,9 +16,8 @@
* limitations under the License.
*/
/**
* </ul>
* <p>
* Copied from hadoop source code.<br/>
* Copied from hadoop source code.<br>
* See https://issues.apache.org/jira/browse/HADOOP-10232 to know why.
* </p>
*/

View File

@ -91,7 +91,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
public class FileLink {
private static final Log LOG = LogFactory.getLog(FileLink.class);
/** Define the Back-reference directory name prefix: .links-<hfile>/ */
/** Define the Back-reference directory name prefix: .links-&lt;hfile&gt;/ */
public static final String BACK_REFERENCES_DIRECTORY_PREFIX = ".links-";
/**

View File

@ -401,7 +401,6 @@ public class HFileLink extends FileLink {
* @param rootDir root hbase directory
* @param linkRefPath Link Back Reference path
* @return full path of the referenced hfile
* @throws IOException on unexpected error.
*/
public static Path getHFileFromBackReference(final Path rootDir, final Path linkRefPath) {
Pair<TableName, String> p = parseBackReferenceName(linkRefPath.getName());

View File

@ -29,8 +29,8 @@ import org.apache.hadoop.hbase.util.FSUtils;
/**
* WALLink describes a link to a WAL.
*
* An wal can be in /hbase/.logs/<server>/<wal>
* or it can be in /hbase/.oldlogs/<wal>
* An wal can be in /hbase/.logs/&lt;server&gt;/&lt;wal&gt;
* or it can be in /hbase/.oldlogs/&lt;wal&gt;
*
* The link checks first in the original path,
* if it is not present it fallbacks to the archived path.

Some files were not shown because too many files have changed in this diff Show More