@@ -388,8 +393,7 @@
lifecycle-mapping
-
-
+
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java
index b137a7da2ceb..b0a5a86d50bb 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/Abortable.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -23,8 +22,8 @@
/**
* Interface to support the aborting of a given server or client.
*
- * This is used primarily for ZooKeeper usage when we could get an unexpected
- * and fatal exception, requiring an abort.
+ * This is used primarily for ZooKeeper usage when we could get an unexpected and fatal exception,
+ * requiring an abort.
*
* Implemented by the Master, RegionServer, and TableServers (client).
*/
@@ -33,13 +32,12 @@ public interface Abortable {
/**
* Abort the server or client.
* @param why Why we're aborting.
- * @param e Throwable that caused abort. Can be null.
+ * @param e Throwable that caused abort. Can be null.
*/
void abort(String why, Throwable e);
/**
- * It just call another abort method and the Throwable
- * parameter is null.
+ * It just calls another abort method and the Throwable parameter is null.
* @param why Why we're aborting.
* @see Abortable#abort(String, Throwable)
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
index b1fcd945b7d6..813c060b5200 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/AsyncMetaTableAccessor.java
@@ -1,4 +1,4 @@
-/**
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -17,6 +17,7 @@
*/
package org.apache.hadoop.hbase;
+import static org.apache.hadoop.hbase.client.RegionLocator.LOCATOR_META_REPLICAS_MODE;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.io.IOException;
@@ -29,6 +30,7 @@
import java.util.Optional;
import java.util.SortedMap;
import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
@@ -63,21 +65,20 @@ public class AsyncMetaTableAccessor {
private static final Logger LOG = LoggerFactory.getLogger(AsyncMetaTableAccessor.class);
-
/** The delimiter for meta columns for replicaIds > 0 */
private static final char META_REPLICA_ID_DELIMITER = '_';
/** A regex for parsing server columns from meta. See above javadoc for meta layout */
- private static final Pattern SERVER_COLUMN_PATTERN = Pattern
- .compile("^server(_[0-9a-fA-F]{4})?$");
+ private static final Pattern SERVER_COLUMN_PATTERN =
+ Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
public static CompletableFuture tableExists(AsyncTable> metaTable,
- TableName tableName) {
+ TableName tableName) {
return getTableState(metaTable, tableName).thenApply(Optional::isPresent);
}
public static CompletableFuture> getTableState(AsyncTable> metaTable,
- TableName tableName) {
+ TableName tableName) {
CompletableFuture> future = new CompletableFuture<>();
Get get = new Get(tableName.getName()).addColumn(getTableFamily(), getStateColumn());
long time = EnvironmentEdgeManager.currentTime();
@@ -100,14 +101,9 @@ public static CompletableFuture> getTableState(AsyncTable
return future;
}
- /**
- * Returns the HRegionLocation from meta for the given region
- * @param metaTable
- * @param regionName region we're looking for
- * @return HRegionLocation for the given region
- */
- public static CompletableFuture> getRegionLocation(
- AsyncTable> metaTable, byte[] regionName) {
+ /** Returns the HRegionLocation from meta for the given region */
+ public static CompletableFuture>
+ getRegionLocation(AsyncTable> metaTable, byte[] regionName) {
CompletableFuture> future = new CompletableFuture<>();
try {
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
@@ -127,14 +123,9 @@ public static CompletableFuture> getRegionLocation(
return future;
}
- /**
- * Returns the HRegionLocation from meta for the given encoded region name
- * @param metaTable
- * @param encodedRegionName region we're looking for
- * @return HRegionLocation for the given region
- */
- public static CompletableFuture> getRegionLocationWithEncodedName(
- AsyncTable> metaTable, byte[] encodedRegionName) {
+ /** Returns the HRegionLocation from meta for the given encoded region name */
+ public static CompletableFuture>
+ getRegionLocationWithEncodedName(AsyncTable> metaTable, byte[] encodedRegionName) {
CompletableFuture> future = new CompletableFuture<>();
addListener(
metaTable
@@ -149,8 +140,10 @@ public static CompletableFuture> getRegionLocationWith
.filter(result -> MetaTableAccessor.getRegionInfo(result) != null).forEach(result -> {
getRegionLocations(result).ifPresent(locations -> {
for (HRegionLocation location : locations.getRegionLocations()) {
- if (location != null &&
- encodedRegionNameStr.equals(location.getRegion().getEncodedName())) {
+ if (
+ location != null
+ && encodedRegionNameStr.equals(location.getRegion().getEncodedName())
+ ) {
future.complete(Optional.of(location));
return;
}
@@ -166,24 +159,23 @@ private static Optional getTableState(Result r) throws IOException {
Cell cell = r.getColumnLatestCell(getTableFamily(), getStateColumn());
if (cell == null) return Optional.empty();
try {
- return Optional.of(TableState.parseFrom(
- TableName.valueOf(r.getRow()),
- Arrays.copyOfRange(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset()
- + cell.getValueLength())));
+ return Optional.of(
+ TableState.parseFrom(TableName.valueOf(r.getRow()), Arrays.copyOfRange(cell.getValueArray(),
+ cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength())));
} catch (DeserializationException e) {
throw new IOException("Failed to parse table state from result: " + r, e);
}
}
/**
- * Used to get all region locations for the specific table.
- * @param metaTable
+ * Used to get all region locations for the specific table
+ * @param metaTable scanner over meta table
* @param tableName table we're looking for, can be null for getting all regions
* @return the list of region locations. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
public static CompletableFuture> getTableHRegionLocations(
- AsyncTable metaTable, TableName tableName) {
+ AsyncTable metaTable, TableName tableName) {
CompletableFuture> future = new CompletableFuture<>();
addListener(getTableRegionsAndLocations(metaTable, tableName, true), (locations, err) -> {
if (err != null) {
@@ -202,53 +194,53 @@ public static CompletableFuture> getTableHRegionLocations(
/**
* Used to get table regions' info and server.
- * @param metaTable
- * @param tableName table we're looking for, can be null for getting all regions
+ * @param metaTable scanner over meta table
+ * @param tableName table we're looking for, can be null for getting all regions
* @param excludeOfflinedSplitParents don't return split parents
* @return the list of regioninfos and server. The return value will be wrapped by a
* {@link CompletableFuture}.
*/
private static CompletableFuture>> getTableRegionsAndLocations(
- final AsyncTable metaTable,
- final TableName tableName, final boolean excludeOfflinedSplitParents) {
+ final AsyncTable metaTable, final TableName tableName,
+ final boolean excludeOfflinedSplitParents) {
CompletableFuture>> future = new CompletableFuture<>();
if (TableName.META_TABLE_NAME.equals(tableName)) {
future.completeExceptionally(new IOException(
- "This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
+ "This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
}
// Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
CollectingVisitor> visitor =
new CollectingVisitor>() {
- private RegionLocations current = null;
-
- @Override
- public boolean visit(Result r) throws IOException {
- Optional currentRegionLocations = getRegionLocations(r);
- current = currentRegionLocations.orElse(null);
- if (current == null || current.getRegionLocation().getRegion() == null) {
- LOG.warn("No serialized RegionInfo in " + r);
- return true;
+ private RegionLocations current = null;
+
+ @Override
+ public boolean visit(Result r) throws IOException {
+ Optional currentRegionLocations = getRegionLocations(r);
+ current = currentRegionLocations.orElse(null);
+ if (current == null || current.getRegionLocation().getRegion() == null) {
+ LOG.warn("No serialized RegionInfo in " + r);
+ return true;
+ }
+ RegionInfo hri = current.getRegionLocation().getRegion();
+ if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
+ // Else call super and add this Result to the collection.
+ return super.visit(r);
}
- RegionInfo hri = current.getRegionLocation().getRegion();
- if (excludeOfflinedSplitParents && hri.isSplitParent()) return true;
- // Else call super and add this Result to the collection.
- return super.visit(r);
- }
- @Override
- void add(Result r) {
- if (current == null) {
- return;
- }
- for (HRegionLocation loc : current.getRegionLocations()) {
- if (loc != null) {
- this.results.add(new Pair(loc.getRegion(), loc
- .getServerName()));
+ @Override
+ void add(Result r) {
+ if (current == null) {
+ return;
+ }
+ for (HRegionLocation loc : current.getRegionLocations()) {
+ if (loc != null) {
+ this.results
+ .add(new Pair(loc.getRegion(), loc.getServerName()));
+ }
}
}
- }
- };
+ };
addListener(scanMeta(metaTable, tableName, QueryType.REGION, visitor), (v, error) -> {
if (error != null) {
@@ -262,28 +254,28 @@ void add(Result r) {
/**
* Performs a scan of META table for given table.
- * @param metaTable
- * @param tableName table withing we scan
- * @param type scanned part of meta
- * @param visitor Visitor invoked against each row
+ * @param metaTable scanner over meta table
+ * @param tableName table within we scan
+ * @param type scanned part of meta
+ * @param visitor Visitor invoked against each row
*/
private static CompletableFuture scanMeta(AsyncTable metaTable,
- TableName tableName, QueryType type, final Visitor visitor) {
+ TableName tableName, QueryType type, final Visitor visitor) {
return scanMeta(metaTable, getTableStartRowForMeta(tableName, type),
getTableStopRowForMeta(tableName, type), type, Integer.MAX_VALUE, visitor);
}
/**
* Performs a scan of META table for given table.
- * @param metaTable
- * @param startRow Where to start the scan
- * @param stopRow Where to stop the scan
- * @param type scanned part of meta
- * @param maxRows maximum rows to return
- * @param visitor Visitor invoked against each row
+ * @param metaTable scanner over meta table
+ * @param startRow Where to start the scan
+ * @param stopRow Where to stop the scan
+ * @param type scanned part of meta
+ * @param maxRows maximum rows to return
+ * @param visitor Visitor invoked against each row
*/
private static CompletableFuture scanMeta(AsyncTable metaTable,
- byte[] startRow, byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) {
+ byte[] startRow, byte[] stopRow, QueryType type, int maxRows, final Visitor visitor) {
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
Scan scan = getMetaScan(metaTable, rowUpperLimit);
for (byte[] family : type.getFamilies()) {
@@ -298,12 +290,42 @@ private static CompletableFuture scanMeta(AsyncTable 1) {
+ int replicaId = ThreadLocalRandom.current().nextInt(numOfReplicas);
+
+ // When the replicaId is 0, do not set to Consistency.TIMELINE
+ if (replicaId > 0) {
+ scan.setReplicaId(replicaId);
+ scan.setConsistency(Consistency.TIMELINE);
+ }
+ }
+ metaTable.scan(scan, new MetaTableScanResultConsumer(rowUpperLimit, visitor, future));
+ });
+ } else {
+ if (metaReplicaMode == CatalogReplicaMode.HEDGED_READ) {
+ scan.setConsistency(Consistency.TIMELINE);
+ }
+ metaTable.scan(scan, new MetaTableScanResultConsumer(rowUpperLimit, visitor, future));
+ }
+
return future;
}
@@ -318,7 +340,7 @@ private static final class MetaTableScanResultConsumer implements AdvancedScanRe
private final CompletableFuture future;
MetaTableScanResultConsumer(int rowUpperLimit, Visitor visitor,
- CompletableFuture future) {
+ CompletableFuture future) {
this.rowUpperLimit = rowUpperLimit;
this.visitor = visitor;
this.future = future;
@@ -332,7 +354,7 @@ public void onError(Throwable error) {
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NP_NONNULL_PARAM_VIOLATION",
- justification = "https://github.com/findbugsproject/findbugs/issues/79")
+ justification = "https://github.com/findbugsproject/findbugs/issues/79")
public void onComplete() {
future.complete(null);
}
@@ -366,8 +388,10 @@ private static Scan getMetaScan(AsyncTable> metaTable, int rowUpperLimit) {
Scan scan = new Scan();
int scannerCaching = metaTable.getConfiguration().getInt(HConstants.HBASE_META_SCANNER_CACHING,
HConstants.DEFAULT_HBASE_META_SCANNER_CACHING);
- if (metaTable.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
- HConstants.DEFAULT_USE_META_REPLICAS)) {
+ if (
+ metaTable.getConfiguration().getBoolean(HConstants.USE_META_REPLICAS,
+ HConstants.DEFAULT_USE_META_REPLICAS)
+ ) {
scan.setConsistency(Consistency.TIMELINE);
}
if (rowUpperLimit <= scannerCaching) {
@@ -384,9 +408,13 @@ private static Scan getMetaScan(AsyncTable> metaTable, int rowUpperLimit) {
* can't deserialize the result.
*/
private static Optional getRegionLocations(final Result r) {
- if (r == null) return Optional.empty();
+ if (r == null) {
+ return Optional.empty();
+ }
Optional regionInfo = getHRegionInfo(r, getRegionInfoColumn());
- if (!regionInfo.isPresent()) return Optional.empty();
+ if (!regionInfo.isPresent()) {
+ return Optional.empty();
+ }
List locations = new ArrayList(1);
NavigableMap> familyMap = r.getNoVersionMap();
@@ -394,15 +422,18 @@ private static Optional getRegionLocations(final Result r) {
locations.add(getRegionLocation(r, regionInfo.get(), 0));
NavigableMap infoMap = familyMap.get(getCatalogFamily());
- if (infoMap == null) return Optional.of(new RegionLocations(locations));
+ if (infoMap == null) {
+ return Optional.of(new RegionLocations(locations));
+ }
// iterate until all serverName columns are seen
int replicaId = 0;
byte[] serverColumn = getServerColumn(replicaId);
- SortedMap serverMap = null;
- serverMap = infoMap.tailMap(serverColumn, false);
+ SortedMap serverMap = infoMap.tailMap(serverColumn, false);
- if (serverMap.isEmpty()) return Optional.of(new RegionLocations(locations));
+ if (serverMap.isEmpty()) {
+ return Optional.of(new RegionLocations(locations));
+ }
for (Map.Entry entry : serverMap.entrySet()) {
replicaId = parseReplicaIdFromServerColumn(entry.getKey());
@@ -423,16 +454,15 @@ private static Optional getRegionLocations(final Result r) {
}
/**
- * Returns the HRegionLocation parsed from the given meta row Result
- * for the given regionInfo and replicaId. The regionInfo can be the default region info
- * for the replica.
- * @param r the meta row result
+ * Returns the HRegionLocation parsed from the given meta row Result for the given regionInfo and
+ * replicaId. The regionInfo can be the default region info for the replica.
+ * @param r the meta row result
* @param regionInfo RegionInfo for default replica
- * @param replicaId the replicaId for the HRegionLocation
+ * @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
- final int replicaId) {
+ final int replicaId) {
Optional serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
@@ -448,8 +478,8 @@ private static Optional getServerName(final Result r, final int repl
byte[] serverColumn = getServerColumn(replicaId);
Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn);
if (cell == null || cell.getValueLength() == 0) return Optional.empty();
- String hostAndPort = Bytes.toString(cell.getValueArray(), cell.getValueOffset(),
- cell.getValueLength());
+ String hostAndPort =
+ Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
byte[] startcodeColumn = getStartCodeColumn(replicaId);
cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn);
if (cell == null || cell.getValueLength() == 0) return Optional.empty();
@@ -463,8 +493,8 @@ private static Optional getServerName(final Result r, final int repl
}
/**
- * The latest seqnum that the server writing to meta observed when opening the region.
- * E.g. the seqNum when the result of {@link #getServerName(Result, int)} was written.
+ * The latest seqnum that the server writing to meta observed when opening the region. E.g. the
+ * seqNum when the result of {@link #getServerName(Result, int)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
*/
@@ -533,7 +563,7 @@ private static byte[] getTableStopRowForMeta(TableName tableName, QueryType type
/**
* Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* qualifier of the catalog table result.
- * @param r a Result object from the catalog table scan
+ * @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
* @return An RegionInfo instance.
*/
@@ -585,7 +615,7 @@ private static byte[] getServerColumn(int replicaId) {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -597,7 +627,7 @@ private static byte[] getStartCodeColumn(int replicaId) {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -609,12 +639,12 @@ private static byte[] getSeqNumColumn(int replicaId) {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
- * Parses the replicaId from the server column qualifier. See top of the class javadoc
- * for the actual meta layout
+ * Parses the replicaId from the server column qualifier. See top of the class javadoc for the
+ * actual meta layout
* @param serverColumn the column qualifier
* @return an int for the replicaId
*/
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStats.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStats.java
index 91cedd60299d..615b3a467e6e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStats.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStats.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -21,7 +20,6 @@
import java.util.Collections;
import java.util.Map;
import java.util.stream.Collectors;
-
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.yetus.audience.InterfaceAudience;
@@ -56,9 +54,8 @@ public int getExceptionCount() {
private String getFailedRegions() {
return exceptions.keySet().stream()
- .map(regionName -> RegionInfo.prettyPrint(RegionInfo.encodeRegionName(regionName)))
- .collect(Collectors.toList())
- .toString();
+ .map(regionName -> RegionInfo.prettyPrint(RegionInfo.encodeRegionName(regionName)))
+ .collect(Collectors.toList()).toString();
}
@InterfaceAudience.Private
@@ -68,11 +65,8 @@ public static CacheEvictionStatsBuilder builder() {
@Override
public String toString() {
- return "CacheEvictionStats{" +
- "evictedBlocks=" + evictedBlocks +
- ", maxCacheSize=" + maxCacheSize +
- ", failedRegionsSize=" + getExceptionCount() +
- ", failedRegions=" + getFailedRegions() +
- '}';
+ return "CacheEvictionStats{" + "evictedBlocks=" + evictedBlocks + ", maxCacheSize="
+ + maxCacheSize + ", failedRegionsSize=" + getExceptionCount() + ", failedRegions="
+ + getFailedRegions() + '}';
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsAggregator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsAggregator.java
index 85d68dcc08bc..fabe7f030278 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsAggregator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsAggregator.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -39,4 +38,4 @@ public synchronized void append(CacheEvictionStats stats) {
public synchronized CacheEvictionStats sum() {
return this.builder.build();
}
-}
\ No newline at end of file
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsBuilder.java
index d9e1400da16b..4b31d98611bc 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CacheEvictionStatsBuilder.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -20,7 +19,6 @@
import java.util.HashMap;
import java.util.Map;
-
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
@@ -42,7 +40,7 @@ public CacheEvictionStatsBuilder withMaxCacheSize(long maxCacheSize) {
return this;
}
- public void addException(byte[] regionName, Throwable ie){
+ public void addException(byte[] regionName, Throwable ie) {
exceptions.put(regionName, ie);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
index 13ab3ed47cee..8bfde779176e 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallDroppedException.java
@@ -1,4 +1,4 @@
-/**
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -15,27 +15,26 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
-import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
/**
- * Returned to the clients when their request was discarded due to server being overloaded.
- * Clients should retry upon receiving it.
+ * Returned to the clients when their request was discarded due to server being overloaded. Clients
+ * should retry upon receiving it.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
-public class CallDroppedException extends IOException {
+public class CallDroppedException extends HBaseServerException {
public CallDroppedException() {
- super();
+ // For now all call drops are due to server being overloaded.
+ // We could decouple this if desired.
+ super(true);
}
// Absence of this constructor prevents proper unwrapping of
// remote exception on the client side
public CallDroppedException(String message) {
- super(message);
+ super(true, message);
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
index 12fa242693c8..ecad4d9f0bc2 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CallQueueTooBigException.java
@@ -1,4 +1,4 @@
-/**
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -15,16 +15,17 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
-import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
+/**
+ * Returned to clients when their request was dropped because the call queue was too big to accept a
+ * new call. Clients should retry upon receiving it.
+ */
@SuppressWarnings("serial")
@InterfaceAudience.Public
-public class CallQueueTooBigException extends IOException {
+public class CallQueueTooBigException extends CallDroppedException {
public CallQueueTooBigException() {
super();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaMode.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogReplicaMode.java
similarity index 61%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaMode.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogReplicaMode.java
index 40062e32e83c..b89673d45a88 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/CatalogReplicaMode.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CatalogReplicaMode.java
@@ -6,34 +6,34 @@
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hbase.client;
+package org.apache.hadoop.hbase;
import org.apache.yetus.audience.InterfaceAudience;
/**
- * There are two modes with catalog replica support.
- *
+ *
+ * There are two modes with catalog replica support.
+ *
*
- * - HEDGED_READ - Client sends requests to the primary region first, within a
- * configured amount of time, if there is no response coming back,
- * client sends requests to all replica regions and takes the first
- * response.
- *
- * - LOAD_BALANCE - Client sends requests to replica regions in a round-robin mode,
- * if results from replica regions are stale, next time, client sends requests for
- * these stale locations to the primary region. In this mode, scan
- * requests are load balanced across all replica regions.
+ * - HEDGED_READ - Client sends requests to the primary region first, within a configured amount
+ * of time, if there is no response coming back, client sends requests to all replica regions and
+ * takes the first response.
+ * - LOAD_BALANCE - Client sends requests to replica regions in a round-robin mode, if results
+ * from replica regions are stale, next time, client sends requests for these stale locations to the
+ * primary region. In this mode, scan requests are load balanced across all replica regions.
*
*/
@InterfaceAudience.Private
-enum CatalogReplicaMode {
+public enum CatalogReplicaMode {
NONE {
@Override
public String toString() {
@@ -54,7 +54,7 @@ public String toString() {
};
public static CatalogReplicaMode fromString(final String value) {
- for(CatalogReplicaMode mode : values()) {
+ for (CatalogReplicaMode mode : values()) {
if (mode.toString().equalsIgnoreCase(value)) {
return mode;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
index a63ca6936ec1..1afcb30ece01 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClockOutOfSyncException.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -19,12 +18,10 @@
package org.apache.hadoop.hbase;
import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
/**
- * This exception is thrown by the master when a region server clock skew is
- * too high.
+ * This exception is thrown by the master when a region server clock skew is too high.
*/
@SuppressWarnings("serial")
@InterfaceAudience.Public
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
index 1dd01faf808a..67438677dadd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterId.java
@@ -15,29 +15,27 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
import java.io.IOException;
import java.util.UUID;
-
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.yetus.audience.InterfaceAudience;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterIdProtos;
-import org.apache.hadoop.hbase.util.Bytes;
/**
- * The identifier for this cluster.
- * It is serialized to the filesystem and up into zookeeper. This is a container for the id.
- * Also knows how to serialize and deserialize the cluster id.
+ * The identifier for this cluster. It is serialized to the filesystem and up into zookeeper. This
+ * is a container for the id. Also knows how to serialize and deserialize the cluster id.
*/
@InterfaceAudience.Private
public class ClusterId {
private final String id;
/**
- * New ClusterID. Generates a uniqueid.
+ * New ClusterID. Generates a uniqueid.
*/
public ClusterId() {
this(UUID.randomUUID().toString());
@@ -47,20 +45,18 @@ public ClusterId(final String uuid) {
this.id = uuid;
}
- /**
- * @return The clusterid serialized using pb w/ pb magic prefix
- */
- public byte [] toByteArray() {
+ /** Returns The clusterid serialized using pb w/ pb magic prefix */
+ public byte[] toByteArray() {
return ProtobufUtil.prependPBMagic(convert().toByteArray());
}
/**
+ * Parse the serialized representation of the {@link ClusterId}
* @param bytes A pb serialized {@link ClusterId} instance with pb magic prefix
* @return An instance of {@link ClusterId} made from bytes
- * @throws DeserializationException
* @see #toByteArray()
*/
- public static ClusterId parseFrom(final byte [] bytes) throws DeserializationException {
+ public static ClusterId parseFrom(final byte[] bytes) throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(bytes)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
ClusterIdProtos.ClusterId.Builder builder = ClusterIdProtos.ClusterId.newBuilder();
@@ -78,18 +74,13 @@ public static ClusterId parseFrom(final byte [] bytes) throws DeserializationExc
}
}
- /**
- * @return A pb instance to represent this instance.
- */
+ /** Returns A pb instance to represent this instance. */
public ClusterIdProtos.ClusterId convert() {
ClusterIdProtos.ClusterId.Builder builder = ClusterIdProtos.ClusterId.newBuilder();
return builder.setClusterId(this.id).build();
}
- /**
- * @param cid
- * @return A {@link ClusterId} made from the passed in cid
- */
+ /** Returns A {@link ClusterId} made from the passed in cid */
public static ClusterId convert(final ClusterIdProtos.ClusterId cid) {
return new ClusterId(cid.getClusterId());
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
index 497ab938856b..a8a1493c349a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetrics.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -16,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
import edu.umd.cs.findbugs.annotations.Nullable;
@@ -39,65 +37,62 @@
* The average cluster load.
* The number of regions deployed on the cluster.
* The number of requests since last report.
- * Detailed region server loading and resource usage information,
- * per server and per region.
+ * Detailed region server loading and resource usage information, per server and per
+ * region.
* Regions in transition at master
* The unique cluster ID
*
- * {@link Option} provides a way to get desired ClusterStatus information.
- * The following codes will get all the cluster information.
+ * {@link Option} provides a way to get desired ClusterStatus information. The following
+ * codes will get all the cluster information.
+ *
*
- * {@code
- * // Original version still works
- * Admin admin = connection.getAdmin();
- * ClusterMetrics metrics = admin.getClusterStatus();
- * // or below, a new version which has the same effects
- * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.allOf(Option.class));
+ * {
+ * @code
+ * // Original version still works
+ * Admin admin = connection.getAdmin();
+ * ClusterMetrics metrics = admin.getClusterStatus();
+ * // or below, a new version which has the same effects
+ * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.allOf(Option.class));
* }
*
- * If information about live servers is the only wanted.
- * then codes in the following way:
+ *
+ * If information about live servers is the only wanted. then codes in the following way:
+ *
*
- * {@code
- * Admin admin = connection.getAdmin();
- * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+ * {
+ * @code
+ * Admin admin = connection.getAdmin();
+ * ClusterMetrics metrics = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
* }
*
*/
@InterfaceAudience.Public
public interface ClusterMetrics {
- /**
- * @return the HBase version string as reported by the HMaster
- */
+ /** Returns the HBase version string as reported by the HMaster */
@Nullable
String getHBaseVersion();
- /**
- * @return the names of region servers on the dead list
- */
+ /** Returns the names of region servers on the dead list */
List getDeadServerNames();
- /**
- * @return the names of region servers on the live list
- */
+ /** Returns the names of region servers on the unknown list */
+ List getUnknownServerNames();
+
+ /** Returns the names of region servers on the live list */
Map getLiveServerMetrics();
- /**
- * @return the number of regions deployed on the cluster
- */
+ /** Returns the number of regions deployed on the cluster */
default int getRegionCount() {
return getLiveServerMetrics().entrySet().stream()
- .mapToInt(v -> v.getValue().getRegionMetrics().size()).sum();
+ .mapToInt(v -> v.getValue().getRegionMetrics().size()).sum();
}
- /**
- * @return the number of requests since last report
- */
+ /** Returns the number of requests since last report */
default long getRequestCount() {
return getLiveServerMetrics().entrySet().stream()
- .flatMap(v -> v.getValue().getRegionMetrics().values().stream())
- .mapToLong(RegionMetrics::getRequestCount).sum();
+ .flatMap(v -> v.getValue().getRegionMetrics().values().stream())
+ .mapToLong(RegionMetrics::getRequestCount).sum();
}
/**
@@ -107,9 +102,7 @@ default long getRequestCount() {
@Nullable
ServerName getMasterName();
- /**
- * @return the names of backup masters
- */
+ /** Returns the names of backup masters */
List getBackupMasterNames();
@InterfaceAudience.Private
@@ -122,17 +115,15 @@ default long getRequestCount() {
default long getLastMajorCompactionTimestamp(TableName table) {
return getLiveServerMetrics().values().stream()
- .flatMap(s -> s.getRegionMetrics().values().stream())
- .filter(r -> RegionInfo.getTable(r.getRegionName()).equals(table))
- .mapToLong(RegionMetrics::getLastMajorCompactionTimestamp).min().orElse(0);
+ .flatMap(s -> s.getRegionMetrics().values().stream())
+ .filter(r -> RegionInfo.getTable(r.getRegionName()).equals(table))
+ .mapToLong(RegionMetrics::getLastMajorCompactionTimestamp).min().orElse(0);
}
default long getLastMajorCompactionTimestamp(byte[] regionName) {
return getLiveServerMetrics().values().stream()
- .filter(s -> s.getRegionMetrics().containsKey(regionName))
- .findAny()
- .map(s -> s.getRegionMetrics().get(regionName).getLastMajorCompactionTimestamp())
- .orElse(0L);
+ .filter(s -> s.getRegionMetrics().containsKey(regionName)).findAny()
+ .map(s -> s.getRegionMetrics().get(regionName).getLastMajorCompactionTimestamp()).orElse(0L);
}
@Nullable
@@ -142,25 +133,28 @@ default long getLastMajorCompactionTimestamp(byte[] regionName) {
List getServersName();
- /**
- * @return the average cluster load
- */
+ /** Returns the average cluster load */
default double getAverageLoad() {
int serverSize = getLiveServerMetrics().size();
if (serverSize == 0) {
return 0;
}
- return (double)getRegionCount() / (double)serverSize;
+ return (double) getRegionCount() / (double) serverSize;
}
/**
- * Provide region states count for given table.
- * e.g howmany regions of give table are opened/closed/rit etc
- *
+ * Provide region states count for given table. e.g howmany regions of give table are
+ * opened/closed/rit etc
* @return map of table to region states count
*/
Map getTableRegionStatesCount();
+ /**
+ * Provide the list of master tasks
+ */
+ @Nullable
+ List getMasterTasks();
+
/**
* Kinds of ClusterMetrics
*/
@@ -185,6 +179,10 @@ enum Option {
* metrics about dead region servers
*/
DEAD_SERVERS,
+ /**
+ * metrics about unknown region servers
+ */
+ UNKNOWN_SERVERS,
/**
* metrics about master name
*/
@@ -213,5 +211,9 @@ enum Option {
* metrics about table to no of regions status count
*/
TABLE_TO_REGIONS_COUNT,
+ /**
+ * metrics about monitored tasks
+ */
+ TASKS,
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
index 493fe71b8b0f..9ca65463e022 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterMetricsBuilder.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -16,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
import edu.umd.cs.findbugs.annotations.Nullable;
@@ -26,13 +24,13 @@
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
-
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.Option;
@@ -43,45 +41,43 @@
public final class ClusterMetricsBuilder {
public static ClusterStatusProtos.ClusterStatus toClusterStatus(ClusterMetrics metrics) {
- ClusterStatusProtos.ClusterStatus.Builder builder
- = ClusterStatusProtos.ClusterStatus.newBuilder()
- .addAllBackupMasters(metrics.getBackupMasterNames().stream()
- .map(ProtobufUtil::toServerName).collect(Collectors.toList()))
- .addAllDeadServers(metrics.getDeadServerNames().stream()
- .map(ProtobufUtil::toServerName).collect(Collectors.toList()))
+ ClusterStatusProtos.ClusterStatus.Builder builder =
+ ClusterStatusProtos.ClusterStatus.newBuilder()
+ .addAllBackupMasters(metrics.getBackupMasterNames().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .addAllDeadServers(metrics.getDeadServerNames().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .addAllUnknownServers(metrics.getUnknownServerNames().stream()
+ .map(ProtobufUtil::toServerName).collect(Collectors.toList()))
.addAllLiveServers(metrics.getLiveServerMetrics().entrySet().stream()
- .map(s -> ClusterStatusProtos.LiveServerInfo
- .newBuilder()
- .setServer(ProtobufUtil.toServerName(s.getKey()))
- .setServerLoad(ServerMetricsBuilder.toServerLoad(s.getValue()))
- .build())
- .collect(Collectors.toList()))
+ .map(s -> ClusterStatusProtos.LiveServerInfo.newBuilder()
+ .setServer(ProtobufUtil.toServerName(s.getKey()))
+ .setServerLoad(ServerMetricsBuilder.toServerLoad(s.getValue())).build())
+ .collect(Collectors.toList()))
.addAllMasterCoprocessors(metrics.getMasterCoprocessorNames().stream()
- .map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
- .collect(Collectors.toList()))
+ .map(n -> HBaseProtos.Coprocessor.newBuilder().setName(n).build())
+ .collect(Collectors.toList()))
.addAllRegionsInTransition(metrics.getRegionStatesInTransition().stream()
- .map(r -> ClusterStatusProtos.RegionInTransition
- .newBuilder()
- .setSpec(HBaseProtos.RegionSpecifier
- .newBuilder()
- .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
- .setValue(UnsafeByteOperations.unsafeWrap(r.getRegion().getRegionName()))
- .build())
- .setRegionState(r.convert())
- .build())
- .collect(Collectors.toList()))
+ .map(r -> ClusterStatusProtos.RegionInTransition.newBuilder()
+ .setSpec(HBaseProtos.RegionSpecifier.newBuilder()
+ .setType(HBaseProtos.RegionSpecifier.RegionSpecifierType.REGION_NAME)
+ .setValue(UnsafeByteOperations.unsafeWrap(r.getRegion().getRegionName())).build())
+ .setRegionState(r.convert()).build())
+ .collect(Collectors.toList()))
.setMasterInfoPort(metrics.getMasterInfoPort())
.addAllServersName(metrics.getServersName().stream().map(ProtobufUtil::toServerName)
.collect(Collectors.toList()))
.addAllTableRegionStatesCount(metrics.getTableRegionStatesCount().entrySet().stream()
- .map(status ->
- ClusterStatusProtos.TableRegionStatesCount.newBuilder()
- .setTableName(ProtobufUtil.toProtoTableName((status.getKey())))
- .setRegionStatesCount(ProtobufUtil.toTableRegionStatesCount(status.getValue()))
- .build())
+ .map(status -> ClusterStatusProtos.TableRegionStatesCount.newBuilder()
+ .setTableName(ProtobufUtil.toProtoTableName(status.getKey()))
+ .setRegionStatesCount(ProtobufUtil.toTableRegionStatesCount(status.getValue())).build())
.collect(Collectors.toList()));
if (metrics.getMasterName() != null) {
- builder.setMaster(ProtobufUtil.toServerName((metrics.getMasterName())));
+ builder.setMaster(ProtobufUtil.toServerName(metrics.getMasterName()));
+ }
+ if (metrics.getMasterTasks() != null) {
+ builder.addAllMasterTasks(metrics.getMasterTasks().stream()
+ .map(t -> ProtobufUtil.toServerTask(t)).collect(Collectors.toList()));
}
if (metrics.getBalancerOn() != null) {
builder.setBalancerOn(metrics.getBalancerOn());
@@ -91,38 +87,35 @@ public static ClusterStatusProtos.ClusterStatus toClusterStatus(ClusterMetrics m
}
if (metrics.getHBaseVersion() != null) {
builder.setHbaseVersion(
- FSProtos.HBaseVersionFileContent.newBuilder()
- .setVersion(metrics.getHBaseVersion()));
+ FSProtos.HBaseVersionFileContent.newBuilder().setVersion(metrics.getHBaseVersion()));
}
return builder.build();
}
- public static ClusterMetrics toClusterMetrics(
- ClusterStatusProtos.ClusterStatus proto) {
+ public static ClusterMetrics toClusterMetrics(ClusterStatusProtos.ClusterStatus proto) {
ClusterMetricsBuilder builder = ClusterMetricsBuilder.newBuilder();
- builder.setLiveServerMetrics(proto.getLiveServersList().stream()
+ builder
+ .setLiveServerMetrics(proto.getLiveServersList().stream()
.collect(Collectors.toMap(e -> ProtobufUtil.toServerName(e.getServer()),
- ServerMetricsBuilder::toServerMetrics)))
- .setDeadServerNames(proto.getDeadServersList().stream()
- .map(ProtobufUtil::toServerName)
- .collect(Collectors.toList()))
- .setBackerMasterNames(proto.getBackupMastersList().stream()
- .map(ProtobufUtil::toServerName)
- .collect(Collectors.toList()))
- .setRegionsInTransition(proto.getRegionsInTransitionList().stream()
- .map(ClusterStatusProtos.RegionInTransition::getRegionState)
- .map(RegionState::convert)
- .collect(Collectors.toList()))
- .setMasterCoprocessorNames(proto.getMasterCoprocessorsList().stream()
- .map(HBaseProtos.Coprocessor::getName)
- .collect(Collectors.toList()))
- .setServerNames(proto.getServersNameList().stream().map(ProtobufUtil::toServerName)
- .collect(Collectors.toList()))
- .setTableRegionStatesCount(
- proto.getTableRegionStatesCountList().stream()
- .collect(Collectors.toMap(
- e -> ProtobufUtil.toTableName(e.getTableName()),
- e -> ProtobufUtil.toTableRegionStatesCount(e.getRegionStatesCount()))));
+ ServerMetricsBuilder::toServerMetrics)))
+ .setDeadServerNames(proto.getDeadServersList().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .setUnknownServerNames(proto.getUnknownServersList().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .setBackerMasterNames(proto.getBackupMastersList().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .setRegionsInTransition(proto.getRegionsInTransitionList().stream()
+ .map(ClusterStatusProtos.RegionInTransition::getRegionState).map(RegionState::convert)
+ .collect(Collectors.toList()))
+ .setMasterCoprocessorNames(proto.getMasterCoprocessorsList().stream()
+ .map(HBaseProtos.Coprocessor::getName).collect(Collectors.toList()))
+ .setServerNames(proto.getServersNameList().stream().map(ProtobufUtil::toServerName)
+ .collect(Collectors.toList()))
+ .setTableRegionStatesCount(proto.getTableRegionStatesCountList().stream()
+ .collect(Collectors.toMap(e -> ProtobufUtil.toTableName(e.getTableName()),
+ e -> ProtobufUtil.toTableRegionStatesCount(e.getRegionStatesCount()))))
+ .setMasterTasks(proto.getMasterTasksList().stream().map(t -> ProtobufUtil.getServerTask(t))
+ .collect(Collectors.toList()));
if (proto.hasClusterId()) {
builder.setClusterId(ClusterId.convert(proto.getClusterId()).toString());
}
@@ -152,20 +145,37 @@ public static ClusterMetrics toClusterMetrics(
*/
public static ClusterMetrics.Option toOption(ClusterStatusProtos.Option option) {
switch (option) {
- case HBASE_VERSION: return ClusterMetrics.Option.HBASE_VERSION;
- case LIVE_SERVERS: return ClusterMetrics.Option.LIVE_SERVERS;
- case DEAD_SERVERS: return ClusterMetrics.Option.DEAD_SERVERS;
- case REGIONS_IN_TRANSITION: return ClusterMetrics.Option.REGIONS_IN_TRANSITION;
- case CLUSTER_ID: return ClusterMetrics.Option.CLUSTER_ID;
- case MASTER_COPROCESSORS: return ClusterMetrics.Option.MASTER_COPROCESSORS;
- case MASTER: return ClusterMetrics.Option.MASTER;
- case BACKUP_MASTERS: return ClusterMetrics.Option.BACKUP_MASTERS;
- case BALANCER_ON: return ClusterMetrics.Option.BALANCER_ON;
- case SERVERS_NAME: return ClusterMetrics.Option.SERVERS_NAME;
- case MASTER_INFO_PORT: return ClusterMetrics.Option.MASTER_INFO_PORT;
- case TABLE_TO_REGIONS_COUNT: return ClusterMetrics.Option.TABLE_TO_REGIONS_COUNT;
+ case HBASE_VERSION:
+ return ClusterMetrics.Option.HBASE_VERSION;
+ case LIVE_SERVERS:
+ return ClusterMetrics.Option.LIVE_SERVERS;
+ case DEAD_SERVERS:
+ return ClusterMetrics.Option.DEAD_SERVERS;
+ case UNKNOWN_SERVERS:
+ return ClusterMetrics.Option.UNKNOWN_SERVERS;
+ case REGIONS_IN_TRANSITION:
+ return ClusterMetrics.Option.REGIONS_IN_TRANSITION;
+ case CLUSTER_ID:
+ return ClusterMetrics.Option.CLUSTER_ID;
+ case MASTER_COPROCESSORS:
+ return ClusterMetrics.Option.MASTER_COPROCESSORS;
+ case MASTER:
+ return ClusterMetrics.Option.MASTER;
+ case BACKUP_MASTERS:
+ return ClusterMetrics.Option.BACKUP_MASTERS;
+ case BALANCER_ON:
+ return ClusterMetrics.Option.BALANCER_ON;
+ case SERVERS_NAME:
+ return ClusterMetrics.Option.SERVERS_NAME;
+ case MASTER_INFO_PORT:
+ return ClusterMetrics.Option.MASTER_INFO_PORT;
+ case TABLE_TO_REGIONS_COUNT:
+ return ClusterMetrics.Option.TABLE_TO_REGIONS_COUNT;
+ case TASKS:
+ return ClusterMetrics.Option.TASKS;
// should not reach here
- default: throw new IllegalArgumentException("Invalid option: " + option);
+ default:
+ throw new IllegalArgumentException("Invalid option: " + option);
}
}
@@ -176,20 +186,37 @@ public static ClusterMetrics.Option toOption(ClusterStatusProtos.Option option)
*/
public static ClusterStatusProtos.Option toOption(ClusterMetrics.Option option) {
switch (option) {
- case HBASE_VERSION: return ClusterStatusProtos.Option.HBASE_VERSION;
- case LIVE_SERVERS: return ClusterStatusProtos.Option.LIVE_SERVERS;
- case DEAD_SERVERS: return ClusterStatusProtos.Option.DEAD_SERVERS;
- case REGIONS_IN_TRANSITION: return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
- case CLUSTER_ID: return ClusterStatusProtos.Option.CLUSTER_ID;
- case MASTER_COPROCESSORS: return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
- case MASTER: return ClusterStatusProtos.Option.MASTER;
- case BACKUP_MASTERS: return ClusterStatusProtos.Option.BACKUP_MASTERS;
- case BALANCER_ON: return ClusterStatusProtos.Option.BALANCER_ON;
- case SERVERS_NAME: return Option.SERVERS_NAME;
- case MASTER_INFO_PORT: return ClusterStatusProtos.Option.MASTER_INFO_PORT;
- case TABLE_TO_REGIONS_COUNT: return ClusterStatusProtos.Option.TABLE_TO_REGIONS_COUNT;
+ case HBASE_VERSION:
+ return ClusterStatusProtos.Option.HBASE_VERSION;
+ case LIVE_SERVERS:
+ return ClusterStatusProtos.Option.LIVE_SERVERS;
+ case DEAD_SERVERS:
+ return ClusterStatusProtos.Option.DEAD_SERVERS;
+ case UNKNOWN_SERVERS:
+ return ClusterStatusProtos.Option.UNKNOWN_SERVERS;
+ case REGIONS_IN_TRANSITION:
+ return ClusterStatusProtos.Option.REGIONS_IN_TRANSITION;
+ case CLUSTER_ID:
+ return ClusterStatusProtos.Option.CLUSTER_ID;
+ case MASTER_COPROCESSORS:
+ return ClusterStatusProtos.Option.MASTER_COPROCESSORS;
+ case MASTER:
+ return ClusterStatusProtos.Option.MASTER;
+ case BACKUP_MASTERS:
+ return ClusterStatusProtos.Option.BACKUP_MASTERS;
+ case BALANCER_ON:
+ return ClusterStatusProtos.Option.BALANCER_ON;
+ case SERVERS_NAME:
+ return Option.SERVERS_NAME;
+ case MASTER_INFO_PORT:
+ return ClusterStatusProtos.Option.MASTER_INFO_PORT;
+ case TABLE_TO_REGIONS_COUNT:
+ return ClusterStatusProtos.Option.TABLE_TO_REGIONS_COUNT;
+ case TASKS:
+ return ClusterStatusProtos.Option.TASKS;
// should not reach here
- default: throw new IllegalArgumentException("Invalid option: " + option);
+ default:
+ throw new IllegalArgumentException("Invalid option: " + option);
}
}
@@ -200,7 +227,7 @@ public static ClusterStatusProtos.Option toOption(ClusterMetrics.Option option)
*/
public static EnumSet toOptions(List options) {
return options.stream().map(ClusterMetricsBuilder::toOption)
- .collect(Collectors.toCollection(() -> EnumSet.noneOf(ClusterMetrics.Option.class)));
+ .collect(Collectors.toCollection(() -> EnumSet.noneOf(ClusterMetrics.Option.class)));
}
/**
@@ -215,9 +242,11 @@ public static List toOptions(EnumSet deadServerNames = Collections.emptyList();
+ private List unknownServerNames = Collections.emptyList();
private Map liveServerMetrics = new TreeMap<>();
@Nullable
private ServerName masterName;
@@ -231,18 +260,27 @@ public static ClusterMetricsBuilder newBuilder() {
private int masterInfoPort;
private List serversName = Collections.emptyList();
private Map tableRegionStatesCount = Collections.emptyMap();
+ @Nullable
+ private List masterTasks;
private ClusterMetricsBuilder() {
}
+
public ClusterMetricsBuilder setHBaseVersion(String value) {
this.hbaseVersion = value;
return this;
}
+
public ClusterMetricsBuilder setDeadServerNames(List value) {
this.deadServerNames = value;
return this;
}
+ public ClusterMetricsBuilder setUnknownServerNames(List value) {
+ this.unknownServerNames = value;
+ return this;
+ }
+
public ClusterMetricsBuilder setLiveServerMetrics(Map value) {
liveServerMetrics.putAll(value);
return this;
@@ -252,62 +290,66 @@ public ClusterMetricsBuilder setMasterName(ServerName value) {
this.masterName = value;
return this;
}
+
public ClusterMetricsBuilder setBackerMasterNames(List value) {
this.backupMasterNames = value;
return this;
}
+
public ClusterMetricsBuilder setRegionsInTransition(List value) {
this.regionsInTransition = value;
return this;
}
+
public ClusterMetricsBuilder setClusterId(String value) {
this.clusterId = value;
return this;
}
+
public ClusterMetricsBuilder setMasterCoprocessorNames(List value) {
this.masterCoprocessorNames = value;
return this;
}
+
public ClusterMetricsBuilder setBalancerOn(@Nullable Boolean value) {
this.balancerOn = value;
return this;
}
+
public ClusterMetricsBuilder setMasterInfoPort(int value) {
this.masterInfoPort = value;
return this;
}
+
public ClusterMetricsBuilder setServerNames(List serversName) {
this.serversName = serversName;
return this;
}
- public ClusterMetricsBuilder setTableRegionStatesCount(
- Map tableRegionStatesCount) {
+ public ClusterMetricsBuilder setMasterTasks(List masterTasks) {
+ this.masterTasks = masterTasks;
+ return this;
+ }
+
+ public ClusterMetricsBuilder
+ setTableRegionStatesCount(Map tableRegionStatesCount) {
this.tableRegionStatesCount = tableRegionStatesCount;
return this;
}
public ClusterMetrics build() {
- return new ClusterMetricsImpl(
- hbaseVersion,
- deadServerNames,
- liveServerMetrics,
- masterName,
- backupMasterNames,
- regionsInTransition,
- clusterId,
- masterCoprocessorNames,
- balancerOn,
- masterInfoPort,
- serversName,
- tableRegionStatesCount
- );
+ return new ClusterMetricsImpl(hbaseVersion, deadServerNames, unknownServerNames,
+ liveServerMetrics, masterName, backupMasterNames, regionsInTransition, clusterId,
+ masterCoprocessorNames, balancerOn, masterInfoPort, serversName, tableRegionStatesCount,
+ masterTasks);
}
+
private static class ClusterMetricsImpl implements ClusterMetrics {
@Nullable
private final String hbaseVersion;
private final List deadServerNames;
private final Map liveServerMetrics;
+ private final List unknownServerNames;
@Nullable
private final ServerName masterName;
private final List backupMasterNames;
@@ -320,20 +362,17 @@ private static class ClusterMetricsImpl implements ClusterMetrics {
private final int masterInfoPort;
private final List serversName;
private final Map tableRegionStatesCount;
+ private final List masterTasks;
ClusterMetricsImpl(String hbaseVersion, List deadServerNames,
- Map liveServerMetrics,
- ServerName masterName,
- List backupMasterNames,
- List regionsInTransition,
- String clusterId,
- List masterCoprocessorNames,
- Boolean balancerOn,
- int masterInfoPort,
- List serversName,
- Map tableRegionStatesCount) {
+ List unknownServerNames, Map liveServerMetrics,
+ ServerName masterName, List backupMasterNames,
+ List regionsInTransition, String clusterId, List masterCoprocessorNames,
+ Boolean balancerOn, int masterInfoPort, List serversName,
+ Map tableRegionStatesCount, List masterTasks) {
this.hbaseVersion = hbaseVersion;
this.deadServerNames = Preconditions.checkNotNull(deadServerNames);
+ this.unknownServerNames = Preconditions.checkNotNull(unknownServerNames);
this.liveServerMetrics = Preconditions.checkNotNull(liveServerMetrics);
this.masterName = masterName;
this.backupMasterNames = Preconditions.checkNotNull(backupMasterNames);
@@ -344,6 +383,7 @@ private static class ClusterMetricsImpl implements ClusterMetrics {
this.masterInfoPort = masterInfoPort;
this.serversName = serversName;
this.tableRegionStatesCount = Preconditions.checkNotNull(tableRegionStatesCount);
+ this.masterTasks = masterTasks;
}
@Override
@@ -356,6 +396,11 @@ public List getDeadServerNames() {
return Collections.unmodifiableList(deadServerNames);
}
+ @Override
+ public List getUnknownServerNames() {
+ return Collections.unmodifiableList(unknownServerNames);
+ }
+
@Override
public Map getLiveServerMetrics() {
return Collections.unmodifiableMap(liveServerMetrics);
@@ -406,6 +451,11 @@ public Map getTableRegionStatesCount() {
return Collections.unmodifiableMap(tableRegionStatesCount);
}
+ @Override
+ public List getMasterTasks() {
+ return masterTasks;
+ }
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder(1024);
@@ -414,15 +464,15 @@ public String toString() {
int backupMastersSize = getBackupMasterNames().size();
sb.append("\nNumber of backup masters: " + backupMastersSize);
if (backupMastersSize > 0) {
- for (ServerName serverName: getBackupMasterNames()) {
+ for (ServerName serverName : getBackupMasterNames()) {
sb.append("\n " + serverName);
}
}
int serversSize = getLiveServerMetrics().size();
int serversNameSize = getServersName().size();
- sb.append("\nNumber of live region servers: "
- + (serversSize > 0 ? serversSize : serversNameSize));
+ sb.append(
+ "\nNumber of live region servers: " + (serversSize > 0 ? serversSize : serversNameSize));
if (serversSize > 0) {
for (ServerName serverName : getLiveServerMetrics().keySet()) {
sb.append("\n " + serverName.getServerName());
@@ -441,6 +491,14 @@ public String toString() {
}
}
+ int unknownServerSize = getUnknownServerNames().size();
+ sb.append("\nNumber of unknown region servers: " + unknownServerSize);
+ if (unknownServerSize > 0) {
+ for (ServerName serverName : getUnknownServerNames()) {
+ sb.append("\n " + serverName);
+ }
+ }
+
sb.append("\nAverage load: " + getAverageLoad());
sb.append("\nNumber of requests: " + getRequestCount());
sb.append("\nNumber of regions: " + getRegionCount());
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
index 6fdb588a4f37..d21d610126a0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ClusterStatus.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -16,7 +15,6 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-
package org.apache.hadoop.hbase;
import edu.umd.cs.findbugs.annotations.Nullable;
@@ -26,7 +24,6 @@
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
-
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.yetus.audience.InterfaceAudience;
@@ -45,32 +42,37 @@
* The average cluster load.
* The number of regions deployed on the cluster.
* The number of requests since last report.
- * Detailed region server loading and resource usage information,
- * per server and per region.
+ * Detailed region server loading and resource usage information, per server and per
+ * region.
* Regions in transition at master
* The unique cluster ID
*
* {@link ClusterMetrics.Option} provides a way to get desired ClusterStatus information.
* The following codes will get all the cluster information.
+ *
*
- * {@code
- * // Original version still works
- * Admin admin = connection.getAdmin();
- * ClusterStatus status = admin.getClusterStatus();
- * // or below, a new version which has the same effects
- * ClusterStatus status = admin.getClusterStatus(EnumSet.allOf(Option.class));
+ * {
+ * @code
+ * // Original version still works
+ * Admin admin = connection.getAdmin();
+ * ClusterStatus status = admin.getClusterStatus();
+ * // or below, a new version which has the same effects
+ * ClusterStatus status = admin.getClusterStatus(EnumSet.allOf(Option.class));
* }
*
- * If information about live servers is the only wanted.
- * then codes in the following way:
+ *
+ * If information about live servers is the only wanted. then codes in the following way:
+ *
*
- * {@code
- * Admin admin = connection.getAdmin();
- * ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
+ * {
+ * @code
+ * Admin admin = connection.getAdmin();
+ * ClusterStatus status = admin.getClusterStatus(EnumSet.of(Option.LIVE_SERVERS));
* }
*
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link ClusterMetrics} instead.
+ *
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use {@link ClusterMetrics}
+ * instead.
*/
@InterfaceAudience.Public
@Deprecated
@@ -86,26 +88,18 @@ public class ClusterStatus implements ClusterMetrics {
*/
@Deprecated
public ClusterStatus(final String hbaseVersion, final String clusterid,
- final Map servers,
- final Collection deadServers,
- final ServerName master,
- final Collection backupMasters,
- final List rit,
- final String[] masterCoprocessors,
- final Boolean balancerOn,
- final int masterInfoPort) {
+ final Map servers, final Collection deadServers,
+ final ServerName master, final Collection backupMasters,
+ final List rit, final String[] masterCoprocessors, final Boolean balancerOn,
+ final int masterInfoPort) {
// TODO: make this constructor private
this(ClusterMetricsBuilder.newBuilder().setHBaseVersion(hbaseVersion)
.setDeadServerNames(new ArrayList<>(deadServers))
- .setLiveServerMetrics(servers.entrySet().stream()
- .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())))
+ .setLiveServerMetrics(
+ servers.entrySet().stream().collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())))
.setBackerMasterNames(new ArrayList<>(backupMasters)).setBalancerOn(balancerOn)
- .setClusterId(clusterid)
- .setMasterCoprocessorNames(Arrays.asList(masterCoprocessors))
- .setMasterName(master)
- .setMasterInfoPort(masterInfoPort)
- .setRegionsInTransition(rit)
- .build());
+ .setClusterId(clusterid).setMasterCoprocessorNames(Arrays.asList(masterCoprocessors))
+ .setMasterName(master).setMasterInfoPort(masterInfoPort).setRegionsInTransition(rit).build());
}
@InterfaceAudience.Private
@@ -113,24 +107,27 @@ public ClusterStatus(ClusterMetrics metrics) {
this.metrics = metrics;
}
- /**
- * @return the names of region servers on the dead list
- */
+ /** Returns the names of region servers on the dead list */
@Override
public List getDeadServerNames() {
return metrics.getDeadServerNames();
}
+ @Override
+ public List getUnknownServerNames() {
+ return metrics.getUnknownServerNames();
+ }
+
@Override
public Map getLiveServerMetrics() {
return metrics.getLiveServerMetrics();
}
/**
- * @return the number of region servers in the cluster
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getLiveServerMetrics()}.
- */
+ * @return the number of region servers in the cluster
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getLiveServerMetrics()}.
+ */
@Deprecated
public int getServersSize() {
return metrics.getLiveServerMetrics().size();
@@ -139,8 +136,8 @@ public int getServersSize() {
/**
* @return the number of dead region servers in the cluster
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-13656).
- * Use {@link #getDeadServerNames()}.
+ * (HBASE-13656). Use
+ * {@link #getDeadServerNames()}.
*/
@Deprecated
public int getDeadServers() {
@@ -149,8 +146,8 @@ public int getDeadServers() {
/**
* @return the number of dead region servers in the cluster
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getDeadServerNames()}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getDeadServerNames()}.
*/
@Deprecated
public int getDeadServersSize() {
@@ -159,8 +156,8 @@ public int getDeadServersSize() {
/**
* @return the number of regions deployed on the cluster
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getRegionCount()}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getRegionCount()}.
*/
@Deprecated
public int getRegionsCount() {
@@ -169,8 +166,8 @@ public int getRegionsCount() {
/**
* @return the number of requests since last report
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getRequestCount()} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getRequestCount()} instead.
*/
@Deprecated
public int getRequestsCount() {
@@ -193,9 +190,8 @@ public List getRegionStatesInTransition() {
return metrics.getRegionStatesInTransition();
}
- /**
- * @return the HBase version string as reported by the HMaster
- */
+ /** Returns the HBase version string as reported by the HMaster */
+ @Override
public String getHBaseVersion() {
return metrics.getHBaseVersion();
}
@@ -214,14 +210,14 @@ public boolean equals(Object o) {
return false;
}
ClusterStatus other = (ClusterStatus) o;
- return Objects.equal(getHBaseVersion(), other.getHBaseVersion()) &&
- Objects.equal(getLiveServerLoads(), other.getLiveServerLoads()) &&
- getDeadServerNames().containsAll(other.getDeadServerNames()) &&
- Arrays.equals(getMasterCoprocessors(), other.getMasterCoprocessors()) &&
- Objects.equal(getMaster(), other.getMaster()) &&
- getBackupMasters().containsAll(other.getBackupMasters()) &&
- Objects.equal(getClusterId(), other.getClusterId()) &&
- getMasterInfoPort() == other.getMasterInfoPort();
+ return Objects.equal(getHBaseVersion(), other.getHBaseVersion())
+ && Objects.equal(getLiveServerLoads(), other.getLiveServerLoads())
+ && getDeadServerNames().containsAll(other.getDeadServerNames())
+ && Arrays.equals(getMasterCoprocessors(), other.getMasterCoprocessors())
+ && Objects.equal(getMaster(), other.getMaster())
+ && getBackupMasters().containsAll(other.getBackupMasters())
+ && Objects.equal(getClusterId(), other.getClusterId())
+ && getMasterInfoPort() == other.getMasterInfoPort();
}
@Override
@@ -239,8 +235,8 @@ public byte getVersion() {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getLiveServerMetrics()} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getLiveServerMetrics()} instead.
*/
@Deprecated
public Collection getServers() {
@@ -250,8 +246,8 @@ public Collection getServers() {
/**
* Returns detailed information about the current master {@link ServerName}.
* @return current master information if it exists
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getMasterName} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use {@link #getMasterName}
+ * instead.
*/
@Deprecated
public ServerName getMaster() {
@@ -260,8 +256,8 @@ public ServerName getMaster() {
/**
* @return the number of backup masters in the cluster
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getBackupMasterNames} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getBackupMasterNames} instead.
*/
@Deprecated
public int getBackupMastersSize() {
@@ -270,8 +266,8 @@ public int getBackupMastersSize() {
/**
* @return the names of backup masters
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getBackupMasterNames} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getBackupMasterNames} instead.
*/
@Deprecated
public List getBackupMasters() {
@@ -279,10 +275,9 @@ public List getBackupMasters() {
}
/**
- * @param sn
* @return Server's load or null if not found.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getLiveServerMetrics} instead.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getLiveServerMetrics} instead.
*/
@Deprecated
public ServerLoad getLoad(final ServerName sn) {
@@ -290,6 +285,7 @@ public ServerLoad getLoad(final ServerName sn) {
return serverMetrics == null ? null : new ServerLoad(serverMetrics);
}
+ @Override
public String getClusterId() {
return metrics.getClusterId();
}
@@ -300,8 +296,9 @@ public List getMasterCoprocessorNames() {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getMasterCoprocessorNames} instead.
+ * Get the list of master coprocessor names.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getMasterCoprocessorNames} instead.
*/
@Deprecated
public String[] getMasterCoprocessors() {
@@ -310,8 +307,9 @@ public String[] getMasterCoprocessors() {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getLastMajorCompactionTimestamp(TableName)} instead.
+ * Get the last major compaction time for a given table.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getLastMajorCompactionTimestamp(TableName)} instead.
*/
@Deprecated
public long getLastMajorCompactionTsForTable(TableName table) {
@@ -319,8 +317,9 @@ public long getLastMajorCompactionTsForTable(TableName table) {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link #getLastMajorCompactionTimestamp(byte[])} instead.
+ * Get the last major compaction time for a given region.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link #getLastMajorCompactionTimestamp(byte[])} instead.
*/
@Deprecated
public long getLastMajorCompactionTsForRegion(final byte[] region) {
@@ -328,8 +327,8 @@ public long getLastMajorCompactionTsForRegion(final byte[] region) {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * No flag in 2.0
+ * Returns true if the balancer is enabled.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 No flag in 2.0
*/
@Deprecated
public boolean isBalancerOn() {
@@ -356,6 +355,11 @@ public Map getTableRegionStatesCount() {
return metrics.getTableRegionStatesCount();
}
+ @Override
+ public List getMasterTasks() {
+ return metrics.getMasterTasks();
+ }
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder(1024);
@@ -364,15 +368,15 @@ public String toString() {
int backupMastersSize = getBackupMastersSize();
sb.append("\nNumber of backup masters: " + backupMastersSize);
if (backupMastersSize > 0) {
- for (ServerName serverName: metrics.getBackupMasterNames()) {
+ for (ServerName serverName : metrics.getBackupMasterNames()) {
sb.append("\n " + serverName);
}
}
int serversSize = getServersSize();
int serversNameSize = getServersName().size();
- sb.append("\nNumber of live region servers: "
- + (serversSize > 0 ? serversSize : serversNameSize));
+ sb.append(
+ "\nNumber of live region servers: " + (serversSize > 0 ? serversSize : serversNameSize));
if (serversSize > 0) {
for (ServerName serverName : metrics.getLiveServerMetrics().keySet()) {
sb.append("\n " + serverName.getServerName());
@@ -398,7 +402,7 @@ public String toString() {
int ritSize = metrics.getRegionStatesInTransition().size();
sb.append("\nNumber of regions in transition: " + ritSize);
if (ritSize > 0) {
- for (RegionState state: metrics.getRegionStatesInTransition()) {
+ for (RegionState state : metrics.getRegionStatesInTransition()) {
sb.append("\n " + state.toDescriptiveString());
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ConcurrentTableModificationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ConcurrentTableModificationException.java
index 86aca2bc8177..b8b2519dc09f 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ConcurrentTableModificationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ConcurrentTableModificationException.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java
index c0d9b603a8ab..20cc35da042d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/Coprocessor.java
@@ -7,33 +7,28 @@
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
-
package org.apache.hadoop.hbase;
+import com.google.protobuf.Service;
import java.io.IOException;
import java.util.Collections;
-
-import com.google.protobuf.Service;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
/**
* Base interface for the 4 coprocessors - MasterCoprocessor, RegionCoprocessor,
- * RegionServerCoprocessor, and WALCoprocessor.
- * Do NOT implement this interface directly. Unless an implementation implements one (or more) of
- * the above mentioned 4 coprocessors, it'll fail to be loaded by any coprocessor host.
+ * RegionServerCoprocessor, and WALCoprocessor. Do NOT implement this interface directly. Unless an
+ * implementation implements one (or more) of the above mentioned 4 coprocessors, it'll fail to be
+ * loaded by any coprocessor host. Example: Building a coprocessor to observe Master operations.
*
- * Example:
- * Building a coprocessor to observe Master operations.
*
* class MyMasterCoprocessor implements MasterCoprocessor {
* @Override
@@ -48,6 +43,7 @@
*
*
* Building a Service which can be loaded by both Master and RegionServer
+ *
*
* class MyCoprocessorService implements MasterCoprocessor, RegionServerCoprocessor {
* @Override
@@ -87,18 +83,19 @@ enum State {
* Called by the {@link CoprocessorEnvironment} during it's own startup to initialize the
* coprocessor.
*/
- default void start(CoprocessorEnvironment env) throws IOException {}
+ default void start(CoprocessorEnvironment env) throws IOException {
+ }
/**
- * Called by the {@link CoprocessorEnvironment} during it's own shutdown to stop the
- * coprocessor.
+ * Called by the {@link CoprocessorEnvironment} during it's own shutdown to stop the coprocessor.
*/
- default void stop(CoprocessorEnvironment env) throws IOException {}
+ default void stop(CoprocessorEnvironment env) throws IOException {
+ }
/**
* Coprocessor endpoints providing protobuf services should override this method.
- * @return Iterable of {@link Service}s or empty collection. Implementations should never
- * return null.
+ * @return Iterable of {@link Service}s or empty collection. Implementations should never return
+ * null.
*/
default Iterable getServices() {
return Collections.EMPTY_SET;
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
index 4fab7333dcd9..32e06d610247 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/CoprocessorEnvironment.java
@@ -7,16 +7,14 @@
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
-
package org.apache.hadoop.hbase;
import org.apache.hadoop.conf.Configuration;
@@ -30,29 +28,27 @@
@InterfaceStability.Evolving
public interface CoprocessorEnvironment {
- /** @return the Coprocessor interface version */
+ /** Returns the Coprocessor interface version */
int getVersion();
- /** @return the HBase version as a string (e.g. "0.21.0") */
+ /** Returns the HBase version as a string (e.g. "0.21.0") */
String getHBaseVersion();
- /** @return the loaded coprocessor instance */
+ /** Returns the loaded coprocessor instance */
C getInstance();
- /** @return the priority assigned to the loaded coprocessor */
+ /** Returns the priority assigned to the loaded coprocessor */
int getPriority();
- /** @return the load sequence number */
+ /** Returns the load sequence number */
int getLoadSequence();
/**
- * @return a Read-only Configuration; throws {@link UnsupportedOperationException} if you try
- * to set a configuration.
+ * Returns a Read-only Configuration; throws {@link UnsupportedOperationException} if you try to
+ * set a configuration.
*/
Configuration getConfiguration();
- /**
- * @return the classloader for the loaded coprocessor instance
- */
+ /** Returns the classloader for the loaded coprocessor instance */
ClassLoader getClassLoader();
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
index 509844e367d8..7e1821de7d47 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DoNotRetryIOException.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -41,7 +40,7 @@ public DoNotRetryIOException(String message) {
}
/**
- * @param message the message for this exception
+ * @param message the message for this exception
* @param throwable the {@link Throwable} to use for this exception
*/
public DoNotRetryIOException(String message, Throwable throwable) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
index 76f374c412f0..f4391f1025c4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/DroppedSnapshotException.java
@@ -7,24 +7,22 @@
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package org.apache.hadoop.hbase;
import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
/**
- * Thrown during flush if the possibility snapshot content was not properly
- * persisted into store files. Response should include replay of wal content.
+ * Thrown during flush if the possibility snapshot content was not properly persisted into store
+ * files. Response should include replay of wal content.
*/
@InterfaceAudience.Public
public class DroppedSnapshotException extends IOException {
@@ -43,9 +41,8 @@ public DroppedSnapshotException(String message) {
/**
* DroppedSnapshotException with cause
- *
* @param message the message for this exception
- * @param cause the cause for this exception
+ * @param cause the cause for this exception
*/
public DroppedSnapshotException(String message, Throwable cause) {
super(message, cause);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/FailedCloseWALAfterInitializedErrorException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/FailedCloseWALAfterInitializedErrorException.java
index 6445be9cfaf8..e5e2f7b7ccaf 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/FailedCloseWALAfterInitializedErrorException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/FailedCloseWALAfterInitializedErrorException.java
@@ -7,35 +7,31 @@
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
- * http://www.apache.org/licenses/LICENSE-2.0
+ * http://www.apache.org/licenses/LICENSE-2.0
*
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
*/
package org.apache.hadoop.hbase;
-
import java.io.IOException;
-
import org.apache.yetus.audience.InterfaceAudience;
/**
* Throw when failed cleanup unsuccessful initialized wal
*/
@InterfaceAudience.Public
-public class FailedCloseWALAfterInitializedErrorException
- extends IOException {
+public class FailedCloseWALAfterInitializedErrorException extends IOException {
private static final long serialVersionUID = -5463156587431677322L;
/**
* constructor with error msg and throwable
* @param msg message
- * @param t throwable
+ * @param t throwable
*/
public FailedCloseWALAfterInitializedErrorException(String msg, Throwable t) {
super(msg, t);
@@ -55,4 +51,4 @@ public FailedCloseWALAfterInitializedErrorException(String msg) {
public FailedCloseWALAfterInitializedErrorException() {
super();
}
-}
\ No newline at end of file
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HBaseServerException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HBaseServerException.java
new file mode 100644
index 000000000000..47a86f9492f5
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HBaseServerException.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * Base class for exceptions thrown by an HBase server. May contain extra info about the state of
+ * the server when the exception was thrown.
+ */
+@InterfaceAudience.Public
+public class HBaseServerException extends HBaseIOException {
+ private boolean serverOverloaded;
+
+ public HBaseServerException() {
+ this(false);
+ }
+
+ public HBaseServerException(String message) {
+ this(false, message);
+ }
+
+ public HBaseServerException(boolean serverOverloaded) {
+ this.serverOverloaded = serverOverloaded;
+ }
+
+ public HBaseServerException(boolean serverOverloaded, String message) {
+ super(message);
+ this.serverOverloaded = serverOverloaded;
+ }
+
+ /** Returns True if the server was considered overloaded when the exception was thrown */
+ public static boolean isServerOverloaded(Throwable t) {
+ if (t instanceof HBaseServerException) {
+ return ((HBaseServerException) t).isServerOverloaded();
+ }
+ return false;
+ }
+
+ /**
+ * Necessary for parsing RemoteException on client side
+ * @param serverOverloaded True if server was overloaded when exception was thrown
+ */
+ public void setServerOverloaded(boolean serverOverloaded) {
+ this.serverOverloaded = serverOverloaded;
+ }
+
+ /** Returns True if server was considered overloaded when exception was thrown */
+ public boolean isServerOverloaded() {
+ return serverOverloaded;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
index 2f21d60878bf..d2d39a4c4156 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HColumnDescriptor.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -19,8 +18,6 @@
package org.apache.hadoop.hbase;
import java.util.Map;
-
-import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder.ModifyableColumnFamilyDescriptor;
@@ -29,33 +26,43 @@
import org.apache.hadoop.hbase.exceptions.HBaseException;
import org.apache.hadoop.hbase.io.compress.Compression;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
+import org.apache.hadoop.hbase.io.encoding.IndexBlockEncoding;
import org.apache.hadoop.hbase.regionserver.BloomType;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.PrettyPrinter.Unit;
+import org.apache.yetus.audience.InterfaceAudience;
/**
- * An HColumnDescriptor contains information about a column family such as the
- * number of versions, compression settings, etc.
- *
- * It is used as input when creating a table or adding a column.
+ * An HColumnDescriptor contains information about a column family such as the number of versions,
+ * compression settings, etc. It is used as input when creating a table or adding a column.
*/
@InterfaceAudience.Public
@Deprecated // remove it in 3.0
public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable {
- public static final String IN_MEMORY_COMPACTION = ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION;
+ public static final String IN_MEMORY_COMPACTION =
+ ColumnFamilyDescriptorBuilder.IN_MEMORY_COMPACTION;
public static final String COMPRESSION = ColumnFamilyDescriptorBuilder.COMPRESSION;
- public static final String COMPRESSION_COMPACT = ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT;
- public static final String COMPRESSION_COMPACT_MAJOR = ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT_MAJOR;
- public static final String COMPRESSION_COMPACT_MINOR = ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT_MINOR;
+ public static final String COMPRESSION_COMPACT =
+ ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT;
+ public static final String COMPRESSION_COMPACT_MAJOR =
+ ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT_MAJOR;
+ public static final String COMPRESSION_COMPACT_MINOR =
+ ColumnFamilyDescriptorBuilder.COMPRESSION_COMPACT_MINOR;
public static final String ENCODE_ON_DISK = "ENCODE_ON_DISK";
- public static final String DATA_BLOCK_ENCODING = ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING;
+ public static final String DATA_BLOCK_ENCODING =
+ ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING;
public static final String BLOCKCACHE = ColumnFamilyDescriptorBuilder.BLOCKCACHE;
- public static final String CACHE_DATA_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_DATA_ON_WRITE;
- public static final String CACHE_INDEX_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_INDEX_ON_WRITE;
- public static final String CACHE_BLOOMS_ON_WRITE = ColumnFamilyDescriptorBuilder.CACHE_BLOOMS_ON_WRITE;
- public static final String EVICT_BLOCKS_ON_CLOSE = ColumnFamilyDescriptorBuilder.EVICT_BLOCKS_ON_CLOSE;
+ public static final String CACHE_DATA_ON_WRITE =
+ ColumnFamilyDescriptorBuilder.CACHE_DATA_ON_WRITE;
+ public static final String CACHE_INDEX_ON_WRITE =
+ ColumnFamilyDescriptorBuilder.CACHE_INDEX_ON_WRITE;
+ public static final String CACHE_BLOOMS_ON_WRITE =
+ ColumnFamilyDescriptorBuilder.CACHE_BLOOMS_ON_WRITE;
+ public static final String EVICT_BLOCKS_ON_CLOSE =
+ ColumnFamilyDescriptorBuilder.EVICT_BLOCKS_ON_CLOSE;
public static final String CACHE_DATA_IN_L1 = "CACHE_DATA_IN_L1";
- public static final String PREFETCH_BLOCKS_ON_OPEN = ColumnFamilyDescriptorBuilder.PREFETCH_BLOCKS_ON_OPEN;
+ public static final String PREFETCH_BLOCKS_ON_OPEN =
+ ColumnFamilyDescriptorBuilder.PREFETCH_BLOCKS_ON_OPEN;
public static final String BLOCKSIZE = ColumnFamilyDescriptorBuilder.BLOCKSIZE;
public static final String LENGTH = "LENGTH";
public static final String TTL = ColumnFamilyDescriptorBuilder.TTL;
@@ -72,46 +79,62 @@ public class HColumnDescriptor implements ColumnFamilyDescriptor, Comparable:
+ * Construct a column descriptor specifying only the family name The other attributes are
+ * defaulted.
+ * @param familyName Column family name. Must be 'printable' -- digit or letter -- and may not
+ * contain a :
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-18433).
- * Use {@link ColumnFamilyDescriptorBuilder#of(String)}.
+ * (HBASE-18433). Use
+ * {@link ColumnFamilyDescriptorBuilder#of(String)}.
*/
@Deprecated
public HColumnDescriptor(final String familyName) {
@@ -119,29 +142,26 @@ public HColumnDescriptor(final String familyName) {
}
/**
- * Construct a column descriptor specifying only the family name
- * The other attributes are defaulted.
- *
- * @param familyName Column family name. Must be 'printable' -- digit or
- * letter -- and may not contain a :
+ * Construct a column descriptor specifying only the family name The other attributes are
+ * defaulted.
+ * @param familyName Column family name. Must be 'printable' -- digit or letter -- and may not
+ * contain a :
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-18433).
- * Use {@link ColumnFamilyDescriptorBuilder#of(byte[])}.
+ * (HBASE-18433). Use
+ * {@link ColumnFamilyDescriptorBuilder#of(byte[])}.
*/
@Deprecated
- public HColumnDescriptor(final byte [] familyName) {
+ public HColumnDescriptor(final byte[] familyName) {
this(new ModifyableColumnFamilyDescriptor(familyName));
}
/**
- * Constructor.
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an UnmodifyableHColumnDescriptor.
- *
+ * Constructor. Makes a deep copy of the supplied descriptor. Can make a modifiable descriptor
+ * from an UnmodifyableHColumnDescriptor.
* @param desc The descriptor.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-18433).
- * Use {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}.
+ * (HBASE-18433). Use
+ * {@link ColumnFamilyDescriptorBuilder#copy(ColumnFamilyDescriptor)}.
*/
@Deprecated
public HColumnDescriptor(HColumnDescriptor desc) {
@@ -149,8 +169,7 @@ public HColumnDescriptor(HColumnDescriptor desc) {
}
protected HColumnDescriptor(HColumnDescriptor desc, boolean deepClone) {
- this(deepClone ? new ModifyableColumnFamilyDescriptor(desc)
- : desc.delegatee);
+ this(deepClone ? new ModifyableColumnFamilyDescriptor(desc) : desc.delegatee);
}
protected HColumnDescriptor(ModifyableColumnFamilyDescriptor delegate) {
@@ -158,51 +177,42 @@ protected HColumnDescriptor(ModifyableColumnFamilyDescriptor delegate) {
}
/**
+ * Check if a given family name is allowed.
* @param b Family name.
* @return b
- * @throws IllegalArgumentException If not null and not a legitimate family
- * name: i.e. 'printable' and ends in a ':' (Null passes are allowed because
- * b can be null when deserializing). Cannot start with a '.'
- * either. Also Family can not be an empty value or equal "recovered.edits".
+ * @throws IllegalArgumentException If not null and not a legitimate family name: i.e. 'printable'
+ * and ends in a ':' (Null passes are allowed because
+ * b can be null when deserializing). Cannot start
+ * with a '.' either. Also Family can not be an empty value or
+ * equal "recovered.edits".
* @deprecated since 2.0.0 and will be removed in 3.0.0. Use
- * {@link ColumnFamilyDescriptorBuilder#isLegalColumnFamilyName(byte[])} instead.
+ * {@link ColumnFamilyDescriptorBuilder#isLegalColumnFamilyName(byte[])} instead.
* @see ColumnFamilyDescriptorBuilder#isLegalColumnFamilyName(byte[])
* @see HBASE-18008
*/
@Deprecated
- public static byte [] isLegalFamilyName(final byte [] b) {
+ public static byte[] isLegalFamilyName(final byte[] b) {
return ColumnFamilyDescriptorBuilder.isLegalColumnFamilyName(b);
}
- /**
- * @return Name of this column family
- */
+ /** Returns Name of this column family */
@Override
- public byte [] getName() {
+ public byte[] getName() {
return delegatee.getName();
}
- /**
- * @return The name string of this column family
- */
+ /** Returns The name string of this column family */
@Override
public String getNameAsString() {
return delegatee.getNameAsString();
}
- /**
- * @param key The key.
- * @return The value.
- */
@Override
public byte[] getValue(byte[] key) {
return delegatee.getValue(key);
}
- /**
- * @param key The key.
- * @return The value as a string.
- */
+ @Override
public String getValue(String key) {
byte[] value = getValue(Bytes.toBytes(key));
return value == null ? null : Bytes.toString(value);
@@ -213,38 +223,25 @@ public Map getValues() {
return delegatee.getValues();
}
- /**
- * @param key The key.
- * @param value The value.
- * @return this (for chained invocation)
- */
public HColumnDescriptor setValue(byte[] key, byte[] value) {
getDelegateeForModification().setValue(key, value);
return this;
}
- /**
- * @param key Key whose key and value we're to remove from HCD parameters.
- */
- public void remove(final byte [] key) {
+ public void remove(final byte[] key) {
getDelegateeForModification().removeValue(new Bytes(key));
}
- /**
- * @param key The key.
- * @param value The value.
- * @return this (for chained invocation)
- */
public HColumnDescriptor setValue(String key, String value) {
getDelegateeForModification().setValue(key, value);
return this;
}
/**
- * @return compression type being used for the column family
+ * Returns compression type being used for the column family
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-13655).
- * Use {@link #getCompressionType()}.
+ * (HBASE-13655). Use
+ * {@link #getCompressionType()}.
*/
@Deprecated
public Compression.Algorithm getCompression() {
@@ -252,10 +249,10 @@ public Compression.Algorithm getCompression() {
}
/**
- * @return compression type being used for the column family for major compaction
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-13655).
- * Use {@link #getCompactionCompressionType()}.
+ * Returns compression type being used for the column family for major compaction
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
+ * (HBASE-13655). Use
+ * {@link #getCompactionCompressionType()}.
*/
@Deprecated
public Compression.Algorithm getCompactionCompression() {
@@ -268,6 +265,7 @@ public int getMaxVersions() {
}
/**
+ * Set maximum versions to keep
* @param value maximum number of versions
* @return this (for chained invocation)
*/
@@ -278,7 +276,6 @@ public HColumnDescriptor setMaxVersions(int value) {
/**
* Set minimum and maximum versions to keep
- *
* @param minVersions minimal number of versions
* @param maxVersions maximum number of versions
* @return this (for chained invocation)
@@ -291,9 +288,9 @@ public HColumnDescriptor setVersions(int minVersions, int maxVersions) {
}
if (maxVersions < minVersions) {
- throw new IllegalArgumentException("Unable to set MaxVersion to " + maxVersions
- + " and set MinVersion to " + minVersions
- + ", as maximum versions must be >= minimum versions.");
+ throw new IllegalArgumentException(
+ "Unable to set MaxVersion to " + maxVersions + " and set MinVersion to " + minVersions
+ + ", as maximum versions must be >= minimum versions.");
}
setMinVersions(minVersions);
setMaxVersions(maxVersions);
@@ -306,8 +303,8 @@ public int getBlocksize() {
}
/**
- * @param value Blocksize to use when writing out storefiles/hfiles on this
- * column family.
+ * Set block size to use when writing
+ * @param value Blocksize to use when writing out storefiles/hfiles on this column family.
* @return this (for chained invocation)
*/
public HColumnDescriptor setBlocksize(int value) {
@@ -326,10 +323,9 @@ public Compression.Algorithm getCompressionType() {
}
/**
- * Compression types supported in hbase.
- * LZO is not bundled as part of the hbase distribution.
- * See LZO Compression
- * for how to enable it.
+ * Compression types supported in hbase. LZO is not bundled as part of the hbase distribution. See
+ * LZO Compression for how to
+ * enable it.
* @param value Compression type setting.
* @return this (for chained invocation)
*/
@@ -343,6 +339,11 @@ public DataBlockEncoding getDataBlockEncoding() {
return delegatee.getDataBlockEncoding();
}
+ @Override
+ public IndexBlockEncoding getIndexBlockEncoding() {
+ return delegatee.getIndexBlockEncoding();
+ }
+
/**
* Set data block encoding algorithm used in block cache.
* @param value What kind of data block encoding will be used.
@@ -356,8 +357,6 @@ public HColumnDescriptor setDataBlockEncoding(DataBlockEncoding value) {
/**
* Set whether the tags should be compressed along with DataBlockEncoding. When no
* DataBlockEncoding is been used, this is having no effect.
- *
- * @param value
* @return this (for chained invocation)
*/
public HColumnDescriptor setCompressTags(boolean value) {
@@ -386,10 +385,9 @@ public Compression.Algorithm getMinorCompactionCompressionType() {
}
/**
- * Compression types supported in hbase.
- * LZO is not bundled as part of the hbase distribution.
- * See LZO Compression
- * for how to enable it.
+ * Compression types supported in hbase. LZO is not bundled as part of the hbase distribution. See
+ * LZO Compression for how to
+ * enable it.
* @param value Compression type setting.
* @return this (for chained invocation)
*/
@@ -414,8 +412,9 @@ public boolean isInMemory() {
}
/**
+ * Set or clear the in memory flag.
* @param value True if we are to favor keeping all values for this column family in the
- * HRegionServer cache
+ * HRegionServer cache
* @return this (for chained invocation)
*/
public HColumnDescriptor setInMemory(boolean value) {
@@ -429,8 +428,8 @@ public MemoryCompactionPolicy getInMemoryCompaction() {
}
/**
- * @param value the prefered in-memory compaction policy
- * for this column family
+ * Set the in memory compaction policy.
+ * @param value the prefered in-memory compaction policy for this column family
* @return this (for chained invocation)
*/
public HColumnDescriptor setInMemoryCompaction(MemoryCompactionPolicy value) {
@@ -444,8 +443,8 @@ public KeepDeletedCells getKeepDeletedCells() {
}
/**
- * @param value True if deleted rows should not be collected
- * immediately.
+ * Set the keep deleted cells policy.
+ * @param value True if deleted rows should not be collected immediately.
* @return this (for chained invocation)
*/
public HColumnDescriptor setKeepDeletedCells(KeepDeletedCells value) {
@@ -454,9 +453,9 @@ public HColumnDescriptor setKeepDeletedCells(KeepDeletedCells value) {
}
/**
- * By default, HBase only consider timestamp in versions. So a previous Delete with higher ts
- * will mask a later Put with lower ts. Set this to true to enable new semantics of versions.
- * We will also consider mvcc in versions. See HBASE-15968 for details.
+ * By default, HBase only consider timestamp in versions. So a previous Delete with higher ts will
+ * mask a later Put with lower ts. Set this to true to enable new semantics of versions. We will
+ * also consider mvcc in versions. See HBASE-15968 for details.
*/
@Override
public boolean isNewVersionBehavior() {
@@ -468,13 +467,13 @@ public HColumnDescriptor setNewVersionBehavior(boolean newVersionBehavior) {
return this;
}
-
@Override
public int getTimeToLive() {
return delegatee.getTimeToLive();
}
/**
+ * Set the time to live of cell contents
* @param value Time-to-live of cell contents, in seconds.
* @return this (for chained invocation)
*/
@@ -484,8 +483,9 @@ public HColumnDescriptor setTimeToLive(int value) {
}
/**
+ * Set the time to live of cell contents
* @param value Time to live of cell contents, in human readable format
- * @see org.apache.hadoop.hbase.util.PrettyPrinter#format(String, Unit)
+ * @see org.apache.hadoop.hbase.util.PrettyPrinter#format(String, Unit)
* @return this (for chained invocation)
*/
public HColumnDescriptor setTimeToLive(String value) throws HBaseException {
@@ -499,8 +499,8 @@ public int getMinVersions() {
}
/**
- * @param value The minimum number of versions to keep.
- * (used when timeToLive is set)
+ * Set the minimum number of versions to keep.
+ * @param value The minimum number of versions to keep. (used when timeToLive is set)
* @return this (for chained invocation)
*/
public HColumnDescriptor setMinVersions(int value) {
@@ -514,8 +514,9 @@ public boolean isBlockCacheEnabled() {
}
/**
- * @param value True if hfile DATA type blocks should be cached (We always cache
- * INDEX and BLOOM blocks; you cannot turn this off).
+ * Set or clear the block cache enabled flag.
+ * @param value True if hfile DATA type blocks should be cached (We always cache INDEX and BLOOM
+ * blocks; you cannot turn this off).
* @return this (for chained invocation)
*/
public HColumnDescriptor setBlockCacheEnabled(boolean value) {
@@ -529,6 +530,7 @@ public BloomType getBloomFilterType() {
}
/**
+ * Set the bloom filter type.
* @param value bloom filter type
* @return this (for chained invocation)
*/
@@ -542,10 +544,6 @@ public int getScope() {
return delegatee.getScope();
}
- /**
- * @param value the scope tag
- * @return this (for chained invocation)
- */
public HColumnDescriptor setScope(int value) {
getDelegateeForModification().setScope(value);
return this;
@@ -557,6 +555,7 @@ public boolean isCacheDataOnWrite() {
}
/**
+ * Set or clear the cache data on write flag.
* @param value true if we should cache data blocks on write
* @return this (for chained invocation)
*/
@@ -566,8 +565,7 @@ public HColumnDescriptor setCacheDataOnWrite(boolean value) {
}
/**
- * This is a noop call from HBase 2.0 onwards
- *
+ * Set or clear the cache in L1 flag. This is a noop call from HBase 2.0 onwards
* @return this (for chained invocation)
* @deprecated Since 2.0 and will be removed in 3.0 with out any replacement. Caching data in on
* heap Cache, when there are both on heap LRU Cache and Bucket Cache will no longer
@@ -584,6 +582,7 @@ public boolean isCacheIndexesOnWrite() {
}
/**
+ * Set or clear the cache indexes on write flag.
* @param value true if we should cache index blocks on write
* @return this (for chained invocation)
*/
@@ -598,6 +597,7 @@ public boolean isCacheBloomsOnWrite() {
}
/**
+ * Set or clear the cache bloom filters on write flag.
* @param value true if we should cache bloomfilter blocks on write
* @return this (for chained invocation)
*/
@@ -612,8 +612,8 @@ public boolean isEvictBlocksOnClose() {
}
/**
- * @param value true if we should evict cached blocks from the blockcache on
- * close
+ * Set or clear the evict bloom filters on close flag.
+ * @param value true if we should evict cached blocks from the blockcache on close
* @return this (for chained invocation)
*/
public HColumnDescriptor setEvictBlocksOnClose(boolean value) {
@@ -627,6 +627,7 @@ public boolean isPrefetchBlocksOnOpen() {
}
/**
+ * Set or clear the prefetch on open flag.
* @param value true if we should prefetch blocks into the blockcache on open
* @return this (for chained invocation)
*/
@@ -635,17 +636,12 @@ public HColumnDescriptor setPrefetchBlocksOnOpen(boolean value) {
return this;
}
- /**
- * @see java.lang.Object#toString()
- */
@Override
public String toString() {
return delegatee.toString();
}
- /**
- * @return Column family descriptor with only the customized attributes.
- */
+ /** Returns Column family descriptor with only the customized attributes. */
@Override
public String toStringCustomizedValues() {
return delegatee.toStringCustomizedValues();
@@ -659,9 +655,6 @@ public static Map getDefaultValues() {
return ColumnFamilyDescriptorBuilder.getDefaultValues();
}
- /**
- * @see java.lang.Object#equals(java.lang.Object)
- */
@Override
public boolean equals(Object obj) {
if (this == obj) {
@@ -673,9 +666,6 @@ public boolean equals(Object obj) {
return false;
}
- /**
- * @see java.lang.Object#hashCode()
- */
@Override
public int hashCode() {
return delegatee.hashCode();
@@ -687,7 +677,7 @@ public int compareTo(HColumnDescriptor other) {
}
/**
- * @return This instance serialized with pb with pb magic prefix
+ * Returns This instance serialized with pb with pb magic prefix
* @see #parseFrom(byte[])
*/
public byte[] toByteArray() {
@@ -695,12 +685,12 @@ public byte[] toByteArray() {
}
/**
+ * Parse a serialized representation of a {@link HColumnDescriptor}
* @param bytes A pb serialized {@link HColumnDescriptor} instance with pb magic prefix
* @return An instance of {@link HColumnDescriptor} made from bytes
- * @throws DeserializationException
* @see #toByteArray()
*/
- public static HColumnDescriptor parseFrom(final byte [] bytes) throws DeserializationException {
+ public static HColumnDescriptor parseFrom(final byte[] bytes) throws DeserializationException {
ColumnFamilyDescriptor desc = ColumnFamilyDescriptorBuilder.parseFrom(bytes);
if (desc instanceof ModifyableColumnFamilyDescriptor) {
return new HColumnDescriptor((ModifyableColumnFamilyDescriptor) desc);
@@ -721,7 +711,7 @@ public Map getConfiguration() {
/**
* Setter for storing a configuration setting.
- * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
+ * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
* @param value String value. If null, removes the configuration.
*/
public HColumnDescriptor setConfiguration(String key, String value) {
@@ -743,7 +733,6 @@ public String getEncryptionType() {
/**
* Set the encryption algorithm for use with this family
- * @param value
*/
public HColumnDescriptor setEncryptionType(String value) {
getDelegateeForModification().setEncryptionType(value);
@@ -814,8 +803,8 @@ public short getDFSReplication() {
/**
* Set the replication factor to hfile(s) belonging to this family
* @param value number of replicas the blocks(s) belonging to this CF should have, or
- * {@link #DEFAULT_DFS_REPLICATION} for the default replication factor set in the
- * filesystem
+ * {@link #DEFAULT_DFS_REPLICATION} for the default replication factor set in the
+ * filesystem
* @return this (for chained invocation)
*/
public HColumnDescriptor setDFSReplication(short value) {
@@ -831,7 +820,7 @@ public String getStoragePolicy() {
/**
* Set the storage policy for use with this family
* @param value the policy to set, valid setting includes: "LAZY_PERSIST",
- * "ALL_SSD", "ONE_SSD", "HOT", "WARM", "COLD"
+ * "ALL_SSD", "ONE_SSD", "HOT", "WARM", "COLD"
*/
public HColumnDescriptor setStoragePolicy(String value) {
getDelegateeForModification().setStoragePolicy(value);
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
index 2a0e804ff7ab..33d7d98c61e0 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionInfo.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -24,7 +23,6 @@
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.KeyValue.KVComparator;
import org.apache.hadoop.hbase.client.RegionInfo;
@@ -38,40 +36,38 @@
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
/**
* Information about a region. A region is a range of keys in the whole keyspace of a table, an
- * identifier (a timestamp) for differentiating between subset ranges (after region split)
- * and a replicaId for differentiating the instance for the same range and some status information
- * about the region.
- *
- * The region has a unique name which consists of the following fields:
+ * identifier (a timestamp) for differentiating between subset ranges (after region split) and a
+ * replicaId for differentiating the instance for the same range and some status information about
+ * the region. The region has a unique name which consists of the following fields:
*
- * - tableName : The name of the table
- * - startKey : The startKey for the region.
- * - regionId : A timestamp when the region is created.
- * - 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.
- * - encodedName : An MD5 encoded string for the region name.
+ * - tableName : The name of the table
+ * - startKey : The startKey for the region.
+ * - regionId : A timestamp when the region is created.
+ * - 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.
+ * - encodedName : An MD5 encoded string for the region name.
*
- *
- *
Other than the fields in the region name, region info contains:
+ *
+ * Other than the fields in the region name, region info contains:
*
- * - endKey : the endKey for the region (exclusive)
- * - split : Whether the region is split
- * - offline : Whether the region is offline
+ * - endKey : the endKey for the region (exclusive)
+ * - split : Whether the region is split
+ * - offline : Whether the region is offline
*
- *
* 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.
* In 0.99+, a region can have multiple instances (called replicas), and thus a range (or row) can
* correspond to multiple HRegionInfo's. These HRI's share the same fields however except the
* replicaId field. If the replicaId is not set, it defaults to 0, which is compatible with the
* previous behavior of a range corresponding to 1 region.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * use {@link RegionInfoBuilder} to build {@link RegionInfo}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. use
+ * {@link RegionInfoBuilder} to build {@link RegionInfo}.
*/
@Deprecated
@InterfaceAudience.Public
@@ -79,46 +75,36 @@ public class HRegionInfo implements RegionInfo {
private static final Logger LOG = LoggerFactory.getLogger(HRegionInfo.class);
/**
- * The new format for a region name contains its encodedName at the end.
- * The encoded name also serves as the directory name for the region
- * in the filesystem.
- *
- * New region name format:
- * <tablename>,,<startkey>,<regionIdTimestamp>.<encodedName>.
- * where,
- * <encodedName> is a hex version of the MD5 hash of
- * <tablename>,<startkey>,<regionIdTimestamp>
- *
- * The old region name format:
- * <tablename>,<startkey>,<regionIdTimestamp>
- * For region names in the old format, the encoded name is a 32-bit
- * JenkinsHash integer value (in its decimal notation, string form).
- *
- * **NOTE**
- *
- * The first hbase:meta region, and regions created by an older
- * version of HBase (0.20 or prior) will continue to use the
- * old region name format.
+ * The new format for a region name contains its encodedName at the end. The encoded name also
+ * serves as the directory name for the region in the filesystem. New region name format:
+ * <tablename>,,<startkey>,<regionIdTimestamp>.<encodedName>. where, <encodedName>
+ * is a hex version of the MD5 hash of <tablename>,<startkey>,<regionIdTimestamp> The old
+ * region name format: <tablename>,<startkey>,<regionIdTimestamp> For region names in the
+ * old format, the encoded name is a 32-bit JenkinsHash integer value (in its decimal notation,
+ * string form).
+ *
+ * **NOTE** The first hbase:meta region, and regions created by an older version of HBase (0.20 or
+ * prior) will continue to use the old region name format.
*/
/** A non-capture group so that this can be embedded. */
- public static final String ENCODED_REGION_NAME_REGEX = RegionInfoBuilder.ENCODED_REGION_NAME_REGEX;
+ public static final String ENCODED_REGION_NAME_REGEX =
+ RegionInfoBuilder.ENCODED_REGION_NAME_REGEX;
private static final int MAX_REPLICA_ID = 0xFFFF;
/**
- * @param regionName
* @return the encodedName
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#encodeRegionName(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#encodeRegionName(byte[])}.
*/
@Deprecated
- public static String encodeRegionName(final byte [] regionName) {
+ public static String encodeRegionName(final byte[] regionName) {
return RegionInfo.encodeRegionName(regionName);
}
/**
- * @return Return a short, printable name for this region (usually encoded name) for us logging.
+ * Returns Return a short, printable name for this region (usually encoded name) for us logging.
*/
@Override
public String getShortNameToLog() {
@@ -126,19 +112,19 @@ public String getShortNameToLog() {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(RegionInfo...)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(RegionInfo...)}.
*/
@Deprecated
- public static String getShortNameToLog(HRegionInfo...hris) {
+ public static String getShortNameToLog(HRegionInfo... hris) {
return RegionInfo.getShortNameToLog(Arrays.asList(hris));
}
/**
- * @return Return a String of short, printable names for hris
- * (usually encoded name) for us logging.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(List)})}.
+ * @return Return a String of short, printable names for hris (usually encoded name)
+ * for us logging.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#getShortNameToLog(List)})}.
*/
@Deprecated
public static String getShortNameToLog(final List hris) {
@@ -149,9 +135,9 @@ public static String getShortNameToLog(final List hris) {
* Use logging.
* @param encodedRegionName The encoded regionname.
* @return hbase:meta if passed 1028785192 else returns
- * encodedRegionName
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#prettyPrint(String)}.
+ * encodedRegionName
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#prettyPrint(String)}.
*/
@Deprecated
@InterfaceAudience.Private
@@ -159,7 +145,7 @@ public static String prettyPrint(final String encodedRegionName) {
return RegionInfo.prettyPrint(encodedRegionName);
}
- private byte [] endKey = HConstants.EMPTY_BYTE_ARRAY;
+ private byte[] endKey = HConstants.EMPTY_BYTE_ARRAY;
// This flag is in the parent of a split while the parent is still referenced by daughter regions.
// We USED to set this flag when we disabled a table but now table state is kept up in zookeeper
// as of 0.90.0 HBase. And now in DisableTableProcedure, finally we will create bunch of
@@ -167,14 +153,14 @@ public static String prettyPrint(final String encodedRegionName) {
// will not change the offLine flag.
private boolean offLine = false;
private long regionId = -1;
- private transient byte [] regionName = HConstants.EMPTY_BYTE_ARRAY;
+ private transient byte[] regionName = HConstants.EMPTY_BYTE_ARRAY;
private boolean split = false;
- private byte [] startKey = HConstants.EMPTY_BYTE_ARRAY;
+ private byte[] startKey = HConstants.EMPTY_BYTE_ARRAY;
private int hashCode = -1;
- //TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
+ // TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final String NO_HASH = null;
private String encodedName = null;
- private byte [] encodedNameAsBytes = null;
+ private byte[] encodedNameAsBytes = null;
private int replicaId = DEFAULT_REPLICA_ID;
// Current TableName
@@ -188,7 +174,7 @@ public static String prettyPrint(final String encodedRegionName) {
/** HRegionInfo for first meta region */
// TODO: How come Meta regions still do not have encoded region names? Fix.
public static final HRegionInfo FIRST_META_REGIONINFO =
- new HRegionInfo(1L, TableName.META_TABLE_NAME);
+ new HRegionInfo(1L, TableName.META_TABLE_NAME);
private void setHashCode() {
int result = Arrays.hashCode(this.regionName);
@@ -202,8 +188,7 @@ private void setHashCode() {
}
/**
- * Private constructor used constructing HRegionInfo for the
- * first meta regions
+ * Private constructor used constructing HRegionInfo for the first meta regions
*/
private HRegionInfo(long regionId, TableName tableName) {
this(regionId, tableName, DEFAULT_REPLICA_ID);
@@ -225,73 +210,61 @@ public HRegionInfo(final TableName tableName) {
/**
* Construct HRegionInfo with explicit parameters
- *
* @param tableName the table name
- * @param startKey first key in region
- * @param endKey end of key range
- * @throws IllegalArgumentException
+ * @param startKey first key in region
+ * @param endKey end of key range
*/
public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey)
- throws IllegalArgumentException {
+ throws IllegalArgumentException {
this(tableName, startKey, endKey, false);
}
/**
* Construct HRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @throws IllegalArgumentException
+ * @param tableName the table name
+ * @param startKey first key in region
+ * @param endKey end of key range
+ * @param split true if this region has split and we have daughter regions regions that may or
+ * may not hold references to this region.
*/
public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
- final boolean split)
- throws IllegalArgumentException {
+ final boolean split) throws IllegalArgumentException {
this(tableName, startKey, endKey, split, EnvironmentEdgeManager.currentTime());
}
/**
* Construct HRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @param regionid Region id to use.
- * @throws IllegalArgumentException
+ * @param tableName the table name
+ * @param startKey first key in region
+ * @param endKey end of key range
+ * @param split true if this region has split and we have daughter regions regions that may or
+ * may not hold references to this region.
+ * @param regionId Region id to use.
*/
- public HRegionInfo(final TableName tableName, final byte[] startKey,
- final byte[] endKey, final boolean split, final long regionid)
- throws IllegalArgumentException {
- this(tableName, startKey, endKey, split, regionid, DEFAULT_REPLICA_ID);
+ public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
+ final boolean split, final long regionId) throws IllegalArgumentException {
+ this(tableName, startKey, endKey, split, regionId, DEFAULT_REPLICA_ID);
}
/**
* Construct HRegionInfo with explicit parameters
- *
- * @param tableName the table descriptor
- * @param startKey first key in region
- * @param endKey end of key range
- * @param split true if this region has split and we have daughter regions
- * regions that may or may not hold references to this region.
- * @param regionid Region id to use.
+ * @param tableName the table name
+ * @param startKey first key in region
+ * @param endKey end of key range
+ * @param split true if this region has split and we have daughter regions regions that may or
+ * may not hold references to this region.
+ * @param regionId Region id to use.
* @param replicaId the replicaId to use
- * @throws IllegalArgumentException
*/
- public HRegionInfo(final TableName tableName, final byte[] startKey,
- final byte[] endKey, final boolean split, final long regionid,
- final int replicaId)
- throws IllegalArgumentException {
+ public HRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
+ final boolean split, final long regionId, final int replicaId) throws IllegalArgumentException {
super();
if (tableName == null) {
throw new IllegalArgumentException("TableName cannot be null");
}
this.tableName = tableName;
this.offLine = false;
- this.regionId = regionid;
+ this.regionId = regionId;
this.replicaId = replicaId;
if (this.replicaId > MAX_REPLICA_ID) {
throw new IllegalArgumentException("ReplicaId cannot be greater than" + MAX_REPLICA_ID);
@@ -300,17 +273,14 @@ public HRegionInfo(final TableName tableName, final byte[] startKey,
this.regionName = createRegionName(this.tableName, startKey, regionId, replicaId, true);
this.split = split;
- this.endKey = endKey == null? HConstants.EMPTY_END_ROW: endKey.clone();
- this.startKey = startKey == null?
- HConstants.EMPTY_START_ROW: startKey.clone();
+ this.endKey = endKey == null ? HConstants.EMPTY_END_ROW : endKey.clone();
+ this.startKey = startKey == null ? HConstants.EMPTY_START_ROW : startKey.clone();
this.tableName = tableName;
setHashCode();
}
/**
- * Costruct a copy of another HRegionInfo
- *
- * @param other
+ * Construct a copy of another HRegionInfo
*/
public HRegionInfo(RegionInfo other) {
super();
@@ -334,92 +304,91 @@ public HRegionInfo(HRegionInfo other, int replicaId) {
/**
* Make a region name of passed parameters.
- * @param tableName
- * @param startKey Can be null
- * @param regionid Region id (Usually timestamp from when region was created).
- * @param newFormat should we create the region name in the new format
- * (such that it contains its encoded name?).
+ * @param tableName the table name
+ * @param startKey Can be null
+ * @param regionId Region id (Usually timestamp from when region was created).
+ * @param newFormat should we create the region name in the new format (such that it contains its
+ * encoded name?).
* @return Region name made of passed tableName, startKey and id
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#createRegionName(TableName, byte[], long, boolean)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#createRegionName(TableName, byte[], long, boolean)}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [] createRegionName(final TableName tableName,
- final byte [] startKey, final long regionid, boolean newFormat) {
- return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionid), newFormat);
+ public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
+ final long regionId, boolean newFormat) {
+ return RegionInfo.createRegionName(tableName, startKey, Long.toString(regionId), newFormat);
}
/**
* Make a region name of passed parameters.
- * @param tableName
- * @param startKey Can be null
- * @param id Region id (Usually timestamp from when region was created).
- * @param newFormat should we create the region name in the new format
- * (such that it contains its encoded name?).
+ * @param tableName the table name
+ * @param startKey Can be null
+ * @param id Region id (Usually timestamp from when region was created).
+ * @param newFormat should we create the region name in the new format (such that it contains its
+ * encoded name?).
* @return Region name made of passed tableName, startKey and id
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#createRegionName(TableName, byte[], String, boolean)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#createRegionName(TableName, byte[], String, boolean)}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [] createRegionName(final TableName tableName,
- final byte [] startKey, final String id, boolean newFormat) {
+ public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
+ final String id, boolean newFormat) {
return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(id), newFormat);
}
/**
* Make a region name of passed parameters.
- * @param tableName
- * @param startKey Can be null
- * @param regionid Region id (Usually timestamp from when region was created).
- * @param replicaId
- * @param newFormat should we create the region name in the new format
- * (such that it contains its encoded name?).
+ * @param tableName the table name
+ * @param startKey Can be null
+ * @param regionId Region id (Usually timestamp from when region was created).
+ * @param newFormat should we create the region name in the new format (such that it contains its
+ * encoded name?).
* @return Region name made of passed tableName, startKey, id and replicaId
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#createRegionName(TableName, byte[], long, int, boolean)}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [] createRegionName(final TableName tableName,
- final byte [] startKey, final long regionid, int replicaId, boolean newFormat) {
- return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionid)),
- replicaId, newFormat);
+ public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
+ final long regionId, int replicaId, boolean newFormat) {
+ return RegionInfo.createRegionName(tableName, startKey, Bytes.toBytes(Long.toString(regionId)),
+ replicaId, newFormat);
}
/**
* Make a region name of passed parameters.
- * @param tableName
- * @param startKey Can be null
- * @param id Region id (Usually timestamp from when region was created).
- * @param newFormat should we create the region name in the new format
- * (such that it contains its encoded name?).
+ * @param tableName the table name
+ * @param startKey Can be null
+ * @param id Region id (Usually timestamp from when region was created).
+ * @param newFormat should we create the region name in the new format (such that it contains its
+ * encoded name?).
* @return Region name made of passed tableName, startKey and id
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], boolean)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#createRegionName(TableName, byte[], byte[], boolean)}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [] createRegionName(final TableName tableName,
- final byte [] startKey, final byte [] id, boolean newFormat) {
+ public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
+ final byte[] id, boolean newFormat) {
return RegionInfo.createRegionName(tableName, startKey, id, DEFAULT_REPLICA_ID, newFormat);
}
+
/**
* Make a region name of passed parameters.
- * @param tableName
- * @param startKey Can be null
- * @param id Region id (Usually timestamp from when region was created).
- * @param replicaId
+ * @param tableName the table name
+ * @param startKey Can be null
+ * @param id Region id (Usually timestamp from when region was created)
* @param newFormat should we create the region name in the new format
* @return Region name made of passed tableName, startKey, id and replicaId
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#createRegionName(TableName, byte[], byte[], int, boolean)}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [] createRegionName(final TableName tableName,
- final byte [] startKey, final byte [] id, final int replicaId, boolean newFormat) {
+ public static byte[] createRegionName(final TableName tableName, final byte[] startKey,
+ final byte[] id, final int replicaId, boolean newFormat) {
return RegionInfo.createRegionName(tableName, startKey, id, replicaId, newFormat);
}
@@ -427,20 +396,19 @@ public HRegionInfo(HRegionInfo other, int replicaId) {
* Gets the table name from the specified region name.
* @param regionName to extract the table name from
* @return Table name
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#getTable(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#getTable(byte[])}.
*/
@Deprecated
- public static TableName getTable(final byte [] regionName) {
+ public static TableName getTable(final byte[] regionName) {
return RegionInfo.getTable(regionName);
}
/**
* Gets the start key from the specified region name.
- * @param regionName
* @return Start key.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#getStartKey(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#getStartKey(byte[])}.
*/
@Deprecated
public static byte[] getStartKey(final byte[] regionName) throws IOException {
@@ -449,34 +417,29 @@ public static byte[] getStartKey(final byte[] regionName) throws IOException {
/**
* Separate elements of a regionName.
- * @param regionName
* @return Array of byte[] containing tableName, startKey and id
- * @throws IOException
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#parseRegionName(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#parseRegionName(byte[])}.
*/
@Deprecated
@InterfaceAudience.Private
- public static byte [][] parseRegionName(final byte [] regionName) throws IOException {
+ public static byte[][] parseRegionName(final byte[] regionName) throws IOException {
return RegionInfo.parseRegionName(regionName);
}
/**
- *
- * @param regionName
* @return if region name is encoded.
- * @throws IOException
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#isEncodedRegionName(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#isEncodedRegionName(byte[])}.
*/
@Deprecated
public static boolean isEncodedRegionName(byte[] regionName) throws IOException {
return RegionInfo.isEncodedRegionName(regionName);
}
- /** @return the regionId */
+ /** Returns the regionId */
@Override
- public long getRegionId(){
+ public long getRegionId() {
return regionId;
}
@@ -485,13 +448,11 @@ public long getRegionId(){
* @see #getRegionNameAsString()
*/
@Override
- public byte [] getRegionName(){
+ public byte[] getRegionName() {
return regionName;
}
- /**
- * @return Region name as a String for use in logging, etc.
- */
+ /** Returns Region name as a String for use in logging, etc. */
@Override
public String getRegionNameAsString() {
if (RegionInfo.hasEncodedName(this.regionName)) {
@@ -505,9 +466,7 @@ public String getRegionNameAsString() {
return Bytes.toStringBinary(this.regionName) + "." + this.getEncodedName();
}
- /**
- * @return the encoded region name
- */
+ /** Returns the encoded region name */
@Override
public synchronized String getEncodedName() {
if (this.encodedName == null) {
@@ -517,37 +476,32 @@ public synchronized String getEncodedName() {
}
@Override
- public synchronized byte [] getEncodedNameAsBytes() {
+ public synchronized byte[] getEncodedNameAsBytes() {
if (this.encodedNameAsBytes == null) {
this.encodedNameAsBytes = Bytes.toBytes(getEncodedName());
}
return this.encodedNameAsBytes;
}
- /**
- * @return the startKey
- */
+ /** Returns the startKey */
@Override
- public byte [] getStartKey(){
+ public byte[] getStartKey() {
return startKey;
}
- /**
- * @return the endKey
- */
+ /** Returns the endKey */
@Override
- public byte [] getEndKey(){
+ public byte[] getEndKey() {
return endKey;
}
/**
* Get current table name of the region
- * @return TableName
*/
@Override
public TableName getTable() {
// This method name should be getTableName but there was already a method getTableName
- // that returned a byte array. It is unfortunate given everywhere else, getTableName returns
+ // that returned a byte array. It is unfortunate given everywhere else, getTableName returns
// a TableName instance.
if (tableName == null || tableName.getName().length == 0) {
tableName = getTable(getRegionName());
@@ -556,94 +510,77 @@ public TableName getTable() {
}
/**
- * Returns true if the given inclusive range of rows is fully contained
- * 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.
+ * Returns true if the given inclusive range of rows is fully contained 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)
*/
@Override
public boolean containsRange(byte[] rangeStartKey, byte[] rangeEndKey) {
if (Bytes.compareTo(rangeStartKey, rangeEndKey) > 0) {
- throw new IllegalArgumentException(
- "Invalid range: " + Bytes.toStringBinary(rangeStartKey) +
- " > " + Bytes.toStringBinary(rangeEndKey));
+ throw new IllegalArgumentException("Invalid range: " + Bytes.toStringBinary(rangeStartKey)
+ + " > " + Bytes.toStringBinary(rangeEndKey));
}
boolean firstKeyInRange = Bytes.compareTo(rangeStartKey, startKey) >= 0;
boolean lastKeyInRange =
- Bytes.compareTo(rangeEndKey, endKey) < 0 ||
- Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
+ Bytes.compareTo(rangeEndKey, endKey) < 0 || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY);
return firstKeyInRange && lastKeyInRange;
}
- /**
- * @return true if the given row falls in this region.
- */
+ /** Returns true if the given row falls in this region. */
@Override
public boolean containsRow(byte[] row) {
- return Bytes.compareTo(row, startKey) >= 0 &&
- (Bytes.compareTo(row, endKey) < 0 ||
- Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
+ return Bytes.compareTo(row, startKey) >= 0
+ && (Bytes.compareTo(row, endKey) < 0 || Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
- /**
- * @return true if this region is from hbase:meta
- */
+ /** Returns true if this region is from hbase:meta */
public boolean isMetaTable() {
return isMetaRegion();
}
- /**
- * @return true if this region is a meta region
- */
+ /** Returns true if this region is a meta region */
@Override
public boolean isMetaRegion() {
- return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
+ return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
}
- /**
- * @return true if this region is from a system table
- */
+ /** Returns true if this region is from a system table */
public boolean isSystemTable() {
return tableName.isSystemTable();
}
- /**
- * @return true if has been split and has daughters.
- */
+ /** Returns true if has been split and has daughters. */
@Override
public boolean isSplit() {
return this.split;
}
/**
+ * Set or clear the split status flag.
* @param split set split status
*/
public void setSplit(boolean split) {
this.split = split;
}
- /**
- * @return true if this region is offline.
- */
+ /** Returns true if this region is offline. */
@Override
public boolean isOffline() {
return this.offLine;
}
/**
- * The parent of a region split is offline while split daughters hold
- * references to the parent. Offlined regions are closed.
+ * The parent of a region split is offline while split daughters hold references to the parent.
+ * Offlined regions are closed.
* @param offLine Set online/offline status.
*/
public void setOffline(boolean offLine) {
this.offLine = offLine;
}
- /**
- * @return true if this is a split parent region.
- */
+ /** Returns true if this is a split parent region. */
@Override
public boolean isSplitParent() {
if (!isSplit()) return false;
@@ -667,14 +604,11 @@ public int getReplicaId() {
*/
@Override
public String toString() {
- return "{ENCODED => " + getEncodedName() + ", " +
- HConstants.NAME + " => '" + Bytes.toStringBinary(this.regionName)
- + "', STARTKEY => '" +
- Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" +
- Bytes.toStringBinary(this.endKey) + "'" +
- (isOffline()? ", OFFLINE => true": "") +
- (isSplit()? ", SPLIT => true": "") +
- ((replicaId > 0)? ", REPLICA_ID => " + replicaId : "") + "}";
+ return "{ENCODED => " + getEncodedName() + ", " + HConstants.NAME + " => '"
+ + Bytes.toStringBinary(this.regionName) + "', STARTKEY => '"
+ + Bytes.toStringBinary(this.startKey) + "', ENDKEY => '" + Bytes.toStringBinary(this.endKey)
+ + "'" + (isOffline() ? ", OFFLINE => true" : "") + (isSplit() ? ", SPLIT => true" : "")
+ + ((replicaId > 0) ? ", REPLICA_ID => " + replicaId : "") + "}";
}
/**
@@ -691,7 +625,7 @@ public boolean equals(Object o) {
if (!(o instanceof HRegionInfo)) {
return false;
}
- return this.compareTo((HRegionInfo)o) == 0;
+ return this.compareTo((HRegionInfo) o) == 0;
}
/**
@@ -704,17 +638,15 @@ public int hashCode() {
/**
* @return Comparator to use comparing {@link KeyValue}s.
- * @deprecated Use Region#getCellComparator(). deprecated for hbase 2.0, remove for hbase 3.0
+ * @deprecated Use Region#getCellComparator(). deprecated for hbase 2.0, remove for hbase 3.0
*/
@Deprecated
public KVComparator getComparator() {
- return isMetaRegion()?
- KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
+ return isMetaRegion() ? KeyValue.META_COMPARATOR : KeyValue.COMPARATOR;
}
/**
* Convert a HRegionInfo to the protobuf RegionInfo
- *
* @return the converted RegionInfo
*/
HBaseProtos.RegionInfo convert() {
@@ -723,12 +655,11 @@ HBaseProtos.RegionInfo convert() {
/**
* Convert a HRegionInfo to a RegionInfo
- *
* @param info the HRegionInfo to convert
* @return the converted RegionInfo
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo)
- * in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * toRegionInfo(org.apache.hadoop.hbase.client.RegionInfo) in
+ * org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
*/
@Deprecated
@InterfaceAudience.Private
@@ -738,12 +669,11 @@ public static HBaseProtos.RegionInfo convert(final HRegionInfo info) {
/**
* Convert a RegionInfo to a HRegionInfo
- *
* @param proto the RegionInfo to convert
* @return the converted HRegionInfo
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use toRegionInfo(HBaseProtos.RegionInfo)
- * in org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * toRegionInfo(HBaseProtos.RegionInfo) in
+ * org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.
*/
@Deprecated
@InterfaceAudience.Private
@@ -753,17 +683,12 @@ public static HRegionInfo convert(final HBaseProtos.RegionInfo proto) {
// RegionInfo into HRegionInfo which is what is wanted here.
HRegionInfo hri;
if (ri.isMetaRegion()) {
- hri = ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID ?
- HRegionInfo.FIRST_META_REGIONINFO :
- new HRegionInfo(ri.getRegionId(), ri.getTable(), ri.getReplicaId());
+ hri = ri.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID
+ ? HRegionInfo.FIRST_META_REGIONINFO
+ : new HRegionInfo(ri.getRegionId(), ri.getTable(), ri.getReplicaId());
} else {
- hri = new HRegionInfo(
- ri.getTable(),
- ri.getStartKey(),
- ri.getEndKey(),
- ri.isSplit(),
- ri.getRegionId(),
- ri.getReplicaId());
+ hri = new HRegionInfo(ri.getTable(), ri.getStartKey(), ri.getEndKey(), ri.isSplit(),
+ ri.getRegionId(), ri.getReplicaId());
if (proto.hasOffline()) {
hri.setOffline(proto.getOffline());
}
@@ -772,38 +697,41 @@ public static HRegionInfo convert(final HBaseProtos.RegionInfo proto) {
}
/**
+ * Serialize a {@link HRegionInfo} into a byte array.
* @return This instance serialized as protobuf w/ a magic pb prefix.
* @see #parseFrom(byte[])
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#toByteArray(RegionInfo)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#toByteArray(RegionInfo)}.
*/
@Deprecated
- public byte [] toByteArray() {
+ public byte[] toByteArray() {
return RegionInfo.toByteArray(this);
}
/**
- * @return A deserialized {@link HRegionInfo}
- * or null if we failed deserialize or passed bytes null
+ * Parse a serialized representation of a {@link HRegionInfo}.
+ * @return A deserialized {@link HRegionInfo} or null if we failed deserialize or passed bytes
+ * null
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[])}.
*/
@Deprecated
- public static HRegionInfo parseFromOrNull(final byte [] bytes) {
+ public static HRegionInfo parseFromOrNull(final byte[] bytes) {
if (bytes == null) return null;
return parseFromOrNull(bytes, 0, bytes.length);
}
/**
- * @return A deserialized {@link HRegionInfo} or null
- * if we failed deserialize or passed bytes null
+ * Parse a serialized representation of a {@link HRegionInfo}.
+ * @return A deserialized {@link HRegionInfo} or null if we failed deserialize or passed bytes
+ * null
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[], int, int)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#parseFromOrNull(byte[], int, int)}.
*/
@Deprecated
- public static HRegionInfo parseFromOrNull(final byte [] bytes, int offset, int len) {
+ public static HRegionInfo parseFromOrNull(final byte[] bytes, int offset, int len) {
if (bytes == null || len <= 0) return null;
try {
return parseFrom(bytes, offset, len);
@@ -813,31 +741,31 @@ public static HRegionInfo parseFromOrNull(final byte [] bytes, int offset, int l
}
/**
+ * Parse a serialized representation of a {@link HRegionInfo}.
* @param bytes A pb RegionInfo serialized with a pb magic prefix.
* @return A deserialized {@link HRegionInfo}
- * @throws DeserializationException
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[])}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[])}.
*/
- public static HRegionInfo parseFrom(final byte [] bytes) throws DeserializationException {
+ public static HRegionInfo parseFrom(final byte[] bytes) throws DeserializationException {
if (bytes == null) return null;
return parseFrom(bytes, 0, bytes.length);
}
/**
- * @param bytes A pb RegionInfo serialized with a pb magic prefix.
+ * Parse a serialized representation of a {@link HRegionInfo}.
+ * @param bytes A pb RegionInfo serialized with a pb magic prefix.
* @param offset starting point in the byte array
- * @param len length to read on the byte array
+ * @param len length to read on the byte array
* @return A deserialized {@link HRegionInfo}
- * @throws DeserializationException
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[], int, int)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#parseFrom(byte[], int, int)}.
*/
@Deprecated
- public static HRegionInfo parseFrom(final byte [] bytes, int offset, int len)
- throws DeserializationException {
+ public static HRegionInfo parseFrom(final byte[] bytes, int offset, int len)
+ throws DeserializationException {
if (ProtobufUtil.isPBMagicPrefix(bytes, offset, len)) {
int pblen = ProtobufUtil.lengthOfPBMagic();
try {
@@ -854,44 +782,38 @@ public static HRegionInfo parseFrom(final byte [] bytes, int offset, int len)
}
/**
- * Use this instead of {@link #toByteArray()} when writing to a stream and you want to use
- * the pb mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
+ * Use this instead of {@link #toByteArray()} when writing to a stream and you want to use the pb
+ * mergeDelimitedFrom (w/o the delimiter, pb reads to EOF which may not be what you want).
* @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
- * @throws IOException
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#toDelimitedByteArray(RegionInfo)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#toDelimitedByteArray(RegionInfo)}.
*/
@Deprecated
- public byte [] toDelimitedByteArray() throws IOException {
+ public byte[] toDelimitedByteArray() throws IOException {
return RegionInfo.toDelimitedByteArray(this);
}
/**
- * Get the descriptive name as {@link RegionState} does it but with hidden
- * startkey optionally
- * @param state
- * @param conf
+ * Get the descriptive name as {@link RegionState} does it but with hidden startkey optionally
* @return descriptive string
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use RegionInfoDisplay#getDescriptiveNameFromRegionStateForDisplay(RegionState, Configuration)
- * over in hbase-server module.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * RegionInfoDisplay#getDescriptiveNameFromRegionStateForDisplay(RegionState,
+ * Configuration) over in hbase-server module.
*/
@Deprecated
@InterfaceAudience.Private
public static String getDescriptiveNameFromRegionStateForDisplay(RegionState state,
- Configuration conf) {
+ Configuration conf) {
return RegionInfoDisplay.getDescriptiveNameFromRegionStateForDisplay(state, conf);
}
/**
* Get the end key for display. Optionally hide the real end key.
- * @param hri
- * @param conf
* @return the endkey
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use RegionInfoDisplay#getEndKeyForDisplay(RegionInfo, Configuration)
- * over in hbase-server module.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * RegionInfoDisplay#getEndKeyForDisplay(RegionInfo, Configuration) over in
+ * hbase-server module.
*/
@Deprecated
@InterfaceAudience.Private
@@ -901,12 +823,10 @@ public static byte[] getEndKeyForDisplay(HRegionInfo hri, Configuration conf) {
/**
* Get the start key for display. Optionally hide the real start key.
- * @param hri
- * @param conf
* @return the startkey
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use RegionInfoDisplay#getStartKeyForDisplay(RegionInfo, Configuration)
- * over in hbase-server module.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * RegionInfoDisplay#getStartKeyForDisplay(RegionInfo, Configuration) over in
+ * hbase-server module.
*/
@Deprecated
@InterfaceAudience.Private
@@ -916,12 +836,10 @@ public static byte[] getStartKeyForDisplay(HRegionInfo hri, Configuration conf)
/**
* Get the region name for display. Optionally hide the start key.
- * @param hri
- * @param conf
* @return region name as String
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use RegionInfoDisplay#getRegionNameAsStringForDisplay(RegionInfo, Configuration)
- * over in hbase-server module.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * RegionInfoDisplay#getRegionNameAsStringForDisplay(RegionInfo, Configuration) over
+ * in hbase-server module.
*/
@Deprecated
@InterfaceAudience.Private
@@ -931,12 +849,10 @@ public static String getRegionNameAsStringForDisplay(HRegionInfo hri, Configurat
/**
* Get the region name for display. Optionally hide the start key.
- * @param hri
- * @param conf
* @return region name bytes
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use RegionInfoDisplay#getRegionNameForDisplay(RegionInfo, Configuration)
- * over in hbase-server module.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * RegionInfoDisplay#getRegionNameForDisplay(RegionInfo, Configuration) over in
+ * hbase-server module.
*/
@Deprecated
@InterfaceAudience.Private
@@ -945,13 +861,11 @@ public static byte[] getRegionNameForDisplay(HRegionInfo hri, Configuration conf
}
/**
- * Parses an HRegionInfo instance from the passed in stream. Presumes the HRegionInfo was
+ * Parses an HRegionInfo instance from the passed in stream. Presumes the HRegionInfo was
* serialized to the stream with {@link #toDelimitedByteArray()}
- * @param in
* @return An instance of HRegionInfo.
- * @throws IOException
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#parseFrom(DataInputStream)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#parseFrom(DataInputStream)}.
*/
@Deprecated
@InterfaceAudience.Private
@@ -959,12 +873,12 @@ public static HRegionInfo parseFrom(final DataInputStream in) throws IOException
// I need to be able to move back in the stream if this is not a pb serialization so I can
// do the Writable decoding instead.
int pblen = ProtobufUtil.lengthOfPBMagic();
- byte [] pbuf = new byte[pblen];
- if (in.markSupported()) { //read it with mark()
+ byte[] pbuf = new byte[pblen];
+ if (in.markSupported()) { // read it with mark()
in.mark(pblen);
}
- //assumption: if Writable serialization, it should be longer than pblen.
+ // assumption: if Writable serialization, it should be longer than pblen.
in.readFully(pbuf, 0, pblen);
if (ProtobufUtil.isPBMagicPrefix(pbuf)) {
return convert(HBaseProtos.RegionInfo.parseDelimitedFrom(in));
@@ -976,14 +890,13 @@ public static HRegionInfo parseFrom(final DataInputStream in) throws IOException
/**
* Serializes given HRegionInfo's as a byte array. Use this instead of {@link #toByteArray()} when
* writing to a stream and you want to use the pb mergeDelimitedFrom (w/o the delimiter, pb reads
- * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can
- * be used to read back the instances.
+ * to EOF which may not be what you want). {@link #parseDelimitedFrom(byte[], int, int)} can be
+ * used to read back the instances.
* @param infos HRegionInfo objects to serialize
* @return This instance serialized as a delimited protobuf w/ a magic pb prefix.
- * @throws IOException
* @see #toByteArray()
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#toDelimitedByteArray(RegionInfo...)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#toDelimitedByteArray(RegionInfo...)}.
*/
@Deprecated
@InterfaceAudience.Private
@@ -994,16 +907,16 @@ public static byte[] toDelimitedByteArray(HRegionInfo... infos) throws IOExcepti
/**
* Parses all the HRegionInfo instances from the passed in stream until EOF. Presumes the
* HRegionInfo's were serialized to the stream with {@link #toDelimitedByteArray()}
- * @param bytes serialized bytes
+ * @param bytes serialized bytes
* @param offset the start offset into the byte[] buffer
* @param length how far we should read into the byte[] buffer
* @return All the hregioninfos that are in the byte array. Keeps reading till we hit the end.
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link RegionInfo#parseDelimitedFrom(byte[], int, int)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link RegionInfo#parseDelimitedFrom(byte[], int, int)}.
*/
@Deprecated
public static List parseDelimitedFrom(final byte[] bytes, final int offset,
- final int length) throws IOException {
+ final int length) throws IOException {
if (bytes == null) {
throw new IllegalArgumentException("Can't build an object with empty bytes array");
}
@@ -1023,11 +936,9 @@ public static List parseDelimitedFrom(final byte[] bytes, final int
/**
* Check whether two regions are adjacent
- * @param regionA
- * @param regionB
* @return true if two regions are adjacent
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * Use {@link org.apache.hadoop.hbase.client.RegionInfo#areAdjacent(RegionInfo, RegionInfo)}.
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 Use
+ * {@link org.apache.hadoop.hbase.client.RegionInfo#areAdjacent(RegionInfo, RegionInfo)}.
*/
@Deprecated
public static boolean areAdjacent(HRegionInfo regionA, HRegionInfo regionB) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
index fd679bd0cbc4..3180baa17a60 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HRegionLocation.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -24,17 +23,13 @@
import org.apache.yetus.audience.InterfaceAudience;
/**
- * Data structure to hold RegionInfo and the address for the hosting
- * HRegionServer. Immutable. Comparable, but we compare the 'location' only:
- * i.e. the hostname and port, and *not* the regioninfo. This means two
- * instances are the same if they refer to the same 'location' (the same
- * hostname and port), though they may be carrying different regions.
- *
- * On a big cluster, each client will have thousands of instances of this object, often
- * 100 000 of them if not million. It's important to keep the object size as small
- * as possible.
- *
- *
This interface has been marked InterfaceAudience.Public in 0.96 and 0.98.
+ * Data structure to hold RegionInfo and the address for the hosting HRegionServer. Immutable.
+ * Comparable, but we compare the 'location' only: i.e. the hostname and port, and *not* the
+ * regioninfo. This means two instances are the same if they refer to the same 'location' (the same
+ * hostname and port), though they may be carrying different regions. On a big cluster, each client
+ * will have thousands of instances of this object, often 100 000 of them if not million. It's
+ * important to keep the object size as small as possible.
+ * This interface has been marked InterfaceAudience.Public in 0.96 and 0.98.
*/
@InterfaceAudience.Public
public class HRegionLocation implements Comparable {
@@ -58,7 +53,7 @@ public HRegionLocation(RegionInfo regionInfo, ServerName serverName, long seqNum
@Override
public String toString() {
return "region=" + (this.regionInfo == null ? "null" : this.regionInfo.getRegionNameAsString())
- + ", hostname=" + this.serverName + ", seqNum=" + seqNum;
+ + ", hostname=" + this.serverName + ", seqNum=" + seqNum;
}
/**
@@ -75,7 +70,7 @@ public boolean equals(Object o) {
if (!(o instanceof HRegionLocation)) {
return false;
}
- return this.compareTo((HRegionLocation)o) == 0;
+ return this.compareTo((HRegionLocation) o) == 0;
}
/**
@@ -87,19 +82,16 @@ public int hashCode() {
}
/**
- *
- * @return Immutable HRegionInfo
+ * Returns immutable HRegionInfo
* @deprecated Since 2.0.0. Will remove in 3.0.0. Use {@link #getRegion()}} instead.
*/
@Deprecated
- public HRegionInfo getRegionInfo(){
+ public HRegionInfo getRegionInfo() {
return regionInfo == null ? null : new ImmutableHRegionInfo(regionInfo);
}
- /**
- * @return regionInfo
- */
- public RegionInfo getRegion(){
+ /** Returns regionInfo */
+ public RegionInfo getRegion() {
return regionInfo;
}
@@ -116,8 +108,8 @@ public long getSeqNum() {
}
/**
- * @return String made of hostname and port formatted as
- * per {@link Addressing#createHostAndPortStr(String, int)}
+ * Returns String made of hostname and port formatted as per
+ * {@link Addressing#createHostAndPortStr(String, int)}
*/
public String getHostnamePort() {
return Addressing.createHostAndPortStr(this.getHostname(), this.getPort());
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
index 8f9e77ac6488..808cb5a40606 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -42,12 +41,12 @@
import org.apache.yetus.audience.InterfaceAudience;
/**
- * HTableDescriptor contains the details about an HBase table such as the descriptors of
- * all the column families, is the table a catalog table, hbase:meta ,
- * if the table is read only, the maximum size of the memstore,
- * when the region split should occur, coprocessors associated with it etc...
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
+ * HTableDescriptor contains the details about an HBase table such as the descriptors of all the
+ * column families, is the table a catalog table, hbase:meta , if the table is read
+ * only, the maximum size of the memstore, when the region split should occur, coprocessors
+ * associated with it etc...
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
+ * {@link TableDescriptorBuilder} to build {@link HTableDescriptor}.
*/
@Deprecated
@InterfaceAudience.Public
@@ -66,26 +65,34 @@ public class HTableDescriptor implements TableDescriptor, ComparableHADOOP-1581 HBASE: (HBASE-174) Un-openable tablename bug
+ * @see HADOOP-1581 HBASE: (HBASE-174)
+ * Un-openable tablename bug
*/
public HTableDescriptor(final TableName name) {
this(new ModifyableTableDescriptor(name));
@@ -94,8 +101,8 @@ public HTableDescriptor(final TableName name) {
/**
* Construct a table descriptor by cloning the descriptor passed as a parameter.
*
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
+ * Makes a deep copy of the supplied descriptor. Can make a modifiable descriptor from an
+ * ImmutableHTableDescriptor.
* @param desc The descriptor.
*/
public HTableDescriptor(final HTableDescriptor desc) {
@@ -103,8 +110,7 @@ public HTableDescriptor(final HTableDescriptor desc) {
}
protected HTableDescriptor(final HTableDescriptor desc, boolean deepClone) {
- this(deepClone ? new ModifyableTableDescriptor(desc.getTableName(), desc)
- : desc.delegatee);
+ this(deepClone ? new ModifyableTableDescriptor(desc.getTableName(), desc) : desc.delegatee);
}
public HTableDescriptor(final TableDescriptor desc) {
@@ -112,11 +118,11 @@ public HTableDescriptor(final TableDescriptor desc) {
}
/**
- * Construct a table descriptor by cloning the descriptor passed as a parameter
- * but using a different table name.
+ * Construct a table descriptor by cloning the descriptor passed as a parameter but using a
+ * different table name.
*
- * Makes a deep copy of the supplied descriptor.
- * Can make a modifiable descriptor from an ImmutableHTableDescriptor.
+ * Makes a deep copy of the supplied descriptor. Can make a modifiable descriptor from an
+ * ImmutableHTableDescriptor.
* @param name Table name.
* @param desc The descriptor.
*/
@@ -130,7 +136,6 @@ protected HTableDescriptor(ModifyableTableDescriptor delegatee) {
/**
* This is vestigial API. It will be removed in 3.0.
- *
* @return always return the false
*/
public boolean isRootRegion() {
@@ -138,11 +143,8 @@ public boolean isRootRegion() {
}
/**
- * Checks if this table is hbase:meta
- * region.
- *
- * @return true if this table is hbase:meta
- * region
+ * Checks if this table is hbase:meta region.
+ * @return true if this table is hbase:meta region
*/
@Override
public boolean isMetaRegion() {
@@ -151,7 +153,6 @@ public boolean isMetaRegion() {
/**
* Checks if the table is a hbase:meta table
- *
* @return true if table is hbase:meta region.
*/
@Override
@@ -159,9 +160,7 @@ public boolean isMetaTable() {
return delegatee.isMetaTable();
}
- /**
- * @return Getter for fetching an unmodifiable map.
- */
+ /** Returns Getter for fetching an unmodifiable map. */
@Override
public Map getValues() {
return delegatee.getValues();
@@ -169,8 +168,7 @@ public Map getValues() {
/**
* Setter for storing metadata as a (key, value) pair in map
- *
- * @param key The key.
+ * @param key The key.
* @param value The value. If null, removes the setting.
*/
public HTableDescriptor setValue(byte[] key, byte[] value) {
@@ -180,7 +178,6 @@ public HTableDescriptor setValue(byte[] key, byte[] value) {
/*
* Setter for storing metadata as a (key, value) pair in map
- *
* @param key The key.
* @param value The value. If null, removes the setting.
*/
@@ -191,8 +188,7 @@ public HTableDescriptor setValue(final Bytes key, final Bytes value) {
/**
* Setter for storing metadata as a (key, value) pair in map
- *
- * @param key The key.
+ * @param key The key.
* @param value The value. If null, removes the setting.
*/
public HTableDescriptor setValue(String key, String value) {
@@ -202,9 +198,7 @@ public HTableDescriptor setValue(String key, String value) {
/**
* Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
+ * @param key Key whose key and value we're to remove from HTableDescriptor parameters.
*/
public void remove(final String key) {
getDelegateeForModification().removeValue(Bytes.toBytes(key));
@@ -212,9 +206,7 @@ public void remove(final String key) {
/**
* Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
+ * @param key Key whose key and value we're to remove from HTableDescriptor parameters.
*/
public void remove(Bytes key) {
getDelegateeForModification().removeValue(key);
@@ -222,18 +214,15 @@ public void remove(Bytes key) {
/**
* Remove metadata represented by the key from the map
- *
- * @param key Key whose key and value we're to remove from HTableDescriptor
- * parameters.
+ * @param key Key whose key and value we're to remove from HTableDescriptor parameters.
*/
- public void remove(final byte [] key) {
+ public void remove(final byte[] key) {
getDelegateeForModification().removeValue(key);
}
/**
- * Check if the readOnly flag of the table is set. If the readOnly flag is
- * set then the contents of the table can only be read from but not modified.
- *
+ * Check if the readOnly flag of the table is set. If the readOnly flag is set then the contents
+ * of the table can only be read from but not modified.
* @return true if all columns in the table should be read only
*/
@Override
@@ -242,12 +231,10 @@ public boolean isReadOnly() {
}
/**
- * Setting the table as read only sets all the columns in the table as read
- * only. By default all tables are modifiable, but if the readOnly flag is
- * set to true then the contents of the table can only be read but not modified.
- *
- * @param readOnly True if all of the columns in the table should be read
- * only.
+ * Setting the table as read only sets all the columns in the table as read only. By default all
+ * tables are modifiable, but if the readOnly flag is set to true then the contents of the table
+ * can only be read but not modified.
+ * @param readOnly True if all of the columns in the table should be read only.
*/
public HTableDescriptor setReadOnly(final boolean readOnly) {
getDelegateeForModification().setReadOnly(readOnly);
@@ -255,9 +242,8 @@ public HTableDescriptor setReadOnly(final boolean readOnly) {
}
/**
- * Check if the compaction enable flag of the table is true. If flag is
- * false then no minor/major compactions will be done in real.
- *
+ * Check if the compaction enable flag of the table is true. If flag is false then no minor/major
+ * compactions will be done in real.
* @return true if table compaction enabled
*/
@Override
@@ -267,7 +253,6 @@ public boolean isCompactionEnabled() {
/**
* Setting the table compaction enable flag.
- *
* @param isEnable True if enable compaction.
*/
public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
@@ -276,9 +261,8 @@ public HTableDescriptor setCompactionEnabled(final boolean isEnable) {
}
/**
- * Check if the region split enable flag of the table is true. If flag is
- * false then no split will be done.
- *
+ * Check if the region split enable flag of the table is true. If flag is false then no split will
+ * be done.
* @return true if table region split enabled
*/
@Override
@@ -288,7 +272,6 @@ public boolean isSplitEnabled() {
/**
* Setting the table region split enable flag.
- *
* @param isEnable True if enable split.
*/
public HTableDescriptor setSplitEnabled(final boolean isEnable) {
@@ -296,11 +279,9 @@ public HTableDescriptor setSplitEnabled(final boolean isEnable) {
return this;
}
-
/**
- * Check if the region merge enable flag of the table is true. If flag is
- * false then no merge will be done.
- *
+ * Check if the region merge enable flag of the table is true. If flag is false then no merge will
+ * be done.
* @return true if table region merge enabled
*/
@Override
@@ -310,7 +291,6 @@ public boolean isMergeEnabled() {
/**
* Setting the table region merge enable flag.
- *
* @param isEnable True if enable merge.
*/
public HTableDescriptor setMergeEnabled(final boolean isEnable) {
@@ -319,9 +299,8 @@ public HTableDescriptor setMergeEnabled(final boolean isEnable) {
}
/**
- * Check if normalization enable flag of the table is true. If flag is
- * false then no region normalizer won't attempt to normalize this table.
- *
+ * Check if normalization enable flag of the table is true. If flag is false then no region
+ * normalizer won't attempt to normalize this table.
* @return true if region normalization is enabled for this table
*/
@Override
@@ -331,7 +310,6 @@ public boolean isNormalizationEnabled() {
/**
* Setting the table normalization enable flag.
- *
* @param isEnable True if enable normalization.
*/
public HTableDescriptor setNormalizationEnabled(final boolean isEnable) {
@@ -379,8 +357,6 @@ public Durability getDurability() {
/**
* Get the name of the table
- *
- * @return TableName
*/
@Override
public TableName getTableName() {
@@ -389,7 +365,6 @@ public TableName getTableName() {
/**
* Get the name of the table as a String
- *
* @return name of table as a String
*/
public String getNameAsString() {
@@ -397,9 +372,9 @@ public String getNameAsString() {
}
/**
- * This sets the class associated with the region split policy which
- * determines when a region split should occur. The class used by
- * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+ * This sets the class associated with the region split policy which determines when a region
+ * split should occur. The class used by default is defined in
+ * org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
* @param clazz the class name
*/
public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
@@ -408,46 +383,40 @@ public HTableDescriptor setRegionSplitPolicyClassName(String clazz) {
}
/**
- * This gets the class associated with the region split policy which
- * determines when a region split should occur. The class used by
- * default is defined in org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
- *
- * @return the class name of the region split policy for this table.
- * If this returns null, the default split policy is used.
+ * This gets the class associated with the region split policy which determines when a region
+ * split should occur. The class used by default is defined in
+ * org.apache.hadoop.hbase.regionserver.RegionSplitPolicy
+ * @return the class name of the region split policy for this table. If this returns null, the
+ * default split policy is used.
*/
@Override
- public String getRegionSplitPolicyClassName() {
+ public String getRegionSplitPolicyClassName() {
return delegatee.getRegionSplitPolicyClassName();
}
/**
- * Returns the maximum size upto which a region can grow to after which a region
- * split is triggered. The region size is represented by the size of the biggest
- * store file in that region.
- *
+ * Returns the maximum size upto which a region can grow to after which a region split is
+ * triggered. The region size is represented by the size of the biggest store file in that region.
* @return max hregion size for table, -1 if not set.
- *
* @see #setMaxFileSize(long)
*/
- @Override
+ @Override
public long getMaxFileSize() {
return delegatee.getMaxFileSize();
}
/**
- * Sets the maximum size upto which a region can grow to after which a region
- * split is triggered. The region size is represented by the size of the biggest
- * store file in that region, i.e. If the biggest store file grows beyond the
- * maxFileSize, then the region split is triggered. This defaults to a value of
- * 256 MB.
+ * Sets the maximum size upto which a region can grow to after which a region split is triggered.
+ * The region size is represented by the size of the biggest store file in that region, i.e. If
+ * the biggest store file grows beyond the maxFileSize, then the region split is triggered. This
+ * defaults to a value of 256 MB.
*
- * This is not an absolute value and might vary. Assume that a single row exceeds
- * the maxFileSize then the storeFileSize will be greater than maxFileSize since
- * a single row cannot be split across multiple regions
+ * This is not an absolute value and might vary. Assume that a single row exceeds the maxFileSize
+ * then the storeFileSize will be greater than maxFileSize since a single row cannot be split
+ * across multiple regions
*
- *
- * @param maxFileSize The maximum file size that a store file can grow to
- * before a split is triggered.
+ * @param maxFileSize The maximum file size that a store file can grow to before a split is
+ * triggered.
*/
public HTableDescriptor setMaxFileSize(long maxFileSize) {
getDelegateeForModification().setMaxFileSize(maxFileSize);
@@ -461,9 +430,7 @@ public HTableDescriptor setMaxFileSize(String maxFileSize) throws HBaseException
/**
* Returns the size of the memstore after which a flush to filesystem is triggered.
- *
* @return memory cache flush size for each hregion, -1 if not set.
- *
* @see #setMemStoreFlushSize(long)
*/
@Override
@@ -472,9 +439,8 @@ public long getMemStoreFlushSize() {
}
/**
- * Represents the maximum size of the memstore after which the contents of the
- * memstore are flushed to the filesystem. This defaults to a size of 64 MB.
- *
+ * Represents the maximum size of the memstore after which the contents of the memstore are
+ * flushed to the filesystem. This defaults to a size of 64 MB.
* @param memstoreFlushSize memory cache flush size for each hregion
*/
public HTableDescriptor setMemStoreFlushSize(long memstoreFlushSize) {
@@ -511,8 +477,8 @@ public String getFlushPolicyClassName() {
}
/**
- * Adds a column family.
- * For the updating purpose please use {@link #modifyFamily(HColumnDescriptor)} instead.
+ * Adds a column family. For the updating purpose please use
+ * {@link #modifyFamily(HColumnDescriptor)} instead.
* @param family HColumnDescriptor of family to add.
*/
public HTableDescriptor addFamily(final HColumnDescriptor family) {
@@ -535,13 +501,12 @@ public HTableDescriptor modifyFamily(final HColumnDescriptor family) {
* @param familyName Family name or column name.
* @return true if the table contains the specified family name
*/
- public boolean hasFamily(final byte [] familyName) {
+ public boolean hasFamily(final byte[] familyName) {
return delegatee.hasColumnFamily(familyName);
}
/**
- * @return Name of this table and then a map of all of the column family
- * descriptors.
+ * @return Name of this table and then a map of all of the column family descriptors.
* @see #getNameAsString()
*/
@Override
@@ -550,28 +515,24 @@ public String toString() {
}
/**
- * @return Name of this table and then a map of all of the column family
- * descriptors (with only the non-default column family attributes)
+ * @return Name of this table and then a map of all of the column family descriptors (with only
+ * the non-default column family attributes)
*/
@Override
public String toStringCustomizedValues() {
return delegatee.toStringCustomizedValues();
}
- /**
- * @return map of all table attributes formatted into string.
- */
+ /** Returns map of all table attributes formatted into string. */
public String toStringTableAttributes() {
- return delegatee.toStringTableAttributes();
+ return delegatee.toStringTableAttributes();
}
/**
- * Compare the contents of the descriptor with another one passed as a parameter.
- * Checks if the obj passed is an instance of HTableDescriptor, if yes then the
- * contents of the descriptors are compared.
- *
+ * Compare the contents of the descriptor with another one passed as a parameter. Checks if the
+ * obj passed is an instance of HTableDescriptor, if yes then the contents of the descriptors are
+ * compared.
* @return true if the contents of the the two descriptors exactly match
- *
* @see java.lang.Object#equals(java.lang.Object)
*/
@Override
@@ -596,11 +557,10 @@ public int hashCode() {
// Comparable
/**
- * Compares the descriptor with another descriptor which is passed as a parameter.
- * This compares the content of the two descriptors and not the reference.
- *
- * @return 0 if the contents of the descriptors are exactly matching,
- * 1 if there is a mismatch in the contents
+ * Compares the descriptor with another descriptor which is passed as a parameter. This compares
+ * the content of the two descriptors and not the reference.
+ * @return 0 if the contents of the descriptors are exactly matching, 1 if there is a mismatch in
+ * the contents
*/
@Override
public int compareTo(final HTableDescriptor other) {
@@ -608,19 +568,17 @@ public int compareTo(final HTableDescriptor other) {
}
/**
- * Returns an unmodifiable collection of all the {@link HColumnDescriptor}
- * of all the column families of the table.
+ * Returns an unmodifiable collection of all the {@link HColumnDescriptor} of all the column
+ * families of the table.
* @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamilies()} instead.
- * @return Immutable collection of {@link HColumnDescriptor} of all the
- * column families.
+ * @return Immutable collection of {@link HColumnDescriptor} of all the column families.
* @see #getColumnFamilies()
* @see HBASE-18008
*/
@Deprecated
public Collection getFamilies() {
- return Stream.of(delegatee.getColumnFamilies())
- .map(this::toHColumnDescriptor)
- .collect(Collectors.toList());
+ return Stream.of(delegatee.getColumnFamilies()).map(this::toHColumnDescriptor)
+ .collect(Collectors.toList());
}
/**
@@ -641,25 +599,23 @@ public HTableDescriptor setRegionReplication(int regionReplication) {
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link #hasRegionMemStoreReplication()} instead
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
+ * {@link #hasRegionMemStoreReplication()} instead
*/
@Deprecated
public boolean hasRegionMemstoreReplication() {
return hasRegionMemStoreReplication();
}
- /**
- * @return true if the read-replicas memstore replication is enabled.
- */
+ /** Returns true if the read-replicas memstore replication is enabled. */
@Override
public boolean hasRegionMemStoreReplication() {
return delegatee.hasRegionMemStoreReplication();
}
/**
- * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
- * Use {@link #setRegionMemStoreReplication(boolean)} instead
+ * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use
+ * {@link #setRegionMemStoreReplication(boolean)} instead
*/
@Deprecated
public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
@@ -667,13 +623,11 @@ public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication
}
/**
- * Enable or Disable the memstore replication from the primary region to the replicas.
- * The replication will be used only for meta operations (e.g. flush, compaction, ...)
- *
- * @param memstoreReplication true if the new data written to the primary region
- * should be replicated.
- * false if the secondaries can tollerate to have new
- * data only when the primary flushes the memstore.
+ * Enable or Disable the memstore replication from the primary region to the replicas. The
+ * replication will be used only for meta operations (e.g. flush, compaction, ...)
+ * @param memstoreReplication true if the new data written to the primary region should be
+ * replicated. false if the secondaries can tollerate to have new data
+ * only when the primary flushes the memstore.
*/
public HTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
getDelegateeForModification().setRegionMemStoreReplication(memstoreReplication);
@@ -691,15 +645,13 @@ public int getPriority() {
}
/**
- * Returns all the column family names of the current table. The map of
- * HTableDescriptor contains mapping of family name to HColumnDescriptors.
- * This returns all the keys of the family map which represents the column
- * family names of the table.
- *
+ * Returns all the column family names of the current table. The map of HTableDescriptor contains
+ * mapping of family name to HColumnDescriptors. This returns all the keys of the family map which
+ * represents the column family names of the table.
* @return Immutable sorted set of the keys of the families.
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
- * (HBASE-18008).
- * Use {@link #getColumnFamilyNames()}.
+ * (HBASE-18008). Use
+ * {@link #getColumnFamilyNames()}.
*/
@Deprecated
public Set getFamiliesKeys() {
@@ -708,7 +660,6 @@ public Set getFamiliesKeys() {
/**
* Returns the count of the column families of the table.
- *
* @return Count of column families of the table
*/
@Override
@@ -717,9 +668,7 @@ public int getColumnFamilyCount() {
}
/**
- * Returns an array all the {@link HColumnDescriptor} of the column families
- * of the table.
- *
+ * Returns an array all the {@link HColumnDescriptor} of the column families of the table.
* @return Array of all the HColumnDescriptors of the current table
* @deprecated since 2.0.0 and will be removed in 3.0.0.
* @see #getFamilies()
@@ -728,19 +677,17 @@ public int getColumnFamilyCount() {
@Deprecated
@Override
public HColumnDescriptor[] getColumnFamilies() {
- return Stream.of(delegatee.getColumnFamilies())
- .map(this::toHColumnDescriptor)
- .toArray(size -> new HColumnDescriptor[size]);
+ return Stream.of(delegatee.getColumnFamilies()).map(this::toHColumnDescriptor)
+ .toArray(size -> new HColumnDescriptor[size]);
}
/**
- * Returns the HColumnDescriptor for a specific column family with name as
- * specified by the parameter column.
+ * Returns the HColumnDescriptor for a specific column family with name as specified by the
+ * parameter column.
* @param column Column family name
- * @return Column descriptor for the passed family name or the family on
- * passed in column.
+ * @return Column descriptor for the passed family name or the family on passed in column.
* @deprecated since 2.0.0 and will be removed in 3.0.0. Use {@link #getColumnFamily(byte[])}
- * instead.
+ * instead.
* @see #getColumnFamily(byte[])
* @see HBASE-18008
*/
@@ -749,16 +696,13 @@ public HColumnDescriptor getFamily(final byte[] column) {
return toHColumnDescriptor(delegatee.getColumnFamily(column));
}
-
/**
- * Removes the HColumnDescriptor with name specified by the parameter column
- * from the table descriptor
- *
+ * Removes the HColumnDescriptor with name specified by the parameter column from the table
+ * descriptor
* @param column Name of the column family to be removed.
- * @return Column descriptor for the passed family name or the family on
- * passed in column.
+ * @return Column descriptor for the passed family name or the family on passed in column.
*/
- public HColumnDescriptor removeFamily(final byte [] column) {
+ public HColumnDescriptor removeFamily(final byte[] column) {
return toHColumnDescriptor(getDelegateeForModification().removeColumnFamily(column));
}
@@ -780,13 +724,11 @@ protected HColumnDescriptor toHColumnDescriptor(ColumnFamilyDescriptor desc) {
}
/**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
+ * Add a table coprocessor to this table. The coprocessor type must be
+ * org.apache.hadoop.hbase.coprocessor.RegionCoprocessor. It won't check if the class can be
+ * loaded or not. Whether a coprocessor is loadable or not will be determined when a region is
+ * opened.
* @param className Full class name.
- * @throws IOException
*/
public HTableDescriptor addCoprocessor(String className) throws IOException {
getDelegateeForModification().setCoprocessor(className);
@@ -794,39 +736,31 @@ public HTableDescriptor addCoprocessor(String className) throws IOException {
}
/**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
- * @param jarFilePath Path of the jar file. If it's null, the class will be
- * loaded from default classloader.
- * @param className Full class name.
- * @param priority Priority
- * @param kvs Arbitrary key-value parameter pairs passed into the coprocessor.
- * @throws IOException
- */
- public HTableDescriptor addCoprocessor(String className, Path jarFilePath,
- int priority, final Map kvs)
- throws IOException {
- getDelegateeForModification().setCoprocessor(
- CoprocessorDescriptorBuilder.newBuilder(className)
- .setJarPath(jarFilePath == null ? null : jarFilePath.toString())
- .setPriority(priority)
- .setProperties(kvs == null ? Collections.emptyMap() : kvs)
- .build());
+ * Add a table coprocessor to this table. The coprocessor type must be
+ * org.apache.hadoop.hbase.coprocessor.RegionCoprocessor. It won't check if the class can be
+ * loaded or not. Whether a coprocessor is loadable or not will be determined when a region is
+ * opened.
+ * @param jarFilePath Path of the jar file. If it's null, the class will be loaded from default
+ * classloader.
+ * @param className Full class name.
+ * @param priority Priority
+ * @param kvs Arbitrary key-value parameter pairs passed into the coprocessor.
+ */
+ public HTableDescriptor addCoprocessor(String className, Path jarFilePath, int priority,
+ final Map kvs) throws IOException {
+ getDelegateeForModification().setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(className)
+ .setJarPath(jarFilePath == null ? null : jarFilePath.toString()).setPriority(priority)
+ .setProperties(kvs == null ? Collections.emptyMap() : kvs).build());
return this;
}
/**
- * Add a table coprocessor to this table. The coprocessor
- * type must be org.apache.hadoop.hbase.coprocessor.RegionCoprocessor.
- * It won't check if the class can be loaded or not.
- * Whether a coprocessor is loadable or not will be determined when
- * a region is opened.
+ * Add a table coprocessor to this table. The coprocessor type must be
+ * org.apache.hadoop.hbase.coprocessor.RegionCoprocessor. It won't check if the class can be
+ * loaded or not. Whether a coprocessor is loadable or not will be determined when a region is
+ * opened.
* @param specStr The Coprocessor specification all in in one String formatted so matches
- * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
- * @throws IOException
+ * {@link HConstants#CP_HTD_ATTR_VALUE_PATTERN}
*/
public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOException {
getDelegateeForModification().setCoprocessorWithSpec(specStr);
@@ -835,7 +769,6 @@ public HTableDescriptor addCoprocessorWithSpec(final String specStr) throws IOEx
/**
* Check if the table has an attached co-processor represented by the name className
- *
* @param classNameToMatch - Class name of the co-processor
* @return true of the table has a co-processor className
*/
@@ -858,14 +791,17 @@ public void removeCoprocessor(String className) {
}
public final static String NAMESPACE_FAMILY_INFO = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO;
- public final static byte[] NAMESPACE_FAMILY_INFO_BYTES = TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
- public final static byte[] NAMESPACE_COL_DESC_BYTES = TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
+ public final static byte[] NAMESPACE_FAMILY_INFO_BYTES =
+ TableDescriptorBuilder.NAMESPACE_FAMILY_INFO_BYTES;
+ public final static byte[] NAMESPACE_COL_DESC_BYTES =
+ TableDescriptorBuilder.NAMESPACE_COL_DESC_BYTES;
/** Table descriptor for namespace table */
- public static final HTableDescriptor NAMESPACE_TABLEDESC
- = new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
+ public static final HTableDescriptor NAMESPACE_TABLEDESC =
+ new HTableDescriptor(TableDescriptorBuilder.NAMESPACE_TABLEDESC);
/**
+ * Set the table owner.
* @deprecated since 0.94.1
* @see HBASE-6188
*/
@@ -876,6 +812,7 @@ public HTableDescriptor setOwner(User owner) {
}
/**
+ * Set the table owner.
* @deprecated since 0.94.1
* @see HBASE-6188
*/
@@ -887,6 +824,7 @@ public HTableDescriptor setOwnerString(String ownerString) {
}
/**
+ * Get the table owner.
* @deprecated since 0.94.1
* @see HBASE-6188
*/
@@ -897,22 +835,20 @@ public String getOwnerString() {
}
/**
- * @return This instance serialized with pb with pb magic prefix
- * @see #parseFrom(byte[])
+ * Returns This instance serialized with pb with pb magic prefix
*/
public byte[] toByteArray() {
return TableDescriptorBuilder.toByteArray(delegatee);
}
/**
+ * Parse the serialized representation of a {@link HTableDescriptor}
* @param bytes A pb serialized {@link HTableDescriptor} instance with pb magic prefix
* @return An instance of {@link HTableDescriptor} made from bytes
- * @throws DeserializationException
- * @throws IOException
* @see #toByteArray()
*/
- public static HTableDescriptor parseFrom(final byte [] bytes)
- throws DeserializationException, IOException {
+ public static HTableDescriptor parseFrom(final byte[] bytes)
+ throws DeserializationException, IOException {
TableDescriptor desc = TableDescriptorBuilder.parseFrom(bytes);
if (desc instanceof ModifyableTableDescriptor) {
return new HTableDescriptor((ModifyableTableDescriptor) desc);
@@ -932,16 +868,14 @@ public String getConfigurationValue(String key) {
* Getter for fetching an unmodifiable map.
*/
public Map getConfiguration() {
- return delegatee.getValues().entrySet().stream()
- .collect(Collectors.toMap(
- e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(), e.getKey().getLength()),
- e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(), e.getValue().getLength())
- ));
+ return delegatee.getValues().entrySet().stream().collect(Collectors.toMap(
+ e -> Bytes.toString(e.getKey().get(), e.getKey().getOffset(), e.getKey().getLength()),
+ e -> Bytes.toString(e.getValue().get(), e.getValue().getOffset(), e.getValue().getLength())));
}
/**
* Setter for storing a configuration setting in map.
- * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
+ * @param key Config key. Same as XML config key e.g. hbase.something.or.other.
* @param value String value. If null, removes the setting.
*/
public HTableDescriptor setConfiguration(String key, String value) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckEmptyRegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckEmptyRegionInfo.java
new file mode 100644
index 000000000000..5d1ca54bf1be
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckEmptyRegionInfo.java
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Empty Region Info from Catalog Janitor Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of Catalog Janitor
+ * inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckEmptyRegionInfo {
+ private final String regionInfo;
+
+ public HbckEmptyRegionInfo(String emptyRegionInfo) {
+ this.regionInfo = emptyRegionInfo;
+ }
+
+ public String getRegionInfo() {
+ return regionInfo;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckInconsistentRegions.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckInconsistentRegions.java
new file mode 100644
index 000000000000..f32f73a73d15
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckInconsistentRegions.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import java.util.List;
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present HBCK Inconsistent Regions from HBCK Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of HBCK inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckInconsistentRegions {
+ private final String regionId;
+ private final HbckServerName serverNameInMeta;
+ private final List listOfServers;
+
+ public HbckInconsistentRegions(String inconsistentRegionId, HbckServerName serverNameInMeta,
+ List listOfServerName) {
+ this.regionId = inconsistentRegionId;
+ this.serverNameInMeta = serverNameInMeta;
+ this.listOfServers = listOfServerName;
+ }
+
+ public String getRegionId() {
+ return regionId;
+ }
+
+ public HbckServerName getServerNameInMeta() {
+ return serverNameInMeta;
+ }
+
+ public List getListOfServers() {
+ return listOfServers;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnFS.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnFS.java
new file mode 100644
index 000000000000..43a045fb2933
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnFS.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Orphan Region on FS from HBCK Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of HBCK Inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckOrphanRegionsOnFS {
+ private final String regionId;
+ private final String regionHdfsPath;
+
+ public HbckOrphanRegionsOnFS(String regionId, String orphanRegionHdfsPath) {
+ this.regionId = regionId;
+ this.regionHdfsPath = orphanRegionHdfsPath;
+ }
+
+ public String getRegionId() {
+ return regionId;
+ }
+
+ public String getRegionHdfsPath() {
+ return regionHdfsPath;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnRS.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnRS.java
new file mode 100644
index 000000000000..2d442b7a9e40
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOrphanRegionsOnRS.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Orphan Region on RS from HBCK Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of HBCK Inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckOrphanRegionsOnRS {
+ private final String regionId;
+ private final HbckServerName rsName;
+
+ public HbckOrphanRegionsOnRS(String orphanRegionId, HbckServerName orphanRegionRsName) {
+ this.regionId = orphanRegionId;
+ this.rsName = orphanRegionRsName;
+ }
+
+ public String getRegionId() {
+ return regionId;
+ }
+
+ public HbckServerName getRsName() {
+ return rsName;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOverlapRegions.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOverlapRegions.java
new file mode 100644
index 000000000000..4170932bf563
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckOverlapRegions.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Region Overlap from Catalog Janitor Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of Catalog Janitor
+ * inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckOverlapRegions {
+ private final HbckRegionDetails region1Info;
+ private final HbckRegionDetails region2Info;
+
+ public HbckOverlapRegions(HbckRegionDetails region1Info, HbckRegionDetails region2Info) {
+ this.region1Info = region1Info;
+ this.region2Info = region2Info;
+ }
+
+ public HbckRegionDetails getRegion1Info() {
+ return region1Info;
+ }
+
+ public HbckRegionDetails getRegion2Info() {
+ return region2Info;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionDetails.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionDetails.java
new file mode 100644
index 000000000000..a79245636276
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionDetails.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO class for HBCK RegionInfo in HBCK Inconsistencies report.
+ */
+@InterfaceAudience.Public
+public class HbckRegionDetails {
+ private final String regionId;
+ private final String tableName;
+ private final String startKey;
+ private final String endKey;
+
+ public HbckRegionDetails(String regionId, String tableName, String startKey, String endKey) {
+ this.regionId = regionId;
+ this.tableName = tableName;
+ this.startKey = startKey;
+ this.endKey = endKey;
+ }
+
+ public String getRegionId() {
+ return regionId;
+ }
+
+ public String getTableName() {
+ return tableName;
+ }
+
+ public String getStartKey() {
+ return startKey;
+ }
+
+ public String getEndKey() {
+ return endKey;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionHoles.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionHoles.java
new file mode 100644
index 000000000000..643e014735a0
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckRegionHoles.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Region Holes from Catalog Janitor Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of Catalog Janitor
+ * inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckRegionHoles {
+ private final HbckRegionDetails region1Info;
+ private final HbckRegionDetails region2Info;
+
+ public HbckRegionHoles(HbckRegionDetails region1Info, HbckRegionDetails region2Info) {
+ this.region1Info = region1Info;
+ this.region2Info = region2Info;
+ }
+
+ public HbckRegionDetails getRegion1Info() {
+ return region1Info;
+ }
+
+ public HbckRegionDetails getRegion2Info() {
+ return region2Info;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckServerName.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckServerName.java
new file mode 100644
index 000000000000..2c6b899fb15c
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckServerName.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO class for ServerName in HBCK Inconsistencies report.
+ */
+@InterfaceAudience.Public
+public class HbckServerName {
+ private final String hostName;
+ private final int hostPort;
+ private final long startCode;
+
+ public HbckServerName(String hostName, int hostPort, long startCode) {
+ this.hostName = hostName;
+ this.hostPort = hostPort;
+ this.startCode = startCode;
+ }
+
+ public String getHostName() {
+ return hostName;
+ }
+
+ public int getHostPort() {
+ return hostPort;
+ }
+
+ public long getStartCode() {
+ return startCode;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckUnknownServers.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckUnknownServers.java
new file mode 100644
index 000000000000..c070f84e69fe
--- /dev/null
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HbckUnknownServers.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase;
+
+import org.apache.yetus.audience.InterfaceAudience;
+
+/**
+ * POJO to present Unknown Regions from Catalog Janitor Inconsistencies Report via REST API. These
+ * inconsistencies are shown on hbck.jsp page on Active HMaster UI as part of Catalog Janitor
+ * inconsistencies.
+ */
+@InterfaceAudience.Public
+public class HbckUnknownServers {
+ private final HbckRegionDetails regionInfo;
+ private final HbckServerName serverName;
+
+ public HbckUnknownServers(HbckRegionDetails regionInfo, HbckServerName unknownServerName) {
+ this.regionInfo = regionInfo;
+ this.serverName = unknownServerName;
+ }
+
+ public HbckRegionDetails getRegionInfo() {
+ return regionInfo;
+ }
+
+ public HbckServerName getServerName() {
+ return serverName;
+ }
+}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
index 63c26e2c393f..2a099157bc76 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/InvalidFamilyOperationException.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -21,12 +20,13 @@
import org.apache.yetus.audience.InterfaceAudience;
/**
- * Thrown if a request is table schema modification is requested but
- * made for an invalid family name.
+ * Thrown if a request is table schema modification is requested but made for an invalid family
+ * name.
*/
@InterfaceAudience.Public
public class InvalidFamilyOperationException extends DoNotRetryIOException {
private static final long serialVersionUID = (1L << 22) - 1L;
+
/** default constructor */
public InvalidFamilyOperationException() {
super();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
index dd19fa1c2279..2ae80cade98a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/KeepDeletedCells.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -24,27 +23,25 @@
* Ways to keep cells marked for delete around.
*/
/*
- * Don't change the TRUE/FALSE labels below, these have to be called
- * this way for backwards compatibility.
+ * Don't change the TRUE/FALSE labels below, these have to be called this way for backwards
+ * compatibility.
*/
@InterfaceAudience.Public
public enum KeepDeletedCells {
/** Deleted Cells are not retained. */
FALSE,
/**
- * Deleted Cells are retained until they are removed by other means
- * such TTL or VERSIONS.
- * If no TTL is specified or no new versions of delete cells are
- * written, they are retained forever.
+ * Deleted Cells are retained until they are removed by other means such TTL or VERSIONS. If no
+ * TTL is specified or no new versions of delete cells are written, they are retained forever.
*/
TRUE,
/**
- * Deleted Cells are retained until the delete marker expires due to TTL.
- * This is useful when TTL is combined with MIN_VERSIONS and one
- * wants to keep a minimum number of versions around but at the same
- * time remove deleted cells after the TTL.
+ * Deleted Cells are retained until the delete marker expires due to TTL. This is useful when TTL
+ * is combined with MIN_VERSIONS and one wants to keep a minimum number of versions around but at
+ * the same time remove deleted cells after the TTL.
*/
TTL;
+
public static KeepDeletedCells getValue(String val) {
return valueOf(val.toUpperCase());
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
index 35cdecba9bb6..86e394e33403 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MasterNotRunningException.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -26,6 +25,7 @@
@InterfaceAudience.Public
public class MasterNotRunningException extends HBaseIOException {
private static final long serialVersionUID = (1L << 23) - 1L;
+
/** default constructor */
public MasterNotRunningException() {
super();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
index 099ea4054591..b913ac0506cd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MemoryCompactionPolicy.java
@@ -1,5 +1,4 @@
-/**
- *
+/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
@@ -31,16 +30,15 @@ public enum MemoryCompactionPolicy {
NONE,
/**
* Basic policy applies optimizations which modify the index to a more compacted representation.
- * This is beneficial in all access patterns. The smaller the cells are the greater the
- * benefit of this policy.
- * This is the default policy.
+ * This is beneficial in all access patterns. The smaller the cells are the greater the benefit of
+ * this policy. This is the default policy.
*/
BASIC,
/**
- * In addition to compacting the index representation as the basic policy, eager policy
- * eliminates duplication while the data is still in memory (much like the
- * on-disk compaction does after the data is flushed to disk). This policy is most useful for
- * applications with high data churn or small working sets.
+ * In addition to compacting the index representation as the basic policy, eager policy eliminates
+ * duplication while the data is still in memory (much like the on-disk compaction does after the
+ * data is flushed to disk). This policy is most useful for applications with high data churn or
+ * small working sets.
*/
EAGER,
/**
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
index c55086c7fbe7..f29104df3c0c 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/MetaTableAccessor.java
@@ -33,11 +33,11 @@
import java.util.NavigableMap;
import java.util.SortedMap;
import java.util.TreeMap;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.Consistency;
import org.apache.hadoop.hbase.client.Delete;
@@ -72,11 +72,12 @@
import org.apache.hadoop.hbase.util.ExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.PairOfSameType;
-import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
+
/**
*
* Read/write operations on hbase:meta region as well as assignment information stored
@@ -153,6 +154,7 @@ public class MetaTableAccessor {
private static final byte SEPARATED_BYTE = 0x00;
@InterfaceAudience.Private
+ @SuppressWarnings("ImmutableEnumChecker")
public enum QueryType {
ALL(HConstants.TABLE_FAMILY, HConstants.CATALOG_FAMILY),
REGION(HConstants.CATALOG_FAMILY),
@@ -174,8 +176,8 @@ byte[][] getFamilies() {
static final char META_REPLICA_ID_DELIMITER = '_';
/** A regex for parsing server columns from meta. See above javadoc for meta layout */
- private static final Pattern SERVER_COLUMN_PATTERN
- = Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
+ private static final Pattern SERVER_COLUMN_PATTERN =
+ Pattern.compile("^server(_[0-9a-fA-F]{4})?$");
////////////////////////
// Reading operations //
@@ -184,10 +186,10 @@ byte[][] getFamilies() {
/**
* Performs a full scan of hbase:meta for regions.
* @param connection connection we're using
- * @param visitor Visitor invoked against each row in regions family.
+ * @param visitor Visitor invoked against each row in regions family.
*/
public static void fullScanRegions(Connection connection, final Visitor visitor)
- throws IOException {
+ throws IOException {
scanMeta(connection, null, null, QueryType.REGION, visitor);
}
@@ -202,17 +204,17 @@ public static List fullScanRegions(Connection connection) throws IOExcep
/**
* Performs a full scan of hbase:meta for tables.
* @param connection connection we're using
- * @param visitor Visitor invoked against each row in tables family.
+ * @param visitor Visitor invoked against each row in tables family.
*/
public static void fullScanTables(Connection connection, final Visitor visitor)
- throws IOException {
+ throws IOException {
scanMeta(connection, null, null, QueryType.TABLE, visitor);
}
/**
* Performs a full scan of hbase:meta.
* @param connection connection we're using
- * @param type scanned part of meta
+ * @param type scanned part of meta
* @return List of {@link Result}
*/
private static List fullScan(Connection connection, QueryType type) throws IOException {
@@ -257,12 +259,10 @@ private static Result get(final Table t, final Get g) throws IOException {
* @deprecated use {@link #getRegionLocation(Connection, byte[])} instead
*/
@Deprecated
- public static Pair getRegion(Connection connection, byte [] regionName)
+ public static Pair getRegion(Connection connection, byte[] regionName)
throws IOException {
HRegionLocation location = getRegionLocation(connection, regionName);
- return location == null
- ? null
- : new Pair<>(location.getRegionInfo(), location.getServerName());
+ return location == null ? null : new Pair<>(location.getRegionInfo(), location.getServerName());
}
/**
@@ -272,7 +272,7 @@ public static Pair getRegion(Connection connection, byte
* @return HRegionLocation for the given region
*/
public static HRegionLocation getRegionLocation(Connection connection, byte[] regionName)
- throws IOException {
+ throws IOException {
byte[] row = regionName;
RegionInfo parsedInfo = null;
try {
@@ -287,8 +287,10 @@ public static HRegionLocation getRegionLocation(Connection connection, byte[] re
get.addFamily(HConstants.CATALOG_FAMILY);
Result r = get(getMetaHTable(connection), get);
RegionLocations locations = getRegionLocations(r);
- return locations == null ? null
- : locations.getRegionLocation(parsedInfo == null ? 0 : parsedInfo.getReplicaId());
+ return locations == null
+ ? null
+ : locations.getRegionLocation(
+ parsedInfo == null ? RegionInfo.DEFAULT_REPLICA_ID : parsedInfo.getReplicaId());
}
/**
@@ -298,16 +300,14 @@ public static HRegionLocation getRegionLocation(Connection connection, byte[] re
* @return HRegionLocation for the given region
*/
public static HRegionLocation getRegionLocation(Connection connection, RegionInfo regionInfo)
- throws IOException {
- return getRegionLocation(getCatalogFamilyRow(connection, regionInfo),
- regionInfo, regionInfo.getReplicaId());
+ throws IOException {
+ return getRegionLocation(getCatalogFamilyRow(connection, regionInfo), regionInfo,
+ regionInfo.getReplicaId());
}
- /**
- * @return Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta.
- */
+ /** Returns Return the {@link HConstants#CATALOG_FAMILY} row from hbase:meta. */
public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
- throws IOException {
+ throws IOException {
Get get = new Get(getMetaKeyForRegion(ri));
get.addFamily(HConstants.CATALOG_FAMILY);
return get(getMetaHTable(connection), get);
@@ -318,81 +318,80 @@ public static byte[] getMetaKeyForRegion(RegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
- /** Returns an HRI parsed from this regionName. Not all the fields of the HRI
- * is stored in the name, so the returned object should only be used for the fields
- * in the regionName.
+ /**
+ * Returns an HRI parsed from this regionName. Not all the fields of the HRI is stored in the
+ * name, so the returned object should only be used for the fields in the regionName.
*/
// This should be moved to RegionInfo? TODO.
public static RegionInfo parseRegionInfoFromRegionName(byte[] regionName) throws IOException {
byte[][] fields = RegionInfo.parseRegionName(regionName);
long regionId = Long.parseLong(Bytes.toString(fields[2]));
int replicaId = fields.length > 3 ? Integer.parseInt(Bytes.toString(fields[3]), 16) : 0;
- return RegionInfoBuilder.newBuilder(TableName.valueOf(fields[0]))
- .setStartKey(fields[1]).setRegionId(regionId).setReplicaId(replicaId).build();
+ return RegionInfoBuilder.newBuilder(TableName.valueOf(fields[0])).setStartKey(fields[1])
+ .setRegionId(regionId).setReplicaId(replicaId).build();
}
/**
* Gets the result in hbase:meta for the specified region.
* @param connection connection we're using
- * @param regionName region we're looking for
+ * @param regionInfo region we're looking for
* @return result of the specified region
*/
- public static Result getRegionResult(Connection connection,
- byte[] regionName) throws IOException {
- Get get = new Get(regionName);
+ public static Result getRegionResult(Connection connection, RegionInfo regionInfo)
+ throws IOException {
+ Get get = new Get(getMetaKeyForRegion(regionInfo));
get.addFamily(HConstants.CATALOG_FAMILY);
return get(getMetaHTable(connection), get);
}
/**
- * Scans META table for a row whose key contains the specified regionEncodedName,
- * returning a single related Result instance if any row is found, null otherwise.
- *
- * @param connection the connection to query META table.
+ * Scans META table for a row whose key contains the specified regionEncodedName, returning
+ * a single related Result instance if any row is found, null otherwise.
+ * @param connection the connection to query META table.
* @param regionEncodedName the region encoded name to look for at META.
* @return Result instance with the row related info in META, null otherwise.
* @throws IOException if any errors occur while querying META.
*/
- public static Result scanByRegionEncodedName(Connection connection,
- String regionEncodedName) throws IOException {
- RowFilter rowFilter = new RowFilter(CompareOperator.EQUAL,
- new SubstringComparator(regionEncodedName));
+ public static Result scanByRegionEncodedName(Connection connection, String regionEncodedName)
+ throws IOException {
+ RowFilter rowFilter =
+ new RowFilter(CompareOperator.EQUAL, new SubstringComparator(regionEncodedName));
Scan scan = getMetaScan(connection.getConfiguration(), 1);
scan.setFilter(rowFilter);
try (Table table = getMetaHTable(connection);
- ResultScanner resultScanner = table.getScanner(scan)) {
+ ResultScanner resultScanner = table.getScanner(scan)) {
return resultScanner.next();
}
}
/**
- * @return Return all regioninfos listed in the 'info:merge*' columns of
- * the regionName row.
+ * Returns Return all regioninfos listed in the 'info:merge*' columns of the {@code regionInfo}
+ * row.
*/
@Nullable
- public static List getMergeRegions(Connection connection, byte[] regionName)
- throws IOException {
- return getMergeRegions(getRegionResult(connection, regionName).rawCells());
+ public static List getMergeRegions(Connection connection, RegionInfo regionInfo)
+ throws IOException {
+ return getMergeRegions(getRegionResult(connection, regionInfo).rawCells());
}
/**
- * Check whether the given {@code regionName} has any 'info:merge*' columns.
+ * Check whether the given {@code regionInfo} has any 'info:merge*' columns.
*/
- public static boolean hasMergeRegions(Connection conn, byte[] regionName) throws IOException {
- return hasMergeRegions(getRegionResult(conn, regionName).rawCells());
+ public static boolean hasMergeRegions(Connection conn, RegionInfo regionInfo) throws IOException {
+ return hasMergeRegions(getRegionResult(conn, regionInfo).rawCells());
}
/**
- * @return Deserialized values of <qualifier,regioninfo> pairs taken from column values that
- * match the regex 'info:merge.*' in array of cells.
+ * Returns Deserialized values of <qualifier,regioninfo> pairs taken from column values that
+ * match the regex 'info:merge.*' in array of cells.
*/
@Nullable
- public static Map getMergeRegionsWithName(Cell [] cells) {
+ public static Map getMergeRegionsWithName(Cell[] cells) {
if (cells == null) {
return null;
}
Map regionsToMerge = null;
- for (Cell cell: cells) {
+ for (Cell cell : cells) {
if (!isMergeQualifierPrefix(cell)) {
continue;
}
@@ -410,21 +409,21 @@ public static Map getMergeRegionsWithName(Cell [] cells) {
}
/**
- * @return Deserialized regioninfo values taken from column values that match
- * the regex 'info:merge.*' in array of cells.
+ * Returns Deserialized regioninfo values taken from column values that match the regex
+ * 'info:merge.*' in array of cells.
*/
@Nullable
- public static List getMergeRegions(Cell [] cells) {
+ public static List getMergeRegions(Cell[] cells) {
Map mergeRegionsWithName = getMergeRegionsWithName(cells);
return (mergeRegionsWithName == null) ? null : new ArrayList<>(mergeRegionsWithName.values());
}
/**
- * @return True if any merge regions present in cells; i.e.
- * the column in cell matches the regex 'info:merge.*'.
+ * Returns True if any merge regions present in cells; i.e. the column in
+ * cell matches the regex 'info:merge.*'.
*/
- public static boolean hasMergeRegions(Cell [] cells) {
- for (Cell cell: cells) {
+ public static boolean hasMergeRegions(Cell[] cells) {
+ for (Cell cell : cells) {
if (!isMergeQualifierPrefix(cell)) {
continue;
}
@@ -433,65 +432,61 @@ public static boolean hasMergeRegions(Cell [] cells) {
return false;
}
- /**
- * @return True if the column in cell matches the regex 'info:merge.*'.
- */
+ /** Returns True if the column in cell matches the regex 'info:merge.*'. */
private static boolean isMergeQualifierPrefix(Cell cell) {
// Check to see if has family and that qualifier starts with the merge qualifier 'merge'
- return CellUtil.matchingFamily(cell, HConstants.CATALOG_FAMILY) &&
- PrivateCellUtil.qualifierStartsWith(cell, HConstants.MERGE_QUALIFIER_PREFIX);
+ return CellUtil.matchingFamily(cell, HConstants.CATALOG_FAMILY)
+ && PrivateCellUtil.qualifierStartsWith(cell, HConstants.MERGE_QUALIFIER_PREFIX);
}
/**
* Lists all of the regions currently in META.
- *
- * @param connection to connect with
+ * @param connection to connect with
* @param excludeOfflinedSplitParents False if we are to include offlined/splitparents regions,
* true and we'll leave out offlined regions from returned list
* @return List of all user-space regions.
*/
public static List getAllRegions(Connection connection,
- boolean excludeOfflinedSplitParents)
- throws IOException {
+ boolean excludeOfflinedSplitParents) throws IOException {
List> result;
- result = getTableRegionsAndLocations(connection, null,
- excludeOfflinedSplitParents);
+ result = getTableRegionsAndLocations(connection, null, excludeOfflinedSplitParents);
return getListOfRegionInfos(result);
}
/**
- * Gets all of the regions of the specified table. Do not use this method
- * to get meta table regions, use methods in MetaTableLocator instead.
+ * Gets all of the regions of the specified table. Do not use this method to get meta table
+ * regions, use methods in MetaTableLocator instead.
* @param connection connection we're using
- * @param tableName table we're looking for
+ * @param tableName table we're looking for
* @return Ordered list of {@link RegionInfo}.
*/
public static List getTableRegions(Connection connection, TableName tableName)
- throws IOException {
+ throws IOException {
return getTableRegions(connection, tableName, false);
}
/**
- * Gets all of the regions of the specified table. Do not use this method
- * to get meta table regions, use methods in MetaTableLocator instead.
- * @param connection connection we're using
- * @param tableName table we're looking for
- * @param excludeOfflinedSplitParents If true, do not include offlined split
- * parents in the return.
+ * Gets all of the regions of the specified table. Do not use this method to get meta table
+ * regions, use methods in MetaTableLocator instead.
+ * @param connection connection we're using
+ * @param tableName table we're looking for
+ * @param excludeOfflinedSplitParents If true, do not include offlined split parents in the
+ * return.
* @return Ordered list of {@link RegionInfo}.
*/
public static List getTableRegions(Connection connection, TableName tableName,
- final boolean excludeOfflinedSplitParents) throws IOException {
+ final boolean excludeOfflinedSplitParents) throws IOException {
List> result =
getTableRegionsAndLocations(connection, tableName, excludeOfflinedSplitParents);
return getListOfRegionInfos(result);
}
- private static List getListOfRegionInfos(
- final List> pairs) {
+ @SuppressWarnings("MixedMutabilityReturnType")
+ private static List
+ getListOfRegionInfos(final List> pairs) {
if (pairs == null || pairs.isEmpty()) {
return Collections.emptyList();
}
@@ -503,30 +498,28 @@ private static List getListOfRegionInfos(
}
/**
- * @param tableName table we're working with
- * @return start row for scanning META according to query type
+ * Returns start row for scanning META according to query type
*/
public static byte[] getTableStartRowForMeta(TableName tableName, QueryType type) {
if (tableName == null) {
return null;
}
switch (type) {
- case REGION:
- byte[] startRow = new byte[tableName.getName().length + 2];
- System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
- startRow[startRow.length - 2] = HConstants.DELIMITER;
- startRow[startRow.length - 1] = HConstants.DELIMITER;
- return startRow;
- case ALL:
- case TABLE:
- default:
- return tableName.getName();
+ case REGION:
+ byte[] startRow = new byte[tableName.getName().length + 2];
+ System.arraycopy(tableName.getName(), 0, startRow, 0, tableName.getName().length);
+ startRow[startRow.length - 2] = HConstants.DELIMITER;
+ startRow[startRow.length - 1] = HConstants.DELIMITER;
+ return startRow;
+ case ALL:
+ case TABLE:
+ default:
+ return tableName.getName();
}
}
/**
- * @param tableName table we're working with
- * @return stop row for scanning META according to query type
+ * Returns stop row for scanning META according to query type
*/
public static byte[] getTableStopRowForMeta(TableName tableName, QueryType type) {
if (tableName == null) {
@@ -534,30 +527,28 @@ public static byte[] getTableStopRowForMeta(TableName tableName, QueryType type)
}
final byte[] stopRow;
switch (type) {
- case REGION:
- stopRow = new byte[tableName.getName().length + 3];
- System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
- stopRow[stopRow.length - 3] = ' ';
- stopRow[stopRow.length - 2] = HConstants.DELIMITER;
- stopRow[stopRow.length - 1] = HConstants.DELIMITER;
- break;
- case ALL:
- case TABLE:
- default:
- stopRow = new byte[tableName.getName().length + 1];
- System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
- stopRow[stopRow.length - 1] = ' ';
- break;
+ case REGION:
+ stopRow = new byte[tableName.getName().length + 3];
+ System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
+ stopRow[stopRow.length - 3] = ' ';
+ stopRow[stopRow.length - 2] = HConstants.DELIMITER;
+ stopRow[stopRow.length - 1] = HConstants.DELIMITER;
+ break;
+ case ALL:
+ case TABLE:
+ default:
+ stopRow = new byte[tableName.getName().length + 1];
+ System.arraycopy(tableName.getName(), 0, stopRow, 0, tableName.getName().length);
+ stopRow[stopRow.length - 1] = ' ';
+ break;
}
return stopRow;
}
/**
- * This method creates a Scan object that will only scan catalog rows that
- * belong to the specified table. It doesn't specify any columns.
- * This is a better alternative to just using a start row and scan until
- * it hits a new table since that requires parsing the HRI to get the table
- * name.
+ * This method creates a Scan object that will only scan catalog rows that belong to the specified
+ * table. It doesn't specify any columns. This is a better alternative to just using a start row
+ * and scan until it hits a new table since that requires parsing the HRI to get the table name.
* @param tableName bytes of table's name
* @return configured Scan object
*/
@@ -591,29 +582,28 @@ private static Scan getMetaScan(Configuration conf, int rowUpperLimit) {
/**
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
* @param connection connection we're using
- * @param tableName table we're looking for
+ * @param tableName table we're looking for
* @return Return list of regioninfos and server.
*/
public static List>
- getTableRegionsAndLocations(Connection connection, TableName tableName)
- throws IOException {
+ getTableRegionsAndLocations(Connection connection, TableName tableName) throws IOException {
return getTableRegionsAndLocations(connection, tableName, true);
}
/**
* Do not use this method to get meta table regions, use methods in MetaTableLocator instead.
- * @param connection connection we're using
- * @param tableName table to work with, can be null for getting all regions
+ * @param connection connection we're using
+ * @param tableName table to work with, can be null for getting all regions
* @param excludeOfflinedSplitParents don't return split parents
* @return Return list of regioninfos and server addresses.
*/
// What happens here when 1M regions in hbase:meta? This won't scale?
public static List> getTableRegionsAndLocations(
- Connection connection, @Nullable final TableName tableName,
- final boolean excludeOfflinedSplitParents) throws IOException {
+ Connection connection, @Nullable final TableName tableName,
+ final boolean excludeOfflinedSplitParents) throws IOException {
if (tableName != null && tableName.equals(TableName.META_TABLE_NAME)) {
- throw new IOException("This method can't be used to locate meta regions;"
- + " use MetaTableLocator instead");
+ throw new IOException(
+ "This method can't be used to locate meta regions;" + " use MetaTableLocator instead");
}
// Make a version of CollectingVisitor that collects RegionInfo and ServerAddress
CollectingVisitor> visitor =
@@ -645,23 +635,19 @@ void add(Result r) {
}
}
};
- scanMeta(connection,
- getTableStartRowForMeta(tableName, QueryType.REGION),
- getTableStopRowForMeta(tableName, QueryType.REGION),
- QueryType.REGION, visitor);
+ scanMeta(connection, getTableStartRowForMeta(tableName, QueryType.REGION),
+ getTableStopRowForMeta(tableName, QueryType.REGION), QueryType.REGION, visitor);
return visitor.getResults();
}
/**
+ * Get the user regions a given server is hosting.
* @param connection connection we're using
* @param serverName server whose regions we're interested in
- * @return List of user regions installed on this server (does not include
- * catalog regions).
- * @throws IOException
+ * @return List of user regions installed on this server (does not include catalog regions).
*/
- public static NavigableMap
- getServerUserRegions(Connection connection, final ServerName serverName)
- throws IOException {
+ public static NavigableMap getServerUserRegions(Connection connection,
+ final ServerName serverName) throws IOException {
final NavigableMap hris = new TreeMap<>();
// Fill the above hris map with entries from hbase:meta that have the passed
// servername.
@@ -684,10 +670,9 @@ void add(Result r) {
return hris;
}
- public static void fullScanMetaAndPrint(Connection connection)
- throws IOException {
+ public static void fullScanMetaAndPrint(Connection connection) throws IOException {
Visitor v = r -> {
- if (r == null || r.isEmpty()) {
+ if (r == null || r.isEmpty()) {
return true;
}
LOG.info("fullScanMetaAndPrint.Current Meta Row: " + r);
@@ -711,18 +696,24 @@ public static void fullScanMetaAndPrint(Connection connection)
}
public static void scanMetaForTableRegions(Connection connection, Visitor visitor,
- TableName tableName) throws IOException {
- scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor);
+ TableName tableName, CatalogReplicaMode metaReplicaMode) throws IOException {
+ scanMeta(connection, tableName, QueryType.REGION, Integer.MAX_VALUE, visitor, metaReplicaMode);
+ }
+
+ public static void scanMetaForTableRegions(Connection connection, Visitor visitor,
+ TableName tableName) throws IOException {
+ scanMetaForTableRegions(connection, visitor, tableName, CatalogReplicaMode.NONE);
}
private static void scanMeta(Connection connection, TableName table, QueryType type, int maxRows,
- final Visitor visitor) throws IOException {
+ final Visitor visitor, CatalogReplicaMode metaReplicaMode) throws IOException {
scanMeta(connection, getTableStartRowForMeta(table, type), getTableStopRowForMeta(table, type),
- type, maxRows, visitor);
+ type, null, maxRows, visitor, metaReplicaMode);
+
}
private static void scanMeta(Connection connection, @Nullable final byte[] startRow,
- @Nullable final byte[] stopRow, QueryType type, final Visitor visitor) throws IOException {
+ @Nullable final byte[] stopRow, QueryType type, final Visitor visitor) throws IOException {
scanMeta(connection, startRow, stopRow, type, Integer.MAX_VALUE, visitor);
}
@@ -735,7 +726,7 @@ private static void scanMeta(Connection connection, @Nullable final byte[] start
* @param rowLimit max number of rows to return
*/
public static void scanMeta(Connection connection, final Visitor visitor,
- final TableName tableName, final byte[] row, final int rowLimit) throws IOException {
+ final TableName tableName, final byte[] row, final int rowLimit) throws IOException {
byte[] startRow = null;
byte[] stopRow = null;
if (tableName != null) {
@@ -753,23 +744,21 @@ public static void scanMeta(Connection connection, final Visitor visitor,
/**
* Performs a scan of META table.
* @param connection connection we're using
- * @param startRow Where to start the scan. Pass null if want to begin scan
- * at first row.
- * @param stopRow Where to stop the scan. Pass null if want to scan all rows
- * from the start one
- * @param type scanned part of meta
- * @param maxRows maximum rows to return
- * @param visitor Visitor invoked against each row.
+ * @param startRow Where to start the scan. Pass null if want to begin scan at first row.
+ * @param stopRow Where to stop the scan. Pass null if want to scan all rows from the start one
+ * @param type scanned part of meta
+ * @param maxRows maximum rows to return
+ * @param visitor Visitor invoked against each row.
*/
static void scanMeta(Connection connection, @Nullable final byte[] startRow,
- @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor)
- throws IOException {
- scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor);
+ @Nullable final byte[] stopRow, QueryType type, int maxRows, final Visitor visitor)
+ throws IOException {
+ scanMeta(connection, startRow, stopRow, type, null, maxRows, visitor, CatalogReplicaMode.NONE);
}
private static void scanMeta(Connection connection, @Nullable final byte[] startRow,
- @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
- final Visitor visitor) throws IOException {
+ @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
+ final Visitor visitor, CatalogReplicaMode metaReplicaMode) throws IOException {
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
Scan scan = getMetaScan(connection.getConfiguration(), rowUpperLimit);
@@ -787,13 +776,32 @@ private static void scanMeta(Connection connection, @Nullable final byte[] start
}
if (LOG.isTraceEnabled()) {
- LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow) +
- " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit +
- " with caching=" + scan.getCaching());
+ LOG.trace("Scanning META" + " starting at row=" + Bytes.toStringBinary(startRow)
+ + " stopping at row=" + Bytes.toStringBinary(stopRow) + " for max=" + rowUpperLimit
+ + " with caching=" + scan.getCaching());
}
int currentRow = 0;
try (Table metaTable = getMetaHTable(connection)) {
+ switch (metaReplicaMode) {
+ case LOAD_BALANCE:
+ int numOfReplicas = metaTable.getDescriptor().getRegionReplication();
+ if (numOfReplicas > 1) {
+ int replicaId = ThreadLocalRandom.current().nextInt(numOfReplicas);
+
+ // When the replicaId is 0, do not set to Consistency.TIMELINE
+ if (replicaId > 0) {
+ scan.setReplicaId(replicaId);
+ scan.setConsistency(Consistency.TIMELINE);
+ }
+ }
+ break;
+ case HEDGED_READ:
+ scan.setConsistency(Consistency.TIMELINE);
+ break;
+ default:
+ // Do nothing
+ }
try (ResultScanner scanner = metaTable.getScanner(scan)) {
Result data;
while ((data = scanner.next()) != null) {
@@ -814,12 +822,10 @@ private static void scanMeta(Connection connection, @Nullable final byte[] start
}
}
- /**
- * @return Get closest metatable region row to passed row
- */
+ /** Returns Get closest metatable region row to passed row */
@NonNull
private static RegionInfo getClosestRegionInfo(Connection connection,
- @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
+ @NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
Scan scan = getMetaScan(connection.getConfiguration(), 1);
scan.setReversed(true);
@@ -827,13 +833,13 @@ private static RegionInfo getClosestRegionInfo(Connection connection,
try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {
Result result = resultScanner.next();
if (result == null) {
- throw new TableNotFoundException("Cannot find row in META " +
- " for table: " + tableName + ", row=" + Bytes.toStringBinary(row));
+ throw new TableNotFoundException("Cannot find row in META " + " for table: " + tableName
+ + ", row=" + Bytes.toStringBinary(row));
}
RegionInfo regionInfo = getRegionInfo(result);
if (regionInfo == null) {
- throw new IOException("RegionInfo was null or empty in Meta for " +
- tableName + ", row=" + Bytes.toStringBinary(row));
+ throw new IOException("RegionInfo was null or empty in Meta for " + tableName + ", row="
+ + Bytes.toStringBinary(row));
}
return regionInfo;
}
@@ -885,9 +891,10 @@ private static byte[] getRegionStateColumn() {
* @return a byte[] for state qualifier
*/
public static byte[] getRegionStateColumn(int replicaId) {
- return replicaId == 0 ? HConstants.STATE_QUALIFIER
- : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ return replicaId == 0
+ ? HConstants.STATE_QUALIFIER
+ : Bytes.toBytes(HConstants.STATE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -896,9 +903,10 @@ public static byte[] getRegionStateColumn(int replicaId) {
* @return a byte[] for sn column qualifier
*/
public static byte[] getServerNameColumn(int replicaId) {
- return replicaId == 0 ? HConstants.SERVERNAME_QUALIFIER
- : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ return replicaId == 0
+ ? HConstants.SERVERNAME_QUALIFIER
+ : Bytes.toBytes(HConstants.SERVERNAME_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -910,7 +918,7 @@ public static byte[] getServerColumn(int replicaId) {
return replicaId == 0
? HConstants.SERVER_QUALIFIER
: Bytes.toBytes(HConstants.SERVER_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -922,7 +930,7 @@ public static byte[] getStartCodeColumn(int replicaId) {
return replicaId == 0
? HConstants.STARTCODE_QUALIFIER
: Bytes.toBytes(HConstants.STARTCODE_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
@@ -934,12 +942,12 @@ public static byte[] getSeqNumColumn(int replicaId) {
return replicaId == 0
? HConstants.SEQNUM_QUALIFIER
: Bytes.toBytes(HConstants.SEQNUM_QUALIFIER_STR + META_REPLICA_ID_DELIMITER
- + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
+ + String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId));
}
/**
- * Parses the replicaId from the server column qualifier. See top of the class javadoc
- * for the actual meta layout
+ * Parses the replicaId from the server column qualifier. See top of the class javadoc for the
+ * actual meta layout
* @param serverColumn the column qualifier
* @return an int for the replicaId
*/
@@ -969,14 +977,14 @@ public static ServerName getServerName(final Result r, final int replicaId) {
byte[] serverColumn = getServerColumn(replicaId);
Cell cell = r.getColumnLatestCell(getCatalogFamily(), serverColumn);
if (cell == null || cell.getValueLength() == 0) return null;
- String hostAndPort = Bytes.toString(
- cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
+ String hostAndPort =
+ Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
byte[] startcodeColumn = getStartCodeColumn(replicaId);
cell = r.getColumnLatestCell(getCatalogFamily(), startcodeColumn);
if (cell == null || cell.getValueLength() == 0) return null;
try {
return ServerName.valueOf(hostAndPort,
- Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
+ Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
} catch (IllegalArgumentException e) {
LOG.error("Ignoring invalid region for server " + hostAndPort + "; cell=" + cell, e);
return null;
@@ -987,15 +995,14 @@ public static ServerName getServerName(final Result r, final int replicaId) {
* Returns the {@link ServerName} from catalog table {@link Result} where the region is
* transitioning on. It should be the same as {@link MetaTableAccessor#getServerName(Result,int)}
* if the server is at OPEN state.
- *
* @param r Result to pull the transitioning server name from
- * @return A ServerName instance or {@link MetaTableAccessor#getServerName(Result,int)}
- * if necessary fields not found or empty.
+ * @return A ServerName instance or {@link MetaTableAccessor#getServerName(Result,int)} if
+ * necessary fields not found or empty.
*/
@Nullable
public static ServerName getTargetServerName(final Result r, final int replicaId) {
- final Cell cell = r.getColumnLatestCell(HConstants.CATALOG_FAMILY,
- getServerNameColumn(replicaId));
+ final Cell cell =
+ r.getColumnLatestCell(HConstants.CATALOG_FAMILY, getServerNameColumn(replicaId));
if (cell == null || cell.getValueLength() == 0) {
RegionLocations locations = MetaTableAccessor.getRegionLocations(r);
if (locations != null) {
@@ -1006,13 +1013,13 @@ public static ServerName getTargetServerName(final Result r, final int replicaId
}
return null;
}
- return ServerName.parseServerName(Bytes.toString(cell.getValueArray(), cell.getValueOffset(),
- cell.getValueLength()));
+ return ServerName.parseServerName(
+ Bytes.toString(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength()));
}
/**
- * The latest seqnum that the server writing to meta observed when opening the region.
- * E.g. the seqNum when the result of {@link #getServerName(Result, int)} was written.
+ * The latest seqnum that the server writing to meta observed when opening the region. E.g. the
+ * seqNum when the result of {@link #getServerName(Result, int)} was written.
* @param r Result to pull the seqNum from
* @return SeqNum, or HConstants.NO_SEQNUM if there's no value written.
*/
@@ -1023,8 +1030,7 @@ private static long getSeqNumDuringOpen(final Result r, final int replicaId) {
}
/**
- * Returns the daughter regions by reading the corresponding columns of the catalog table
- * Result.
+ * Returns the daughter regions by reading the corresponding columns of the catalog table Result.
* @param data a Result object from the catalog table scan
* @return pair of RegionInfo or PairOfSameType(null, null) if region is not a split parent
*/
@@ -1036,8 +1042,8 @@ public static PairOfSameType getDaughterRegions(Result data) {
/**
* Returns an HRegionLocationList extracted from the result.
- * @return an HRegionLocationList containing all locations for the region range or null if
- * we can't deserialize the result.
+ * @return an HRegionLocationList containing all locations for the region range or null if we
+ * can't deserialize the result.
*/
@Nullable
public static RegionLocations getRegionLocations(final Result r) {
@@ -1046,7 +1052,7 @@ public static RegionLocations getRegionLocations(final Result r) {
if (regionInfo == null) return null;
List locations = new ArrayList<>(1);
- NavigableMap> familyMap = r.getNoVersionMap();
+ NavigableMap> familyMap = r.getNoVersionMap();
locations.add(getRegionLocation(r, regionInfo, 0));
@@ -1080,16 +1086,15 @@ public static RegionLocations getRegionLocations(final Result r) {
}
/**
- * Returns the HRegionLocation parsed from the given meta row Result
- * for the given regionInfo and replicaId. The regionInfo can be the default region info
- * for the replica.
- * @param r the meta row result
+ * Returns the HRegionLocation parsed from the given meta row Result for the given regionInfo and
+ * replicaId. The regionInfo can be the default region info for the replica.
+ * @param r the meta row result
* @param regionInfo RegionInfo for default replica
- * @param replicaId the replicaId for the HRegionLocation
+ * @param replicaId the replicaId for the HRegionLocation
* @return HRegionLocation parsed from the given meta row Result for the given replicaId
*/
private static HRegionLocation getRegionLocation(final Result r, final RegionInfo regionInfo,
- final int replicaId) {
+ final int replicaId) {
ServerName serverName = getServerName(r, replicaId);
long seqNum = getSeqNumDuringOpen(r, replicaId);
RegionInfo replicaInfo = RegionReplicaUtil.getRegionInfoForReplica(regionInfo, replicaId);
@@ -1098,8 +1103,7 @@ private static HRegionLocation getRegionLocation(final Result r, final RegionInf
/**
* Returns RegionInfo object from the column
- * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog
- * table Result.
+ * HConstants.CATALOG_FAMILY:HConstants.REGIONINFO_QUALIFIER of the catalog table Result.
* @param data a Result object from the catalog table scan
* @return RegionInfo or null
*/
@@ -1110,26 +1114,25 @@ public static RegionInfo getRegionInfo(Result data) {
/**
* Returns the RegionInfo object from the column {@link HConstants#CATALOG_FAMILY} and
* qualifier of the catalog table result.
- * @param r a Result object from the catalog table scan
+ * @param r a Result object from the catalog table scan
* @param qualifier Column family qualifier
* @return An RegionInfo instance or null.
*/
@Nullable
- public static RegionInfo getRegionInfo(final Result r, byte [] qualifier) {
+ public static RegionInfo getRegionInfo(final Result r, byte[] qualifier) {
Cell cell = r.getColumnLatestCell(getCatalogFamily(), qualifier);
if (cell == null) return null;
- return RegionInfo.parseFromOrNull(cell.getValueArray(),
- cell.getValueOffset(), cell.getValueLength());
+ return RegionInfo.parseFromOrNull(cell.getValueArray(), cell.getValueOffset(),
+ cell.getValueLength());
}
/**
* Fetch table state for given table from META table
- * @param conn connection to use
+ * @param conn connection to use
* @param tableName table to fetch state for
*/
@Nullable
- public static TableState getTableState(Connection conn, TableName tableName)
- throws IOException {
+ public static TableState getTableState(Connection conn, TableName tableName) throws IOException {
if (tableName.equals(TableName.META_TABLE_NAME)) {
return new TableState(tableName, TableState.State.ENABLED);
}
@@ -1144,8 +1147,7 @@ public static TableState getTableState(Connection conn, TableName tableName)
* @param conn connection to use
* @return map {tableName -> state}
*/
- public static Map getTableStates(Connection conn)
- throws IOException {
+ public static Map getTableStates(Connection conn) throws IOException {
final Map states = new LinkedHashMap<>();
Visitor collector = r -> {
TableState state = getTableState(r);
@@ -1159,19 +1161,17 @@ public static Map getTableStates(Connection conn)
}
/**
- * Updates state in META
- * Do not use. For internal use only.
- * @param conn connection to use
+ * Updates state in META Do not use. For internal use only.
+ * @param conn connection to use
* @param tableName table to look for
*/
- public static void updateTableState(Connection conn, TableName tableName,
- TableState.State actual) throws IOException {
+ public static void updateTableState(Connection conn, TableName tableName, TableState.State actual)
+ throws IOException {
updateTableState(conn, new TableState(tableName, actual));
}
/**
- * Decode table state from META Result.
- * Should contain cell from HConstants.TABLE_FAMILY
+ * Decode table state from META Result. Should contain cell from HConstants.TABLE_FAMILY
* @return null if not found
*/
@Nullable
@@ -1196,8 +1196,7 @@ public interface Visitor {
/**
* Visit the catalog table row.
* @param r A row from catalog table
- * @return True if we are to proceed scanning the table, else false if
- * we are to stop now.
+ * @return True if we are to proceed scanning the table, else false if we are to stop now.
*/
boolean visit(final Result r) throws IOException;
}
@@ -1213,6 +1212,7 @@ public interface CloseableVisitor extends Visitor, Closeable {
*/
static abstract class CollectingVisitor implements Visitor {
final List results = new ArrayList<>();
+
@Override
public boolean visit(Result r) throws IOException {
if (r != null && !r.isEmpty()) {
@@ -1223,10 +1223,7 @@ public boolean visit(Result r) throws IOException {
abstract void add(Result r);
- /**
- * @return Collected results; wait till visits complete to collect all
- * possible results
- */
+ /** Returns Collected results; wait till visits complete to collect all possible results */
List getResults() {
return this.results;
}
@@ -1260,7 +1257,7 @@ public boolean visit(Result rowResult) throws IOException {
return true;
}
- //skip over offline and split regions
+ // skip over offline and split regions
if (!(info.isOffline() || info.isSplit())) {
return visitInternal(rowResult);
}
@@ -1269,10 +1266,10 @@ public boolean visit(Result rowResult) throws IOException {
}
/**
- * A Visitor for a table. Provides a consistent view of the table's
- * hbase:meta entries during concurrent splits (see HBASE-5986 for details). This class
- * does not guarantee ordered traversal of meta entries, and can block until the
- * hbase:meta entries for daughters are available during splits.
+ * A Visitor for a table. Provides a consistent view of the table's hbase:meta entries during
+ * concurrent splits (see HBASE-5986 for details). This class does not guarantee ordered traversal
+ * of meta entries, and can block until the hbase:meta entries for daughters are available during
+ * splits.
*/
public static abstract class TableVisitorBase extends DefaultVisitorBase {
private TableName tableName;
@@ -1288,7 +1285,7 @@ public final boolean visit(Result rowResult) throws IOException {
if (info == null) {
return true;
}
- if (!(info.getTable().equals(tableName))) {
+ if (!info.getTable().equals(tableName)) {
return false;
}
return super.visit(rowResult);
@@ -1298,11 +1295,21 @@ public final boolean visit(Result rowResult) throws IOException {
////////////////////////
// Editing operations //
////////////////////////
+
+ /**
+ * Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.
+ * @throws IllegalArgumentException when the provided RegionInfo is not the default replica.
+ */
+ public static Put makePutFromRegionInfo(RegionInfo regionInfo) throws IOException {
+ return makePutFromRegionInfo(regionInfo, EnvironmentEdgeManager.currentTime());
+ }
+
/**
- * Generates and returns a Put containing the region into for the catalog table
+ * Generates and returns a {@link Put} containing the {@link RegionInfo} for the catalog table.
+ * @throws IllegalArgumentException when the provided RegionInfo is not the default replica.
*/
public static Put makePutFromRegionInfo(RegionInfo regionInfo, long ts) throws IOException {
- return addRegionInfo(new Put(regionInfo.getRegionName(), ts), regionInfo);
+ return addRegionInfo(new Put(getMetaKeyForRegion(regionInfo), ts), regionInfo);
}
/**
@@ -1312,7 +1319,11 @@ public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
if (regionInfo == null) {
throw new IllegalArgumentException("Can't make a delete for null region");
}
- Delete delete = new Delete(regionInfo.getRegionName());
+ if (regionInfo.getReplicaId() != RegionInfo.DEFAULT_REPLICA_ID) {
+ throw new IllegalArgumentException(
+ "Can't make delete for a replica region. Operate on the primary");
+ }
+ Delete delete = new Delete(getMetaKeyForRegion(regionInfo));
delete.addFamily(getCatalogFamily(), ts);
return delete;
}
@@ -1321,26 +1332,18 @@ public static Delete makeDeleteFromRegionInfo(RegionInfo regionInfo, long ts) {
* Adds split daughters to the Put
*/
private static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo splitB)
- throws IOException {
+ throws IOException {
if (splitA != null) {
- put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
- .setRow(put.getRow())
- .setFamily(HConstants.CATALOG_FAMILY)
- .setQualifier(HConstants.SPLITA_QUALIFIER)
- .setTimestamp(put.getTimestamp())
- .setType(Type.Put)
- .setValue(RegionInfo.toByteArray(splitA))
- .build());
+ put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+ .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITA_QUALIFIER)
+ .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
+ .setValue(RegionInfo.toByteArray(splitA)).build());
}
if (splitB != null) {
- put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
- .setRow(put.getRow())
- .setFamily(HConstants.CATALOG_FAMILY)
- .setQualifier(HConstants.SPLITB_QUALIFIER)
- .setTimestamp(put.getTimestamp())
- .setType(Type.Put)
- .setValue(RegionInfo.toByteArray(splitB))
- .build());
+ put.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(put.getRow())
+ .setFamily(HConstants.CATALOG_FAMILY).setQualifier(HConstants.SPLITB_QUALIFIER)
+ .setTimestamp(put.getTimestamp()).setType(Cell.Type.Put)
+ .setValue(RegionInfo.toByteArray(splitB)).build());
}
return put;
}
@@ -1348,7 +1351,7 @@ private static Put addDaughtersToPut(Put put, RegionInfo splitA, RegionInfo spli
/**
* Put the passed