HBASE-14890 Fix broken checkstyle tool. Some javadocs had parsing errors because of wrongly formatted HTML tags. (Apekshit)
Signed-off-by: stack <stack@apache.org>
This commit is contained in:
parent
2317ec583d
commit
3bac31b2a4
|
@ -198,7 +198,7 @@ public abstract class Query extends OperationWithAttributes {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return Map<byte[], TimeRange> a map of column families to time ranges
|
||||
* @return A map of column families to time ranges
|
||||
*/
|
||||
public Map<byte[], TimeRange> getColumnFamilyTimeRange() {
|
||||
return this.colFamTimeRangeMap;
|
||||
|
|
|
@ -792,7 +792,7 @@ public class RpcClientImpl extends AbstractRpcClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Write the RPC header: <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>
|
||||
* Write the RPC header: {@code <MAGIC WORD -- 'HBas'> <ONEBYTE_VERSION> <ONEBYTE_AUTH_TYPE>}
|
||||
*/
|
||||
private void writeConnectionHeaderPreamble(OutputStream outStream) throws IOException {
|
||||
// Assemble the preamble up in a buffer first and then send it. Writing individual elements,
|
||||
|
|
|
@ -924,7 +924,7 @@ public final class RequestConverter {
|
|||
}
|
||||
|
||||
/**
|
||||
* @see {@link #buildRollWALWriterRequest()
|
||||
* @see {@link #buildRollWALWriterRequest()}
|
||||
*/
|
||||
private static RollWALWriterRequest ROLL_WAL_WRITER_REQUEST =
|
||||
RollWALWriterRequest.newBuilder().build();
|
||||
|
|
|
@ -63,7 +63,7 @@ public class ReplicationQueueInfo {
|
|||
/**
|
||||
* Parse dead server names from znode string servername can contain "-" such as
|
||||
* "ip-10-46-221-101.ec2.internal", so we need skip some "-" during parsing for the following
|
||||
* cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-<server name>-...
|
||||
* cases: 2-ip-10-46-221-101.ec2.internal,52170,1364333181125-<server name>-...
|
||||
*/
|
||||
private static void
|
||||
extractDeadServersFromZNodeString(String deadServerListStr, List<String> result) {
|
||||
|
|
|
@ -883,12 +883,12 @@ public final class HConstants {
|
|||
Pattern.compile("^coprocessor\\$([0-9]+)$", Pattern.CASE_INSENSITIVE);
|
||||
|
||||
/**
|
||||
* <pre>
|
||||
* Pattern that matches a coprocessor specification. Form is:
|
||||
* <code>
|
||||
*<coprocessor jar file location> '|' <<class name> ['|' <priority> ['|' <arguments>]]
|
||||
* </code>
|
||||
* ...where arguments are <code><KEY> '=' <VALUE> [,...]</code>
|
||||
* <p>For example: <code>hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2</code>
|
||||
* {@code <coprocessor jar file location> '|' <class name> ['|' <priority> ['|' <arguments>]]}
|
||||
* where arguments are {@code <KEY> '=' <VALUE> [,...]}
|
||||
* For example: {@code hdfs:///foo.jar|com.foo.FooRegionObserver|1001|arg1=1,arg2=2}
|
||||
* </pre>
|
||||
*/
|
||||
public static final Pattern CP_HTD_ATTR_VALUE_PATTERN =
|
||||
Pattern.compile("(^[^\\|]*)\\|([^\\|]+)\\|[\\s]*([\\d]*)[\\s]*(\\|.*)?$");
|
||||
|
|
|
@ -42,24 +42,23 @@ public class Tag {
|
|||
private int offset = 0;
|
||||
private int length = 0;
|
||||
|
||||
// The special tag will write the length of each tag and that will be
|
||||
// followed by the type and then the actual tag.
|
||||
// So every time the length part is parsed we need to add + 1 byte to it to
|
||||
// get the type and then get the actual tag.
|
||||
/**
|
||||
* The special tag will write the length of each tag and that will be
|
||||
* followed by the type and then the actual tag.
|
||||
* So every time the length part is parsed we need to add + 1 byte to it to
|
||||
* get the type and then get the actual tag.
|
||||
*/
|
||||
public Tag(byte tagType, String tag) {
|
||||
this(tagType, Bytes.toBytes(tag));
|
||||
}
|
||||
|
||||
/**
|
||||
* @param tagType
|
||||
* @param tag
|
||||
* Format for a tag :
|
||||
* {@code <length of tag - 2 bytes><type code - 1 byte><tag>} tag length is serialized
|
||||
* using 2 bytes only but as this will be unsigned, we can have max tag length of
|
||||
* (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
|
||||
*/
|
||||
public Tag(byte tagType, byte[] tag) {
|
||||
/**
|
||||
* Format for a tag : <length of tag - 2 bytes><type code - 1 byte><tag> taglength is serialized
|
||||
* using 2 bytes only but as this will be unsigned, we can have max taglength of
|
||||
* (Short.MAX_SIZE * 2) +1. It includes 1 byte type length and actual tag bytes length.
|
||||
*/
|
||||
int tagLength = tag.length + TYPE_LENGTH_SIZE;
|
||||
if (tagLength > MAX_TAG_LENGTH) {
|
||||
throw new IllegalArgumentException(
|
||||
|
@ -78,10 +77,7 @@ public class Tag {
|
|||
* <code>bytes</code> content starting at <code>offset</code> is formatted as
|
||||
* a Tag blob.
|
||||
* The bytes to include the tag type, tag length and actual tag bytes.
|
||||
* @param bytes
|
||||
* byte array
|
||||
* @param offset
|
||||
* offset to start of Tag
|
||||
* @param offset offset to start of Tag
|
||||
*/
|
||||
public Tag(byte[] bytes, int offset) {
|
||||
this(bytes, offset, getLength(bytes, offset));
|
||||
|
@ -95,12 +91,6 @@ public class Tag {
|
|||
* Creates a Tag from the specified byte array, starting at offset, and for length
|
||||
* <code>length</code>. Presumes <code>bytes</code> content starting at <code>offset</code> is
|
||||
* formatted as a Tag blob.
|
||||
* @param bytes
|
||||
* byte array
|
||||
* @param offset
|
||||
* offset to start of the Tag
|
||||
* @param length
|
||||
* length of the Tag
|
||||
*/
|
||||
public Tag(byte[] bytes, int offset, int length) {
|
||||
if (length > MAX_TAG_LENGTH) {
|
||||
|
|
|
@ -154,12 +154,10 @@ public final class Constraints {
|
|||
/**
|
||||
* Get the kv {@link Entry} in the descriptor for the specified class
|
||||
*
|
||||
* @param desc
|
||||
* {@link HTableDescriptor} to read
|
||||
* @param clazz
|
||||
* to search for
|
||||
* @return the {@link Pair} of <key, value> in the table, if that class is
|
||||
* present. <tt>null</tt> otherwise.
|
||||
* @param desc {@link HTableDescriptor} to read
|
||||
* @param clazz To search for
|
||||
* @return The {@link Pair} of {@literal <key, value>} in the table, if that class is
|
||||
* present. {@code NULL} otherwise.
|
||||
*/
|
||||
private static Pair<String, String> getKeyValueForClass(
|
||||
HTableDescriptor desc, Class<? extends Constraint> clazz) {
|
||||
|
|
|
@ -806,7 +806,7 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
* Process all that are passed in even if failure being sure to remove from ramCache else we'll
|
||||
* never undo the references and we'll OOME.
|
||||
* @param entries Presumes list passed in here will be processed by this invocation only. No
|
||||
* interference expected.
|
||||
* interference expected.
|
||||
* @throws InterruptedException
|
||||
*/
|
||||
@VisibleForTesting
|
||||
|
@ -911,23 +911,23 @@ public class BucketCache implements BlockCache, HeapSize {
|
|||
}
|
||||
|
||||
/**
|
||||
* Blocks until elements available in <code>q</code> then tries to grab as many as possible
|
||||
* Blocks until elements available in {@code q} then tries to grab as many as possible
|
||||
* before returning.
|
||||
* @param recepticle Where to stash the elements taken from queue. We clear before we use it
|
||||
* just in case.
|
||||
* @param receptacle Where to stash the elements taken from queue. We clear before we use it
|
||||
* just in case.
|
||||
* @param q The queue to take from.
|
||||
* @return <code>receptical laden with elements taken from the queue or empty if none found.
|
||||
* @return {@code receptacle} laden with elements taken from the queue or empty if none found.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
static List<RAMQueueEntry> getRAMQueueEntries(final BlockingQueue<RAMQueueEntry> q,
|
||||
final List<RAMQueueEntry> receptical)
|
||||
final List<RAMQueueEntry> receptacle)
|
||||
throws InterruptedException {
|
||||
// Clear sets all entries to null and sets size to 0. We retain allocations. Presume it
|
||||
// ok even if list grew to accommodate thousands.
|
||||
receptical.clear();
|
||||
receptical.add(q.take());
|
||||
q.drainTo(receptical);
|
||||
return receptical;
|
||||
receptacle.clear();
|
||||
receptacle.add(q.take());
|
||||
q.drainTo(receptacle);
|
||||
return receptacle;
|
||||
}
|
||||
|
||||
private void persistToFile() throws IOException {
|
||||
|
|
|
@ -521,8 +521,7 @@ public class LoadIncrementalHFiles extends Configured implements Tool {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return A Multimap<startkey, LoadQueueItem> that groups LQI by likely
|
||||
* bulk load region targets.
|
||||
* @return A map that groups LQI by likely bulk load region targets.
|
||||
*/
|
||||
private Multimap<ByteBuffer, LoadQueueItem> groupOrSplitPhase(final Table table,
|
||||
ExecutorService pool, Deque<LoadQueueItem> queue,
|
||||
|
|
|
@ -143,10 +143,7 @@ public class ServerManager {
|
|||
private final Map<ServerName, AdminService.BlockingInterface> rsAdmins =
|
||||
new HashMap<ServerName, AdminService.BlockingInterface>();
|
||||
|
||||
/**
|
||||
* List of region servers <ServerName> that should not get any more new
|
||||
* regions.
|
||||
*/
|
||||
/** List of region servers that should not get any more new regions. */
|
||||
private final ArrayList<ServerName> drainingServers =
|
||||
new ArrayList<ServerName>();
|
||||
|
||||
|
|
|
@ -399,8 +399,7 @@ public class SplitLogManager {
|
|||
* It removes recovering regions under /hbase/recovering-regions/[encoded region name] so that the
|
||||
* region server hosting the region can allow reads to the recovered region
|
||||
* @param serverNames servers which are just recovered
|
||||
* @param isMetaRecovery whether current recovery is for the meta region on
|
||||
* <code>serverNames<code>
|
||||
* @param isMetaRecovery whether current recovery is for the meta region on {@code serverNames}
|
||||
*/
|
||||
private void removeRecoveringRegions(final Set<ServerName> serverNames, Boolean isMetaRecovery) {
|
||||
if (!isLogReplaying()) {
|
||||
|
|
|
@ -35,8 +35,8 @@ import org.apache.zookeeper.KeeperException;
|
|||
|
||||
/**
|
||||
* The mapper of a sweep job.
|
||||
* Takes the rows from the table and their results and map to <filename:Text, mobValue:KeyValue>
|
||||
* where mobValue is the actual cell in HBase.
|
||||
* Takes the rows from the table and their results and map to {@literal <filename:Text,
|
||||
* mobValue:KeyValue>} where mobValue is the actual cell in HBase.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class SweepMapper extends TableMapper<Text, KeyValue> {
|
||||
|
|
|
@ -157,7 +157,7 @@ public class TaskMonitor {
|
|||
/**
|
||||
* This class encapsulates an object as well as a weak reference to a proxy
|
||||
* that passes through calls to that object. In art form:
|
||||
* <code>
|
||||
* <pre>
|
||||
* Proxy <------------------
|
||||
* | \
|
||||
* v \
|
||||
|
@ -166,7 +166,7 @@ public class TaskMonitor {
|
|||
* MonitoredTaskImpl /
|
||||
* | /
|
||||
* StatAndWeakRefProxy ------/
|
||||
*
|
||||
* </pre>
|
||||
* Since we only return the Proxy to the creator of the MonitorableStatus,
|
||||
* this means that they can leak that object, and we'll detect it
|
||||
* since our weak reference will go null. But, we still have the actual
|
||||
|
|
|
@ -5234,7 +5234,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
* Determines whether multiple column families are present
|
||||
* Precondition: familyPaths is not null
|
||||
*
|
||||
* @param familyPaths List of Pair<byte[] column family, String hfilePath>
|
||||
* @param familyPaths List of (column family, hfilePath)
|
||||
*/
|
||||
private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
|
||||
boolean multipleFamilies = false;
|
||||
|
@ -5947,10 +5947,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
/**
|
||||
* This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines
|
||||
* both filterRow & filterRow(List<KeyValue> kvs) functions. While 0.94 code or older, it may
|
||||
* not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only returns
|
||||
* true when filterRow(List<KeyValue> kvs) is overridden not the filterRow(). Therefore, the
|
||||
* filterRow() will be skipped.
|
||||
* both filterRow & filterRow({@code List<KeyValue> kvs}) functions. While 0.94 code or older,
|
||||
* it may not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only
|
||||
* returns true when filterRow({@code List<KeyValue> kvs}) is overridden not the filterRow().
|
||||
* Therefore, the filterRow() will be skipped.
|
||||
*/
|
||||
private boolean filterRow() throws IOException {
|
||||
// when hasFilterRow returns true, filter.filterRow() will be called automatically inside
|
||||
|
@ -6950,9 +6950,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
}
|
||||
|
||||
/**
|
||||
* @param cell
|
||||
* @param tags
|
||||
* @return The passed-in List<Tag> but with the tags from <code>cell</code> added.
|
||||
* @return The passed-in {@code tags} but with the tags from {@code cell} added.
|
||||
*/
|
||||
private static List<Tag> carryForwardTags(final Cell cell, final List<Tag> tags) {
|
||||
if (cell.getTagsLength() <= 0) return tags;
|
||||
|
@ -6965,12 +6963,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
|
|||
|
||||
/**
|
||||
* Run a Get against passed in <code>store</code> on passed <code>row</code>, etc.
|
||||
* @param store
|
||||
* @param row
|
||||
* @param family
|
||||
* @param tr
|
||||
* @return Get result.
|
||||
* @throws IOException
|
||||
*/
|
||||
private List<Cell> doGet(final Store store, final byte [] row,
|
||||
final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
|
||||
|
|
|
@ -25,7 +25,7 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience;
|
|||
import org.apache.hadoop.hbase.client.Scan;
|
||||
|
||||
/**
|
||||
* Scanner scans the MOB Store. Coalesce KeyValue stream into List<KeyValue>
|
||||
* Scanner scans the MOB Store. Coalesce KeyValue stream into {@code List<KeyValue>}
|
||||
* for a single row. It's only used in the compaction of mob-enabled columns.
|
||||
* It outputs the normal cells and delete markers when outputDeleteMarkers is set as true.
|
||||
*/
|
||||
|
|
|
@ -28,9 +28,8 @@ import org.apache.hadoop.hbase.client.Scan;
|
|||
import org.apache.hadoop.hbase.mob.MobUtils;
|
||||
|
||||
/**
|
||||
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into List<KeyValue>
|
||||
* for a single row.
|
||||
*
|
||||
* Scanner scans both the memstore and the MOB Store. Coalesce KeyValue stream into
|
||||
* {@code List<KeyValue>} for a single row.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
public class MobStoreScanner extends StoreScanner {
|
||||
|
|
|
@ -82,14 +82,14 @@ public class ScannerContext {
|
|||
* some limits and then repeatedly invoke {@link InternalScanner#next(List)} or
|
||||
* {@link RegionScanner#next(List)} where each invocation respects these limits separately.
|
||||
* <p>
|
||||
* For example: <code><pre>
|
||||
* For example: <pre> {@code
|
||||
* ScannerContext context = new ScannerContext.newBuilder().setBatchLimit(5).build();
|
||||
* RegionScanner scanner = ...
|
||||
* List<Cell> results = new ArrayList<Cell>();
|
||||
* while(scanner.next(results, context)) {
|
||||
* // Do something with a batch of 5 cells
|
||||
* }
|
||||
* </pre></code> However, in the case of RPCs, the server wants to be able to define a set of
|
||||
* }</pre> However, in the case of RPCs, the server wants to be able to define a set of
|
||||
* limits for a particular RPC request and have those limits respected across multiple
|
||||
* invocations. This means that the progress made towards the limits in earlier calls will be
|
||||
* saved and considered in future invocations
|
||||
|
|
|
@ -361,7 +361,7 @@ public class StoreFile {
|
|||
/**
|
||||
* Check if this storefile was created by bulk load.
|
||||
* When a hfile is bulk loaded into HBase, we append
|
||||
* '_SeqId_<id-when-loaded>' to the hfile name, unless
|
||||
* {@code '_SeqId_<id-when-loaded>'} to the hfile name, unless
|
||||
* "hbase.mapreduce.bulkload.assign.sequenceNumbers" is
|
||||
* explicitly turned off.
|
||||
* If "hbase.mapreduce.bulkload.assign.sequenceNumbers"
|
||||
|
|
|
@ -68,8 +68,8 @@ public class StoreFileInfo {
|
|||
Pattern.compile("^(" + DELFILE_NAME_REGEX + ")");
|
||||
|
||||
/**
|
||||
* Regex that will work for straight reference names (<hfile>.<parentEncRegion>)
|
||||
* and hfilelink reference names (<table>=<region>-<hfile>.<parentEncRegion>)
|
||||
* Regex that will work for straight reference names ({@code <hfile>.<parentEncRegion>})
|
||||
* and hfilelink reference names ({@code <table>=<region>-<hfile>.<parentEncRegion>})
|
||||
* If reference, then the regex has more than just one group.
|
||||
* Group 1, hfile/hfilelink pattern, is this file's id.
|
||||
* Group 2 '(.+)' is the reference's parent region name.
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.hbase.executor.EventHandler;
|
|||
import org.apache.hadoop.hbase.executor.EventType;
|
||||
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
|
||||
import org.apache.hadoop.hbase.regionserver.HRegion;
|
||||
import org.apache.hadoop.hbase.regionserver.Region;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
|
||||
import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
|
||||
|
@ -218,9 +219,8 @@ public class OpenRegionHandler extends EventHandler {
|
|||
}
|
||||
|
||||
/**
|
||||
* Thread to run region post open tasks. Call {@link #getException()} after
|
||||
* the thread finishes to check for exceptions running
|
||||
* {@link RegionServerServices#postOpenDeployTasks(HRegion)
|
||||
* Thread to run region post open tasks. Call {@link #getException()} after the thread finishes
|
||||
* to check for exceptions running {@link RegionServerServices#postOpenDeployTasks(Region)}.
|
||||
*/
|
||||
static class PostOpenDeployTasksThread extends Thread {
|
||||
private Throwable exception = null;
|
||||
|
|
|
@ -450,13 +450,11 @@ public class VisibilityController extends BaseMasterAndRegionObserver implements
|
|||
* Checks whether cell contains any tag with type as VISIBILITY_TAG_TYPE. This
|
||||
* tag type is reserved and should not be explicitly set by user.
|
||||
*
|
||||
* @param cell
|
||||
* - the cell under consideration
|
||||
* @param pair - an optional pair of type <Boolean, Tag> which would be reused
|
||||
* if already set and new one will be created if null is passed
|
||||
* @return a pair<Boolean, Tag> - if the boolean is false then it indicates
|
||||
* that the cell has a RESERVERD_VIS_TAG and with boolean as true, not
|
||||
* null tag indicates that a string modified tag was found.
|
||||
* @param cell The cell under consideration
|
||||
* @param pair An optional pair of type {@code <Boolean, Tag>} which would be reused if already
|
||||
* set and new one will be created if NULL is passed
|
||||
* @return If the boolean is false then it indicates that the cell has a RESERVERD_VIS_TAG and
|
||||
* with boolean as true, not null tag indicates that a string modified tag was found.
|
||||
*/
|
||||
private Pair<Boolean, Tag> checkForReservedVisibilityTagPresence(Cell cell,
|
||||
Pair<Boolean, Tag> pair) throws IOException {
|
||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.hadoop.hbase.util.FSUtils;
|
|||
|
||||
/**
|
||||
* Thread that walks over the filesystem, and computes the mappings
|
||||
* <Region -> BestHost> and <Region -> Map<HostName, fractional-locality-of-region>>
|
||||
* Region -> BestHost and Region -> {@code Map<HostName, fractional-locality-of-region>}
|
||||
*
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
|
|
|
@ -607,8 +607,8 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
* region servers and the masters. It makes each region's state in HDFS, in
|
||||
* hbase:meta, and deployments consistent.
|
||||
*
|
||||
* @return If > 0 , number of errors detected, if < 0 there was an unrecoverable
|
||||
* error. If 0, we have a clean hbase.
|
||||
* @return If > 0 , number of errors detected, if < 0 there was an unrecoverable
|
||||
* error. If 0, we have a clean hbase.
|
||||
*/
|
||||
public int onlineConsistencyRepair() throws IOException, KeeperException,
|
||||
InterruptedException {
|
||||
|
@ -1516,10 +1516,10 @@ public class HBaseFsck extends Configured implements Closeable {
|
|||
/**
|
||||
* Sideline a region dir (instead of deleting it)
|
||||
*
|
||||
* @param parentDir if specified, the region will be sidelined to
|
||||
* folder like .../parentDir/<table name>/<region name>. The purpose
|
||||
* is to group together similar regions sidelined, for example, those
|
||||
* regions should be bulk loaded back later on. If null, it is ignored.
|
||||
* @param parentDir if specified, the region will be sidelined to folder like
|
||||
* {@literal .../parentDir/<table name>/<region name>}. The purpose is to group together
|
||||
* similar regions sidelined, for example, those regions should be bulk loaded back later
|
||||
* on. If NULL, it is ignored.
|
||||
*/
|
||||
Path sidelineRegionDir(FileSystem fs,
|
||||
String parentDir, HbckInfo hi) throws IOException {
|
||||
|
|
|
@ -119,9 +119,8 @@ public class RegionMover extends AbstractHBaseTool {
|
|||
private int port = HConstants.DEFAULT_REGIONSERVER_PORT;
|
||||
|
||||
/**
|
||||
* Hostname to unload regions from or load regions to Valid format: <hostname> or
|
||||
* <hostname:port>
|
||||
* @param hostname
|
||||
* @param hostname Hostname to unload regions from or load regions to. Can be either hostname
|
||||
* or hostname:port.
|
||||
*/
|
||||
public RegionMoverBuilder(String hostname) {
|
||||
String[] splitHostname = hostname.split(":");
|
||||
|
@ -150,8 +149,6 @@ public class RegionMover extends AbstractHBaseTool {
|
|||
|
||||
/**
|
||||
* Set the max number of threads that will be used to move regions
|
||||
* @param threads
|
||||
* @return RegionMoverBuilder object
|
||||
*/
|
||||
public RegionMoverBuilder maxthreads(int threads) {
|
||||
this.maxthreads = threads;
|
||||
|
@ -159,11 +156,9 @@ public class RegionMover extends AbstractHBaseTool {
|
|||
}
|
||||
|
||||
/**
|
||||
* Path of file containing hostnames to be excluded during region movement Exclude file should
|
||||
* have <host:port> per line.Port is mandatory here as we can have many RS running on a single
|
||||
* host
|
||||
* @param excludefile
|
||||
* @return RegionMoverBuilder object
|
||||
* Path of file containing hostnames to be excluded during region movement. Exclude file should
|
||||
* have 'host:port' per line. Port is mandatory here as we can have many RS running on a single
|
||||
* host.
|
||||
*/
|
||||
public RegionMoverBuilder excludeFile(String excludefile) {
|
||||
this.excludeFile = excludefile;
|
||||
|
@ -751,10 +746,7 @@ public class RegionMover extends AbstractHBaseTool {
|
|||
}
|
||||
|
||||
/**
|
||||
* Create an Arraylst of servers listed in exclude file
|
||||
* @param excludeFile
|
||||
* @return ArrayList of servers to be excluded in format <hostname:port>
|
||||
* @throws IOException
|
||||
* @return List of servers from the exclude file in format 'hostname:port'.
|
||||
*/
|
||||
private ArrayList<String> readExcludes(String excludeFile) throws IOException {
|
||||
ArrayList<String> excludeServers = new ArrayList<String>();
|
||||
|
|
|
@ -51,8 +51,6 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
|
||||
|
||||
// imports for things that haven't moved from regionserver.wal yet.
|
||||
import org.apache.hadoop.hbase.regionserver.wal.CompressionContext;
|
||||
import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
|
||||
|
@ -87,10 +85,10 @@ public class WALKey implements SequenceId, Comparable<WALKey> {
|
|||
/**
|
||||
* Will block until a write entry has been assigned by they WAL subsystem.
|
||||
* @return A WriteEntry gotten from local WAL subsystem. Must be completed by calling
|
||||
* mvcc#complete or mvcc#completeAndWait.
|
||||
* @throws InterruptedIOException
|
||||
* @see
|
||||
* #setWriteEntry(org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry)
|
||||
* {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
|
||||
* or
|
||||
* {@link MultiVersionConcurrencyControl#complete(MultiVersionConcurrencyControl.WriteEntry)}
|
||||
* @see {@link #setWriteEntry(MultiVersionConcurrencyControl.WriteEntry)}
|
||||
*/
|
||||
@InterfaceAudience.Private // For internal use only.
|
||||
public MultiVersionConcurrencyControl.WriteEntry getWriteEntry() throws InterruptedIOException {
|
||||
|
|
|
@ -1643,7 +1643,7 @@ public class WALSplitter {
|
|||
.synchronizedMap(new TreeMap<TableName, HConnection>());
|
||||
/**
|
||||
* Map key -> value layout
|
||||
* <servername>:<table name> -> Queue<Row>
|
||||
* {@literal <servername>:<table name> -> Queue<Row>}
|
||||
*/
|
||||
private Map<String, List<Pair<HRegionLocation, Entry>>> serverToBufferQueueMap =
|
||||
new ConcurrentHashMap<String, List<Pair<HRegionLocation, Entry>>>();
|
||||
|
@ -2173,7 +2173,7 @@ public class WALSplitter {
|
|||
}
|
||||
private TableName getTableFromLocationStr(String loc) {
|
||||
/**
|
||||
* location key is in format <server name:port>#<table name>
|
||||
* location key is in format {@literal <server name:port>#<table name>}
|
||||
*/
|
||||
String[] splits = loc.split(KEY_DELIMITER);
|
||||
if (splits.length != 2) {
|
||||
|
|
Loading…
Reference in New Issue