You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by to...@apache.org on 2020/08/05 09:49:08 UTC
[hbase] 01/09: partially working
This is an automated email from the ASF dual-hosted git repository.
toffer pushed a commit to branch HBASE-11288.branch-2
in repository https://gitbox.apache.org/repos/asf/hbase.git
commit 280ed9f34b6368755edbec6ec879ab585cc068ea
Author: Francis Liu <to...@apache.org>
AuthorDate: Mon Jul 27 01:38:34 2020 -0700
partially working
---
.../hadoop/hbase/AsyncMetaTableAccessor.java | 6 +-
.../java/org/apache/hadoop/hbase/HRegionInfo.java | 35 ++-
.../org/apache/hadoop/hbase/MetaTableAccessor.java | 82 ++++--
...Locator.java => AsyncNonRootRegionLocator.java} | 56 +++-
.../hadoop/hbase/client/AsyncRegionLocator.java | 23 +-
...ionLocator.java => AsyncRootRegionLocator.java} | 4 +-
.../hbase/client/AsyncTableRegionLocatorImpl.java | 6 +-
.../hbase/client/ConnectionImplementation.java | 31 ++-
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 40 ++-
.../org/apache/hadoop/hbase/client/RegionInfo.java | 10 +
.../hadoop/hbase/client/RegionInfoBuilder.java | 30 +-
.../hadoop/hbase/client/ZKConnectionRegistry.java | 6 +-
.../hadoop/hbase/shaded/protobuf/ProtobufUtil.java | 4 +-
...ava => TestAsyncRootRegionLocatorFailFast.java} | 8 +-
.../apache/hadoop/hbase/CellComparatorImpl.java | 81 +++++-
.../java/org/apache/hadoop/hbase/KeyValue.java | 80 ++++++
.../java/org/apache/hadoop/hbase/TableName.java | 11 +
.../apache/hadoop/hbase/TestCellComparator.java | 69 +++++
.../src/main/protobuf/MasterProcedure.proto | 14 +
.../coordination/ZkSplitLogWorkerCoordination.java | 14 +-
.../hadoop/hbase/io/hfile/FixedFileTrailer.java | 12 +
.../hadoop/hbase/io/hfile/HFileWriterImpl.java | 4 +-
.../org/apache/hadoop/hbase/master/HMaster.java | 62 ++---
.../hadoop/hbase/master/MasterMetaBootstrap.java | 64 ++++-
.../hadoop/hbase/master/MasterRpcServices.java | 8 +-
.../hadoop/hbase/master/MasterStatusServlet.java | 2 +-
.../hadoop/hbase/master/MasterWalManager.java | 50 +++-
.../hbase/master/MetaRegionLocationCache.java | 2 +-
.../hadoop/hbase/master/SplitWALManager.java | 46 ++-
.../hbase/master/assignment/AssignmentManager.java | 310 +++++++++++++++++++--
.../assignment/RegionRemoteProcedureBase.java | 17 +-
.../hbase/master/assignment/RegionStateStore.java | 37 +--
.../hbase/master/assignment/RegionStates.java | 16 ++
.../hbase/master/assignment/ServerState.java | 15 +-
.../assignment/TransitRegionStateProcedure.java | 17 +-
.../master/procedure/HBCKServerCrashProcedure.java | 4 +-
.../hbase/master/procedure/InitRootProcedure.java | 193 +++++++++++++
.../master/procedure/MasterProcedureUtil.java | 8 +-
.../master/procedure/ServerCrashProcedure.java | 117 +++++++-
.../master/procedure/ServerProcedureInterface.java | 5 +
.../hbase/master/procedure/SplitWALProcedure.java | 4 +
.../master/procedure/SplitWALRemoteProcedure.java | 4 +
.../procedure/SwitchRpcThrottleProcedure.java | 4 +
.../SwitchRpcThrottleRemoteProcedure.java | 4 +
.../apache/hadoop/hbase/regionserver/HRegion.java | 19 +-
.../hadoop/hbase/util/FSTableDescriptors.java | 70 ++++-
.../hadoop/hbase/wal/AbstractFSWALProvider.java | 24 ++
.../hadoop/hbase/wal/AsyncFSWALProvider.java | 8 +-
.../wal/BoundedRecoveredHFilesOutputSink.java | 11 +-
.../apache/hadoop/hbase/wal/FSHLogProvider.java | 8 +-
.../hadoop/hbase/wal/RegionGroupingProvider.java | 20 +-
.../org/apache/hadoop/hbase/wal/WALFactory.java | 60 +++-
.../java/org/apache/hadoop/hbase/HBaseCluster.java | 7 +
.../apache/hadoop/hbase/TestMetaTableAccessor.java | 30 +-
.../apache/hadoop/hbase/TestMetaTableLocator.java | 20 +-
.../hbase/client/RegionReplicaTestHelper.java | 3 +-
...tor.java => TestAsyncNonRootRegionLocator.java} | 8 +-
...tAsyncNonRootRegionLocatorConcurrenyLimit.java} | 10 +-
...ocator.java => TestAsyncRootRegionLocator.java} | 10 +-
.../hbase/client/TestAsyncTableLocatePrefetch.java | 6 +-
.../TestAsyncTableLocateRegionForDeletedTable.java | 2 +-
.../hbase/client/TestAsyncTableRSCrashPublish.java | 2 +-
.../hbase/client/TestMetaRegionLocationCache.java | 2 +-
.../hadoop/hbase/client/TestMetaWithReplicas.java | 0
.../hadoop/hbase/http/TestInfoServersACL.java | 2 +-
.../apache/hadoop/hbase/master/TestDeadServer.java | 2 +-
.../hadoop/hbase/master/TestMasterFailover.java | 4 +-
.../hadoop/hbase/master/TestMasterNoCluster.java | 6 +-
.../hbase/master/TestMetaShutdownHandler.java | 4 +-
.../hbase/master/TestRegionsRecoveryChore.java | 9 +
.../hadoop/hbase/master/TestSplitWALManager.java | 6 +-
.../master/assignment/TestAssignmentManager.java | 10 +-
.../assignment/TestAssignmentManagerBase.java | 2 +
.../assignment/TestCloseRegionWhileRSCrash.java | 4 +
.../master/assignment/TestRegionStateStore.java | 6 +-
.../hadoop/hbase/master/procedure/TestSCP.java | 2 +-
.../procedure/TestServerRemoteProcedure.java | 4 +
.../hadoop/hbase/protobuf/TestProtobufUtil.java | 6 +-
.../hadoop/hbase/regionserver/TestHRegionInfo.java | 3 +-
.../regionserver/TestRegionServerNoMaster.java | 4 +-
.../regionserver/wal/TestLogRollingNoCluster.java | 2 +-
.../hadoop/hbase/zookeeper/MetaTableLocator.java | 24 +-
.../org/apache/hadoop/hbase/zookeeper/ZKUtil.java | 4 +-
83 files changed, 1721 insertions(+), 318 deletions(-)
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 b1fcd94..6529615 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
@@ -111,7 +111,7 @@ public class AsyncMetaTableAccessor {
CompletableFuture<Optional<HRegionLocation>> future = new CompletableFuture<>();
try {
RegionInfo parsedRegionInfo = MetaTableAccessor.parseRegionInfoFromRegionName(regionName);
- addListener(metaTable.get(new Get(MetaTableAccessor.getMetaKeyForRegion(parsedRegionInfo))
+ addListener(metaTable.get(new Get(MetaTableAccessor.getCatalogKeyForRegion(parsedRegionInfo))
.addFamily(HConstants.CATALOG_FAMILY)), (r, err) -> {
if (err != null) {
future.completeExceptionally(err);
@@ -206,13 +206,13 @@ public class AsyncMetaTableAccessor {
* @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}.
+ * {@link CompletableFuture}.META_TABLE_NAME
*/
private static CompletableFuture<List<Pair<RegionInfo, ServerName>>> getTableRegionsAndLocations(
final AsyncTable<AdvancedScanResultConsumer> metaTable,
final TableName tableName, final boolean excludeOfflinedSplitParents) {
CompletableFuture<List<Pair<RegionInfo, ServerName>>> future = new CompletableFuture<>();
- if (TableName.META_TABLE_NAME.equals(tableName)) {
+ if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
future.completeExceptionally(new IOException(
"This method can't be used to locate meta regions;" + " use MetaTableLocator instead"));
}
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 6edd3fa..80c63b5 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
@@ -184,6 +184,12 @@ public class HRegionInfo implements RegionInfo {
public final static byte[] HIDDEN_END_KEY = RegionInfoDisplay.HIDDEN_END_KEY;
public final static byte[] HIDDEN_START_KEY = RegionInfoDisplay.HIDDEN_START_KEY;
+ /** HRegionInfo for root region, we're using the new encoded format
+ * to try and reduce the amount of boilerplate code need to support the legacy format */
+ public static final HRegionInfo ROOT_REGIONINFO =
+ new HRegionInfo(TableName.ROOT_TABLE_NAME, HConstants.EMPTY_START_ROW,
+ HConstants.EMPTY_END_ROW, false, 0);
+
/** 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 =
@@ -581,8 +587,16 @@ public class HRegionInfo implements RegionInfo {
*/
@Override
public boolean containsRow(byte[] row) {
- return Bytes.compareTo(row, startKey) >= 0 &&
- (Bytes.compareTo(row, endKey) < 0 ||
+ return containsRow(row, 0, (short)row.length);
+ }
+
+ /**
+ * @return true if the given row falls in this region.
+ */
+ @Override
+ public boolean containsRow(byte[] row, int offset, short length) {
+ return Bytes.compareTo(row, offset, length, startKey, 0, startKey.length) >= 0 &&
+ (Bytes.compareTo(row, offset, length, endKey, 0, endKey.length) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
@@ -597,6 +611,14 @@ public class HRegionInfo implements RegionInfo {
* @return true if this region is a meta region
*/
@Override
+ public boolean isRootRegion() {
+ return tableName.equals(HRegionInfo.ROOT_REGIONINFO.getTable());
+ }
+
+ /**
+ * @return true if this region is a meta region
+ */
+ @Override
public boolean isMetaRegion() {
return tableName.equals(HRegionInfo.FIRST_META_REGIONINFO.getTable());
}
@@ -707,8 +729,13 @@ public class HRegionInfo implements RegionInfo {
*/
@Deprecated
public KVComparator getComparator() {
- return isMetaRegion()?
- KeyValue.META_COMPARATOR: KeyValue.COMPARATOR;
+ if (isRootRegion()) {
+ return KeyValue.ROOT_COMPARATOR;
+ }
+ if (isMetaRegion()) {
+ return KeyValue.META_COMPARATOR;
+ }
+ return KeyValue.COMPARATOR;
}
/**
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 dd0d1ff..fa90a36 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
@@ -192,6 +192,16 @@ public class MetaTableAccessor {
* @param connection connection we're using
* @param visitor Visitor invoked against each row in regions family.
*/
+ public static void fullScanRegions(Connection connection, boolean useRoot, final Visitor visitor)
+ throws IOException {
+ scanCatalogTable(connection, useRoot, null, null, QueryType.REGION, null, Integer.MAX_VALUE, visitor);
+ }
+
+ /**
+ * Performs a full scan of <code>hbase:meta</code> for regions.
+ * @param connection connection we're using
+ * @param visitor Visitor invoked against each row in regions family.
+ */
public static void fullScanRegions(Connection connection, final Visitor visitor)
throws IOException {
scanMeta(connection, null, null, QueryType.REGION, visitor);
@@ -232,7 +242,23 @@ public class MetaTableAccessor {
* @param connection connection we're using to access Meta
* @return An {@link Table} for <code>hbase:meta</code>
*/
- public static Table getMetaHTable(final Connection connection) throws IOException {
+ public static Table getRootHTable(final Connection connection)
+ throws IOException {
+ // We used to pass whole CatalogTracker in here, now we just pass in Connection
+ if (connection.isClosed()) {
+ throw new IOException("connection is closed");
+ }
+ return connection.getTable(TableName.ROOT_TABLE_NAME);
+ }
+
+ /**
+ * Callers should call close on the returned {@link Table} instance.
+ * @param connection connection we're using to access Meta
+ * @return An {@link Table} for <code>hbase:meta</code>
+ * @throws NullPointerException if {@code connection} is {@code null}
+ */
+ public static Table getMetaHTable(final Connection connection)
+ throws IOException {
// We used to pass whole CatalogTracker in here, now we just pass in Connection
if (connection == null) {
throw new NullPointerException("No connection");
@@ -283,7 +309,7 @@ public class MetaTableAccessor {
RegionInfo parsedInfo = null;
try {
parsedInfo = parseRegionInfoFromRegionName(regionName);
- row = getMetaKeyForRegion(parsedInfo);
+ row = getCatalogKeyForRegion(parsedInfo);
} catch (Exception parseEx) {
// Ignore. This is used with tableName passed as regionName.
}
@@ -312,13 +338,13 @@ public class MetaTableAccessor {
*/
public static Result getCatalogFamilyRow(Connection connection, RegionInfo ri)
throws IOException {
- Get get = new Get(getMetaKeyForRegion(ri));
+ Get get = new Get(getCatalogKeyForRegion(ri));
get.addFamily(HConstants.CATALOG_FAMILY);
return get(getMetaHTable(connection), get);
}
/** Returns the row key to use for this regionInfo */
- public static byte[] getMetaKeyForRegion(RegionInfo regionInfo) {
+ public static byte[] getCatalogKeyForRegion(RegionInfo regionInfo) {
return RegionReplicaUtil.getRegionInfoForDefaultReplica(regionInfo).getRegionName();
}
@@ -348,9 +374,18 @@ public class MetaTableAccessor {
*/
public static Result getRegionResult(Connection connection,
byte[] regionName) throws IOException {
+ Table catalogTable = null;
+ if (Bytes.equals(RegionInfoBuilder.ROOT_REGIONINFO.getRegionName(), regionName)) {
+ throw new IllegalStateException("This method cannot be used for hbase:root region");
+ }
+ if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), regionName)) {
+ catalogTable = getRootHTable(connection);
+ } else {
+ catalogTable = getMetaHTable(connection);
+ }
Get get = new Get(regionName);
get.addFamily(HConstants.CATALOG_FAMILY);
- return get(getMetaHTable(connection), get);
+ return get(catalogTable, get);
}
/**
@@ -366,10 +401,17 @@ public class MetaTableAccessor {
String regionEncodedName) throws IOException {
RowFilter rowFilter = new RowFilter(CompareOperator.EQUAL,
new SubstringComparator(regionEncodedName));
- Scan scan = getMetaScan(connection, 1);
+ Scan scan = getCatalogScan(connection, 1);
scan.setFilter(rowFilter);
ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan);
- return resultScanner.next();
+ Result res = resultScanner.next();
+ if (res == null) {
+ scan = getCatalogScan(connection, 1);
+ scan.setFilter(rowFilter);
+ resultScanner = getRootHTable(connection).getScanner(scan);
+ res = resultScanner.next();
+ }
+ return res;
}
/**
@@ -584,13 +626,13 @@ public class MetaTableAccessor {
// Stop key appends the smallest possible char to the table name
byte[] stopKey = getTableStopRowForMeta(tableName, QueryType.REGION);
- Scan scan = getMetaScan(connection, -1);
+ Scan scan = getCatalogScan(connection, -1);
scan.setStartRow(startKey);
scan.setStopRow(stopKey);
return scan;
}
- private static Scan getMetaScan(Connection connection, int rowUpperLimit) {
+ private static Scan getCatalogScan(Connection connection, int rowUpperLimit) {
Scan scan = new Scan();
int scannerCaching = connection.getConfiguration()
.getInt(HConstants.HBASE_META_SCANNER_CACHING,
@@ -782,14 +824,15 @@ public class MetaTableAccessor {
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);
+ scanCatalogTable(connection, false, startRow, stopRow, type, null, maxRows, visitor);
}
- private static void scanMeta(Connection connection, @Nullable final byte[] startRow,
- @Nullable final byte[] stopRow, QueryType type, @Nullable Filter filter, int maxRows,
+ private static void scanCatalogTable(Connection connection, boolean useRoot,
+ @Nullable final byte[] startRow, @Nullable final byte[] stopRow,
+ QueryType type, @Nullable Filter filter, int maxRows,
final Visitor visitor) throws IOException {
int rowUpperLimit = maxRows > 0 ? maxRows : Integer.MAX_VALUE;
- Scan scan = getMetaScan(connection, rowUpperLimit);
+ Scan scan = getCatalogScan(connection, rowUpperLimit);
for (byte[] family : type.getFamilies()) {
scan.addFamily(family);
@@ -811,8 +854,8 @@ public class MetaTableAccessor {
}
int currentRow = 0;
- try (Table metaTable = getMetaHTable(connection)) {
- try (ResultScanner scanner = metaTable.getScanner(scan)) {
+ try (Table catalogTable = useRoot ? getRootHTable(connection) : getMetaHTable(connection)) {
+ try (ResultScanner scanner = catalogTable.getScanner(scan)) {
Result data;
while ((data = scanner.next()) != null) {
if (data.isEmpty()) continue;
@@ -839,7 +882,7 @@ public class MetaTableAccessor {
private static RegionInfo getClosestRegionInfo(Connection connection,
@NonNull final TableName tableName, @NonNull final byte[] row) throws IOException {
byte[] searchRow = RegionInfo.createRegionName(tableName, row, HConstants.NINES, false);
- Scan scan = getMetaScan(connection, 1);
+ Scan scan = getCatalogScan(connection, 1);
scan.setReversed(true);
scan.withStartRow(searchRow);
try (ResultScanner resultScanner = getMetaHTable(connection).getScanner(scan)) {
@@ -1153,7 +1196,8 @@ public class MetaTableAccessor {
@Nullable
public static TableState getTableState(Connection conn, TableName tableName)
throws IOException {
- if (tableName.equals(TableName.META_TABLE_NAME)) {
+ if (tableName.equals(TableName.ROOT_TABLE_NAME) ||
+ tableName.equals(TableName.META_TABLE_NAME)) {
return new TableState(tableName, TableState.State.ENABLED);
}
Table metaHTable = getMetaHTable(conn);
@@ -1847,7 +1891,7 @@ public class MetaTableAccessor {
private static void updateLocation(Connection connection, RegionInfo regionInfo, ServerName sn,
long openSeqNum, long masterSystemTime) throws IOException {
// region replicas are kept in the primary region's row
- Put put = new Put(getMetaKeyForRegion(regionInfo), masterSystemTime);
+ Put put = new Put(getCatalogKeyForRegion(regionInfo), masterSystemTime);
addRegionInfo(put, regionInfo);
addLocation(put, sn, openSeqNum, regionInfo.getReplicaId());
putToMetaTable(connection, put);
@@ -2209,7 +2253,7 @@ public class MetaTableAccessor {
public static List<String> getTableEncodedRegionNamesForSerialReplication(Connection conn,
TableName tableName) throws IOException {
List<String> list = new ArrayList<>();
- scanMeta(conn, getTableStartRowForMeta(tableName, QueryType.REPLICATION),
+ scanCatalogTable(conn, false, getTableStartRowForMeta(tableName, QueryType.REPLICATION),
getTableStopRowForMeta(tableName, QueryType.REPLICATION), QueryType.REPLICATION,
new FirstKeyOnlyFilter(), Integer.MAX_VALUE, r -> {
list.add(RegionInfo.encodeRegionName(r.getRow()));
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
similarity index 92%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
index 5cdacf0..f082778 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncNonRootRegionLocator.java
@@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.HConstants.NINES;
import static org.apache.hadoop.hbase.HConstants.USE_META_REPLICAS;
import static org.apache.hadoop.hbase.HConstants.ZEROES;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.canUpdateOnError;
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.createRegionLocations;
import static org.apache.hadoop.hbase.client.AsyncRegionLocatorHelper.isGood;
@@ -35,6 +36,7 @@ import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
import java.io.IOException;
import java.util.Arrays;
+import java.util.Comparator;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
@@ -47,9 +49,12 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentNavigableMap;
import java.util.concurrent.ConcurrentSkipListMap;
import org.apache.commons.lang3.ObjectUtils;
+import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
@@ -68,9 +73,9 @@ import org.apache.hbase.thirdparty.com.google.common.base.Objects;
* The asynchronous locator for regions other than meta.
*/
@InterfaceAudience.Private
-class AsyncNonMetaRegionLocator {
+class AsyncNonRootRegionLocator {
- private static final Logger LOG = LoggerFactory.getLogger(AsyncNonMetaRegionLocator.class);
+ private static final Logger LOG = LoggerFactory.getLogger(AsyncNonRootRegionLocator.class);
@VisibleForTesting
static final String MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE =
@@ -121,8 +126,15 @@ class AsyncNonMetaRegionLocator {
private static final class TableCache {
- private final ConcurrentNavigableMap<byte[], RegionLocations> cache =
- new ConcurrentSkipListMap<>(BYTES_COMPARATOR);
+ private TableCache(TableName tableName) {
+ final KeyValue.KVComparator comparator = getComparator(tableName);
+ cache = new ConcurrentSkipListMap<>(new Comparator<byte[]>() {
+ @Override public int compare(byte[] left, byte[] right) {
+ return comparator.compareRows(left, 0, left.length, right, 0, right.length);
+ }
+ });
+ }
+ private final ConcurrentNavigableMap<byte[], RegionLocations> cache;
private final Set<LocateRequest> pendingRequests = new HashSet<>();
@@ -174,10 +186,14 @@ class AsyncNonMetaRegionLocator {
// startKey < req.row and endKey >= req.row. Here we split it to endKey == req.row ||
// (endKey > req.row && startKey < req.row). The two conditions are equal since startKey <
// endKey.
+ KeyValue.KVComparator comparator = getComparator(loc.getRegion().getTable());
byte[] endKey = loc.getRegion().getEndKey();
- int c = Bytes.compareTo(endKey, req.row);
+ int c = comparator.compareRows(endKey, 0, endKey.length,
+ req.row,0, req.row.length);
completed = c == 0 || ((c > 0 || Bytes.equals(EMPTY_END_ROW, endKey)) &&
- Bytes.compareTo(loc.getRegion().getStartKey(), req.row) < 0);
+ comparator.compareRows(
+ loc.getRegion().getStartKey(), 0, loc.getRegion().getStartKey().length,
+ req.row, 0, req.row.length) < 0);
} else {
completed = loc.getRegion().containsRow(req.row);
}
@@ -190,7 +206,7 @@ class AsyncNonMetaRegionLocator {
}
}
- AsyncNonMetaRegionLocator(AsyncConnectionImpl conn) {
+ AsyncNonRootRegionLocator(AsyncConnectionImpl conn) {
this.conn = conn;
this.maxConcurrentLocateRequestPerTable = conn.getConfiguration().getInt(
MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE, DEFAULT_MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE);
@@ -201,7 +217,7 @@ class AsyncNonMetaRegionLocator {
}
private TableCache getTableCache(TableName tableName) {
- return computeIfAbsent(cache, tableName, TableCache::new);
+ return computeIfAbsent(cache, tableName, ()-> new TableCache(tableName));
}
private boolean isEqual(RegionLocations locs1, RegionLocations locs2) {
@@ -369,8 +385,10 @@ class AsyncNonMetaRegionLocator {
recordCacheMiss();
return null;
}
+ KeyValue.KVComparator comparator = getComparator(tableName);
byte[] endKey = loc.getRegion().getEndKey();
- if (isEmptyStopRow(endKey) || Bytes.compareTo(row, endKey) < 0) {
+ if (isEmptyStopRow(endKey) ||
+ comparator.compareRows(row, 0, row.length, endKey, 0, endKey.length) < 0) {
if (LOG.isTraceEnabled()) {
LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
Bytes.toStringBinary(row), RegionLocateType.CURRENT, replicaId);
@@ -398,8 +416,12 @@ class AsyncNonMetaRegionLocator {
recordCacheMiss();
return null;
}
+ KeyValue.KVComparator comparator = getComparator(tableName);
if (isEmptyStopRow(loc.getRegion().getEndKey()) ||
- (!isEmptyStopRow && Bytes.compareTo(loc.getRegion().getEndKey(), row) >= 0)) {
+ (!isEmptyStopRow &&
+ comparator.compareRows(
+ loc.getRegion().getEndKey(), 0, loc.getRegion().getEndKey().length,
+ row, 0, row.length) >= 0)) {
if (LOG.isTraceEnabled()) {
LOG.trace("Found {} in cache for {}, row='{}', locateType={}, replicaId={}", loc, tableName,
Bytes.toStringBinary(row), RegionLocateType.BEFORE, replicaId);
@@ -413,6 +435,8 @@ class AsyncNonMetaRegionLocator {
}
private void locateInMeta(TableName tableName, LocateRequest req) {
+ TableName parentTableName =
+ META_TABLE_NAME.equals(tableName) ? ROOT_TABLE_NAME : META_TABLE_NAME;
if (LOG.isTraceEnabled()) {
LOG.trace("Try locate '" + tableName + "', row='" + Bytes.toStringBinary(req.row) +
"', locateType=" + req.locateType + " in meta");
@@ -436,7 +460,7 @@ class AsyncNonMetaRegionLocator {
if (useMetaReplicas) {
scan.setConsistency(Consistency.TIMELINE);
}
- conn.getTable(META_TABLE_NAME).scan(scan, new AdvancedScanResultConsumer() {
+ conn.getTable(parentTableName).scan(scan, new AdvancedScanResultConsumer() {
private boolean completeNormally = false;
@@ -657,4 +681,14 @@ class AsyncNonMetaRegionLocator {
}
return locateRowInCache(tableCache, tableName, row, RegionReplicaUtil.DEFAULT_REPLICA_ID);
}
+
+ private static KeyValue.KVComparator getComparator(TableName tableName) {
+ if (TableName.ROOT_TABLE_NAME.equals(tableName)) {
+ return KeyValue.ROOT_COMPARATOR;
+ }
+ if (META_TABLE_NAME.equals(tableName)) {
+ return KeyValue.META_COMPARATOR;
+ }
+ return KeyValue.COMPARATOR;
+ }
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
index 9e1d5e8..4f1f6e7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRegionLocator.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.util.concurrent.CompletableFuture;
@@ -50,14 +51,14 @@ class AsyncRegionLocator {
private final AsyncConnectionImpl conn;
- private final AsyncMetaRegionLocator metaRegionLocator;
+ private final AsyncRootRegionLocator metaRegionLocator;
- private final AsyncNonMetaRegionLocator nonMetaRegionLocator;
+ private final AsyncNonRootRegionLocator nonMetaRegionLocator;
AsyncRegionLocator(AsyncConnectionImpl conn, HashedWheelTimer retryTimer) {
this.conn = conn;
- this.metaRegionLocator = new AsyncMetaRegionLocator(conn.registry);
- this.nonMetaRegionLocator = new AsyncNonMetaRegionLocator(conn);
+ this.metaRegionLocator = new AsyncRootRegionLocator(conn.registry);
+ this.nonMetaRegionLocator = new AsyncNonRootRegionLocator(conn);
this.retryTimer = retryTimer;
}
@@ -81,13 +82,17 @@ class AsyncRegionLocator {
return future;
}
+ private boolean isRoot(TableName tableName) {
+ return TableName.isRootTableName(tableName);
+ }
+
private boolean isMeta(TableName tableName) {
return TableName.isMetaTableName(tableName);
}
CompletableFuture<RegionLocations> getRegionLocations(TableName tableName, byte[] row,
RegionLocateType type, boolean reload, long timeoutNs) {
- CompletableFuture<RegionLocations> future = isMeta(tableName)
+ CompletableFuture<RegionLocations> future = isRoot(tableName)
? metaRegionLocator.getRegionLocations(RegionReplicaUtil.DEFAULT_REPLICA_ID, reload)
: nonMetaRegionLocator.getRegionLocations(tableName, row,
RegionReplicaUtil.DEFAULT_REPLICA_ID, type, reload);
@@ -103,7 +108,7 @@ class AsyncRegionLocator {
// Change it later if the meta table can have more than one regions.
CompletableFuture<HRegionLocation> future = new CompletableFuture<>();
CompletableFuture<RegionLocations> locsFuture =
- isMeta(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
+ isRoot(tableName) ? metaRegionLocator.getRegionLocations(replicaId, reload)
: nonMetaRegionLocator.getRegionLocations(tableName, row, replicaId, type, reload);
addListener(locsFuture, (locs, error) -> {
if (error != null) {
@@ -147,7 +152,7 @@ class AsyncRegionLocator {
}
void updateCachedLocationOnError(HRegionLocation loc, Throwable exception) {
- if (loc.getRegion().isMetaRegion()) {
+ if (loc.getRegion().isRootRegion()) {
metaRegionLocator.updateCachedLocationOnError(loc, exception);
} else {
nonMetaRegionLocator.updateCachedLocationOnError(loc, exception);
@@ -156,7 +161,7 @@ class AsyncRegionLocator {
void clearCache(TableName tableName) {
LOG.debug("Clear meta cache for {}", tableName);
- if (tableName.equals(META_TABLE_NAME)) {
+ if (tableName.equals(ROOT_TABLE_NAME)) {
metaRegionLocator.clearCache();
} else {
nonMetaRegionLocator.clearCache(tableName);
@@ -176,7 +181,7 @@ class AsyncRegionLocator {
}
@VisibleForTesting
- AsyncNonMetaRegionLocator getNonMetaRegionLocator() {
+ AsyncNonRootRegionLocator getNonMetaRegionLocator() {
return nonMetaRegionLocator;
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
similarity index 98%
rename from hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
rename to hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
index 9df8efb..fec7048 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncMetaRegionLocator.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncRootRegionLocator.java
@@ -34,7 +34,7 @@ import org.apache.yetus.audience.InterfaceAudience;
* The asynchronous locator for meta region.
*/
@InterfaceAudience.Private
-class AsyncMetaRegionLocator {
+class AsyncRootRegionLocator {
private final ConnectionRegistry registry;
@@ -43,7 +43,7 @@ class AsyncMetaRegionLocator {
private final AtomicReference<CompletableFuture<RegionLocations>> metaRelocateFuture =
new AtomicReference<>();
- AsyncMetaRegionLocator(ConnectionRegistry registry) {
+ AsyncRootRegionLocator(ConnectionRegistry registry) {
this.registry = registry;
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
index fa3ea1c..146ea12 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncTableRegionLocatorImpl.java
@@ -54,10 +54,14 @@ class AsyncTableRegionLocatorImpl implements AsyncTableRegionLocator {
@Override
public CompletableFuture<List<HRegionLocation>> getAllRegionLocations() {
- if (TableName.isMetaTableName(tableName)) {
+ if (TableName.isRootTableName(tableName)) {
return conn.registry.getMetaRegionLocations()
.thenApply(locs -> Arrays.asList(locs.getRegionLocations()));
}
+ if (TableName.isMetaTableName(tableName)) {
+ return AsyncMetaTableAccessor.getTableHRegionLocations(
+ conn.getTable(TableName.ROOT_TABLE_NAME), tableName);
+ }
return AsyncMetaTableAccessor.getTableHRegionLocations(conn.getTable(TableName.META_TABLE_NAME),
tableName);
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
index 06e243b..4d15bf9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java
@@ -757,7 +757,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
// Since this is an explicit request not to use any caching, finding
// disabled tables should not be desirable. This will ensure that an exception is thrown when
// the first time a disabled table is interacted with.
- if (!tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) {
+ if (!tableName.equals(TableName.META_TABLE_NAME) &&
+ !tableName.equals(TableName.META_TABLE_NAME) && isTableDisabled(tableName)) {
throw new TableNotEnabledException(tableName.getNameAsString() + " is disabled.");
}
@@ -777,15 +778,15 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
if (tableName == null || tableName.getName().length == 0) {
throw new IllegalArgumentException("table name cannot be null or zero length");
}
- if (tableName.equals(TableName.META_TABLE_NAME)) {
- return locateMeta(tableName, useCache, replicaId);
+ if (tableName.equals(TableName.ROOT_TABLE_NAME)) {
+ return locateRoot(tableName, useCache, replicaId);
} else {
// Region not in the cache - have to go to the meta RS
return locateRegionInMeta(tableName, row, useCache, retry, replicaId);
}
}
- private RegionLocations locateMeta(final TableName tableName,
+ private RegionLocations locateRoot(final TableName tableName,
boolean useCache, int replicaId) throws IOException {
// HBASE-10785: We cache the location of the META itself, so that we are not overloading
// zookeeper with one request for every region lookup. We cache the META with empty row
@@ -833,6 +834,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return locations;
}
}
+
+ TableName parentTable = TableName.META_TABLE_NAME;
+ if (TableName.META_TABLE_NAME.equals(tableName)) {
+ parentTable = TableName.ROOT_TABLE_NAME;
+ }
+
// build the key of the meta region we should be looking for.
// the extra 9's on the end are necessary to allow "exact" matches
// without knowing the precise region names.
@@ -846,7 +853,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
s.setConsistency(Consistency.TIMELINE);
}
int maxAttempts = (retry ? numTries : 1);
- boolean relocateMeta = false;
+ boolean relocateParent = false;
for (int tries = 0; ; tries++) {
if (tries >= maxAttempts) {
throw new NoServerForRegionException("Unable to find region for "
@@ -873,13 +880,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
return locations;
}
}
- if (relocateMeta) {
- relocateRegion(TableName.META_TABLE_NAME, HConstants.EMPTY_START_ROW,
+ if (relocateParent) {
+ relocateRegion(parentTable, HConstants.EMPTY_START_ROW,
RegionInfo.DEFAULT_REPLICA_ID);
}
s.resetMvccReadPoint();
try (ReversedClientScanner rcs =
- new ReversedClientScanner(conf, s, TableName.META_TABLE_NAME, this, rpcCallerFactory,
+ new ReversedClientScanner(conf, s, parentTable, this, rpcCallerFactory,
rpcControllerFactory, getMetaLookupPool(), metaReplicaCallTimeoutScanInMicroSecond)) {
boolean tableNotFound = true;
for (;;) {
@@ -900,7 +907,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
RegionInfo regionInfo = locations.getRegionLocation(replicaId).getRegion();
if (regionInfo == null) {
- throw new IOException("RegionInfo null or empty in " + TableName.META_TABLE_NAME +
+ throw new IOException("RegionInfo null or empty in " + parentTable +
", row=" + regionInfoRow);
}
// See HBASE-20182. It is possible that we locate to a split parent even after the
@@ -922,7 +929,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
ServerName serverName = locations.getRegionLocation(replicaId).getServerName();
if (serverName == null) {
throw new NoServerForRegionException("No server address listed in " +
- TableName.META_TABLE_NAME + " for region " + regionInfo.getRegionNameAsString() +
+ parentTable + " for region " + regionInfo.getRegionNameAsString() +
" containing row " + Bytes.toStringBinary(row));
}
if (isDeadServer(serverName)) {
@@ -951,12 +958,12 @@ class ConnectionImplementation implements ClusterConnection, Closeable {
}
if (tries < maxAttempts - 1) {
LOG.debug("locateRegionInMeta parentTable='{}', attempt={} of {} failed; retrying " +
- "after sleep of {}", TableName.META_TABLE_NAME, tries, maxAttempts, maxAttempts, e);
+ "after sleep of {}", parentTable, tries, maxAttempts, maxAttempts, e);
} else {
throw e;
}
// Only relocate the parent region if necessary
- relocateMeta =
+ relocateParent =
!(e instanceof RegionOfflineException || e instanceof NoServerForRegionException);
} finally {
userRegionLock.unlock();
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
index 023e8fc..3b0ede9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RawAsyncHBaseAdmin.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.HConstants.HIGH_QOS;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import static org.apache.hadoop.hbase.util.FutureUtils.unwrapCompletionException;
import com.google.protobuf.Message;
@@ -55,6 +56,7 @@ import org.apache.hadoop.hbase.ClusterMetrics;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.ClusterMetricsBuilder;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.MetaTableAccessor.QueryType;
@@ -322,6 +324,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private final HashedWheelTimer retryTimer;
+ private final AsyncTable<AdvancedScanResultConsumer> rootTable;
+
private final AsyncTable<AdvancedScanResultConsumer> metaTable;
private final long rpcTimeoutNs;
@@ -338,10 +342,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
private final NonceGenerator ng;
+ private final int metaReplicasNum;
+
RawAsyncHBaseAdmin(AsyncConnectionImpl connection, HashedWheelTimer retryTimer,
AsyncAdminBuilderBase builder) {
this.connection = connection;
this.retryTimer = retryTimer;
+ this.rootTable = connection.getTable(ROOT_TABLE_NAME);
this.metaTable = connection.getTable(META_TABLE_NAME);
this.rpcTimeoutNs = builder.rpcTimeoutNs;
this.operationTimeoutNs = builder.operationTimeoutNs;
@@ -359,6 +366,8 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
this.maxAttempts = builder.maxAttempts;
this.startLogErrorsCnt = builder.startLogErrorsCnt;
this.ng = connection.getNonceGenerator();
+ this.metaReplicasNum = connection.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
+ HConstants.DEFAULT_META_REPLICA_NUM);
}
private <T> MasterRequestCallerBuilder<T> newMasterCaller() {
@@ -2351,11 +2360,27 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
CompletableFuture<Optional<HRegionLocation>> future;
if (RegionInfo.isEncodedRegionName(regionNameOrEncodedRegionName)) {
String encodedName = Bytes.toString(regionNameOrEncodedRegionName);
- if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
+
+ //TODO francis do we really need to support root replica encoded name lookup?
+ //if so can we make this better?
+ //The same goes for meta when it's split?
+ boolean isRootRegion = false;
+ for (int i=0; i<metaReplicasNum; i++) {
+ RegionInfo regionInfo =
+ RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.ROOT_REGIONINFO, i);
+ if (encodedName.equals(regionInfo.getEncodedName())) {
+ isRootRegion = true;
+ break;
+ }
+ }
+ if (isRootRegion) {
// old format encodedName, should be meta region
future = connection.registry.getMetaRegionLocations()
.thenApply(locs -> Stream.of(locs.getRegionLocations())
.filter(loc -> loc.getRegion().getEncodedName().equals(encodedName)).findFirst());
+ } else if (encodedName.length() < RegionInfo.MD5_HEX_LENGTH) {
+ future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(rootTable,
+ regionNameOrEncodedRegionName);
} else {
future = AsyncMetaTableAccessor.getRegionLocationWithEncodedName(metaTable,
regionNameOrEncodedRegionName);
@@ -2363,11 +2388,15 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
} else {
RegionInfo regionInfo =
MetaTableAccessor.parseRegionInfoFromRegionName(regionNameOrEncodedRegionName);
- if (regionInfo.isMetaRegion()) {
+ if (regionInfo.isRootRegion()) {
future = connection.registry.getMetaRegionLocations()
.thenApply(locs -> Stream.of(locs.getRegionLocations())
.filter(loc -> loc.getRegion().getReplicaId() == regionInfo.getReplicaId())
.findFirst());
+ //TODO francis it won't reach here once meta is split
+ } else if (regionInfo.isMetaRegion()) {
+ future =
+ AsyncMetaTableAccessor.getRegionLocation(rootTable, regionNameOrEncodedRegionName);
} else {
future =
AsyncMetaTableAccessor.getRegionLocation(metaTable, regionNameOrEncodedRegionName);
@@ -2406,6 +2435,13 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
}
if (Bytes.equals(regionNameOrEncodedRegionName,
+ RegionInfoBuilder.ROOT_REGIONINFO.getRegionName()) ||
+ Bytes.equals(regionNameOrEncodedRegionName,
+ RegionInfoBuilder.ROOT_REGIONINFO.getEncodedNameAsBytes())) {
+ return CompletableFuture.completedFuture(RegionInfoBuilder.ROOT_REGIONINFO);
+ }
+
+ if (Bytes.equals(regionNameOrEncodedRegionName,
RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
Bytes.equals(regionNameOrEncodedRegionName,
RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
index dbda4a5..07ff8bd 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfo.java
@@ -224,6 +224,11 @@ public interface RegionInfo extends Comparable<RegionInfo> {
/**
* @return true if this region is a meta region.
*/
+ boolean isRootRegion();
+
+ /**
+ * @return true if this region is a meta region.
+ */
boolean isMetaRegion();
/**
@@ -241,6 +246,11 @@ public interface RegionInfo extends Comparable<RegionInfo> {
boolean containsRow(byte[] row);
/**
+ * @return true if the given row falls in this region.
+ */
+ boolean containsRow(byte[] row, int offset, short length);
+
+ /**
* Does region name contain its encoded name?
* @param regionName region name
* @return boolean indicating if this a new format region
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
index 34221f3..b70bbe4 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RegionInfoBuilder.java
@@ -38,6 +38,19 @@ public class RegionInfoBuilder {
public static final String NO_HASH = null;
/**
+ * RegionInfo for first root region
+ * You cannot use this builder to make an instance of the {@link #ROOT_REGIONINFO}.
+ * Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
+ * just RI so the mutable methods are not available (unless you go casting); it appears
+ * as immutable (I tried adding Immutable type but it just makes a mess).
+ *
+ * We are using the non-legacy encoding format to reduce the boilerplace code
+ */
+ // TODO: How come Root regions still do not have encoded region names? Fix.
+ public static final RegionInfo ROOT_REGIONINFO =
+ new MutableRegionInfo(0, TableName.ROOT_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
+
+ /**
* RegionInfo for first meta region
* You cannot use this builder to make an instance of the {@link #FIRST_META_REGIONINFO}.
* Just refer to this instance. Also, while the instance is actually a MutableRI, its type is
@@ -307,18 +320,29 @@ public class RegionInfoBuilder {
return firstKeyInRange && lastKeyInRange;
}
+ @Override
+ public boolean containsRow(byte[] row) {
+ return containsRow(row, 0, (short)row.length);
+ }
+
/**
* Return 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 ||
+ public boolean containsRow(byte[] row, int offset, short length) {
+ return Bytes.compareTo(row, 0, row.length, startKey, 0, startKey.length) >= 0 &&
+ (Bytes.compareTo(row, 0, row.length, endKey, 0, endKey.length) < 0 ||
Bytes.equals(endKey, HConstants.EMPTY_BYTE_ARRAY));
}
/** @return true if this region is a meta region */
@Override
+ public boolean isRootRegion() {
+ return tableName.equals(ROOT_REGIONINFO.getTable());
+ }
+
+ /** @return true if this region is a meta region */
+ @Override
public boolean isMetaRegion() {
return tableName.equals(FIRST_META_REGIONINFO.getTable());
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
index 42a4188..c0e09f9 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZKConnectionRegistry.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.client;
import static org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
-import static org.apache.hadoop.hbase.client.RegionInfoBuilder.FIRST_META_REGIONINFO;
+import static org.apache.hadoop.hbase.client.RegionInfoBuilder.ROOT_REGIONINFO;
import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForDefaultReplica;
import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica;
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
@@ -158,7 +158,7 @@ class ZKConnectionRegistry implements ConnectionRegistry {
LOG.warn("Meta region is in state " + stateAndServerName.getFirst());
}
locs[DEFAULT_REPLICA_ID] = new HRegionLocation(
- getRegionInfoForDefaultReplica(FIRST_META_REGIONINFO), stateAndServerName.getSecond());
+ getRegionInfoForDefaultReplica(ROOT_REGIONINFO), stateAndServerName.getSecond());
tryComplete(remaining, locs, future);
});
} else {
@@ -180,7 +180,7 @@ class ZKConnectionRegistry implements ConnectionRegistry {
locs[replicaId] = null;
} else {
locs[replicaId] =
- new HRegionLocation(getRegionInfoForReplica(FIRST_META_REGIONINFO, replicaId),
+ new HRegionLocation(getRegionInfoForReplica(ROOT_REGIONINFO, replicaId),
stateAndServerName.getSecond());
}
}
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
index 09db446..ff3c4a7 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/shaded/protobuf/ProtobufUtil.java
@@ -3084,7 +3084,7 @@ public final class ProtobufUtil {
* @return RegionState instance corresponding to the serialized data.
* @throws DeserializationException if the data is invalid.
*/
- public static RegionState parseMetaRegionStateFrom(final byte[] data, int replicaId)
+ public static RegionState parseRootRegionStateFrom(final byte[] data, int replicaId)
throws DeserializationException {
RegionState.State state = RegionState.State.OPEN;
ServerName serverName;
@@ -3111,7 +3111,7 @@ public final class ProtobufUtil {
state = RegionState.State.OFFLINE;
}
return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+ RegionInfoBuilder.ROOT_REGIONINFO, replicaId), state, serverName);
}
/**
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
similarity index 90%
rename from hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
rename to hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
index b306500..af23fb9 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocatorFailFast.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocatorFailFast.java
@@ -33,15 +33,15 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ ClientTests.class, SmallTests.class })
-public class TestAsyncMetaRegionLocatorFailFast {
+public class TestAsyncRootRegionLocatorFailFast {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncMetaRegionLocatorFailFast.class);
+ HBaseClassTestRule.forClass(TestAsyncRootRegionLocatorFailFast.class);
private static Configuration CONF = HBaseConfiguration.create();
- private static AsyncMetaRegionLocator LOCATOR;
+ private static AsyncRootRegionLocator LOCATOR;
private static final class FaultyConnectionRegistry extends DoNothingConnectionRegistry {
@@ -57,7 +57,7 @@ public class TestAsyncMetaRegionLocatorFailFast {
@BeforeClass
public static void setUp() {
- LOCATOR = new AsyncMetaRegionLocator(new FaultyConnectionRegistry(CONF));
+ LOCATOR = new AsyncRootRegionLocator(new FaultyConnectionRegistry(CONF));
}
@Test(expected = DoNotRetryIOException.class)
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
index e6c8e3d..ac84c2a 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/CellComparatorImpl.java
@@ -60,6 +60,12 @@ public class CellComparatorImpl implements CellComparator {
* A {@link CellComparatorImpl} for <code>hbase:meta</code> catalog table
* {@link KeyValue}s.
*/
+ public static final CellComparatorImpl ROOT_COMPARATOR = new RootCellComparator();
+
+ /**
+ * A {@link CellComparatorImpl} for <code>hbase:meta</code> catalog table
+ * {@link KeyValue}s.
+ */
public static final CellComparatorImpl META_COMPARATOR = new MetaCellComparator();
@Override
@@ -292,6 +298,72 @@ public class CellComparatorImpl implements CellComparator {
}
/**
+ * A {@link CellComparatorImpl} for <code>hbase:root</code> catalog table
+ * {@link KeyValue}s.
+ */
+ public static class RootCellComparator extends MetaCellComparator {
+
+ private static int compareRows(byte[] left, int loffset, int llength, byte[] right, int roffset,
+ int rlength) {
+ // Rows look like this: hbase:meta,ROW_FROM_META,RID
+ // LOG.info("ROOT " + Bytes.toString(left, loffset, llength) +
+ // "---" + Bytes.toString(right, roffset, rlength));
+ final int metalength = TableName.META_TABLE_NAME.getName().length+1; // 'hbase:meta,' length
+ int leftDelimiter = loffset+metalength;
+ int rightDelimiter = roffset+metalength;
+ int leftFarDelimiter = Bytes.searchDelimiterIndexInReverse(left,
+ leftDelimiter,
+ llength - metalength, HConstants.DELIMITER);
+ int rightFarDelimiter = Bytes.searchDelimiterIndexInReverse(right,
+ rightDelimiter, rlength - metalength,
+ HConstants.DELIMITER);
+ if (leftFarDelimiter < 0 && rightFarDelimiter >= 0) {
+ // Nothing between hbase:meta and regionid. Its first key.
+ return -1;
+ } else if (rightFarDelimiter < 0 && leftFarDelimiter >= 0) {
+ return 1;
+ } else if (leftFarDelimiter < 0 && rightFarDelimiter < 0) {
+ return 0;
+ }
+ //Use superclass comparator
+ int result = MetaCellComparator.compareRows(left, leftDelimiter,
+ leftFarDelimiter - leftDelimiter,
+ right, rightDelimiter,
+ rightFarDelimiter - rightDelimiter);
+ if (result != 0) {
+ return result;
+ }
+
+ // Compare last part of row, the rowid.
+ leftFarDelimiter++;
+ rightFarDelimiter++;
+ result = Bytes.compareTo(left, leftFarDelimiter, llength - (leftFarDelimiter - loffset),
+ right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+ return result;
+ }
+
+ @Override
+ public int compareRows(ByteBuffer row, Cell cell) {
+ byte [] array;
+ int offset;
+ int len = row.remaining();
+ if (row.hasArray()) {
+ array = row.array();
+ offset = row.position() + row.arrayOffset();
+ } else {
+ // We copy the row array if offheap just so we can do a compare. We do this elsewhere too
+ // in BBUtils when Cell is backed by an offheap ByteBuffer. Needs fixing so no copy. TODO.
+ array = new byte[len];
+ offset = 0;
+ ByteBufferUtils.copyFromBufferToArray(array, row, row.position(),
+ 0, len);
+ }
+ // Reverse result since we swap the order of the params we pass below.
+ return -compareRows(cell, array, offset, len);
+ }
+ }
+
+ /**
* A {@link CellComparatorImpl} for <code>hbase:meta</code> catalog table
* {@link KeyValue}s.
*/
@@ -423,7 +495,12 @@ public class CellComparatorImpl implements CellComparator {
*/
public static CellComparator getCellComparator(byte [] tableName) {
// FYI, TableName.toBytes does not create an array; just returns existing array pointer.
- return Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())?
- CellComparatorImpl.META_COMPARATOR: CellComparatorImpl.COMPARATOR;
+ if(Bytes.equals(tableName, TableName.ROOT_TABLE_NAME.toBytes())) {
+ return CellComparatorImpl.ROOT_COMPARATOR;
+ }
+ if(Bytes.equals(tableName, TableName.META_TABLE_NAME.toBytes())) {
+ return CellComparatorImpl.META_COMPARATOR;
+ }
+ return CellComparatorImpl.COMPARATOR;
}
}
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
index ff09ea6..1304e43 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/KeyValue.java
@@ -109,6 +109,14 @@ public class KeyValue implements ExtendedCell, Cloneable {
@Deprecated
public static final KVComparator META_COMPARATOR = new MetaComparator();
+ /**
+ * A {@link KVComparator} for <code>hbase:root</code> catalog table
+ * {@link KeyValue}s.
+ * @deprecated Use {@link CellComparatorImpl#ROOT_COMPARATOR} instead. Deprecated for hbase 2.0, remove for hbase 3.0.
+ */
+ @Deprecated
+ public static final KVComparator ROOT_COMPARATOR = new RootComparator();
+
/** Size of the key length field in bytes*/
public static final int KEY_LENGTH_SIZE = Bytes.SIZEOF_INT;
@@ -1604,6 +1612,78 @@ public class KeyValue implements ExtendedCell, Cloneable {
}
/**
+ * A {@link KVComparator} for <code>-ROOT-</code> catalog table
+ * {@link KeyValue}s.
+ * @deprecated : {@link CellComparatorImpl#ROOT_COMPARATOR} to be used. Deprecated for hbase 2.0, remove for hbase 3.0.
+ */
+ @Deprecated
+ public static class RootComparator extends MetaComparator {
+ public int compareRows(byte [] left, int loffset, int llength,
+ byte [] right, int roffset, int rlength) {
+ // Rows look like this: .META.,ROW_FROM_META,RID
+ // LOG.info("ROOT " + Bytes.toString(left, loffset, llength) +
+ // "---" + Bytes.toString(right, roffset, rlength));
+ final int metalength = TableName.META_TABLE_NAME.getName().length+1; // '.META.' length
+ int lmetaOffsetPlusDelimiter = loffset + metalength;
+ int leftFarDelimiter = getDelimiterInReverse(left,
+ lmetaOffsetPlusDelimiter,
+ llength - metalength, HConstants.DELIMITER);
+ int rmetaOffsetPlusDelimiter = roffset + metalength;
+ int rightFarDelimiter = getDelimiterInReverse(right,
+ rmetaOffsetPlusDelimiter, rlength - metalength,
+ HConstants.DELIMITER);
+ if (leftFarDelimiter < 0 && rightFarDelimiter >= 0) {
+ // Nothing between .META. and regionid. Its first key.
+ return -1;
+ } else if (rightFarDelimiter < 0 && leftFarDelimiter >= 0) {
+ return 1;
+ } else if (leftFarDelimiter < 0 && rightFarDelimiter < 0) {
+ return 0;
+ }
+ int result = super.compareRows(left, lmetaOffsetPlusDelimiter,
+ leftFarDelimiter - lmetaOffsetPlusDelimiter,
+ right, rmetaOffsetPlusDelimiter,
+ rightFarDelimiter - rmetaOffsetPlusDelimiter);
+ if (result != 0) {
+ return result;
+ }
+ // Compare last part of row, the rowid.
+ leftFarDelimiter++;
+ rightFarDelimiter++;
+ result = Bytes.compareTo(left, leftFarDelimiter,
+ llength - (leftFarDelimiter - loffset),
+ right, rightFarDelimiter, rlength - (rightFarDelimiter - roffset));
+ return result;
+ }
+
+ /**
+ * The HFileV2 file format's trailer contains this class name. We reinterpret this and
+ * instantiate the appropriate comparator.
+ * TODO: With V3 consider removing this.
+ * @return legacy class name for FileFileTrailer#comparatorClassName
+ */
+ @Override
+ public String getLegacyKeyComparatorName() {
+ return "org.apache.hadoop.hbase.KeyValue$RootKeyComparator";
+ }
+
+ /**
+ * Compare key portion of a {@link KeyValue} for keys in <code>hbase:meta</code>
+ * table.
+ */
+ @Override
+ public int compare(final Cell left, final Cell right) {
+ return PrivateCellUtil.compareKeyIgnoresMvcc(CellComparatorImpl.ROOT_COMPARATOR, left, right);
+ }
+
+ @Override
+ protected Object clone() throws CloneNotSupportedException {
+ return new RootComparator();
+ }
+ }
+
+
+ /**
* A {@link KVComparator} for <code>hbase:meta</code> catalog table
* {@link KeyValue}s.
* @deprecated : {@link CellComparatorImpl#META_COMPARATOR} to be used. Deprecated for hbase 2.0, remove for hbase 3.0.
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
index b1205e0..68becf2 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/TableName.java
@@ -77,6 +77,10 @@ public final class TableName implements Comparable<TableName> {
"(?:"+VALID_TABLE_QUALIFIER_REGEX+"))";
/** The hbase:meta table's name. */
+ public static final TableName ROOT_TABLE_NAME =
+ valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "root");
+
+ /** The hbase:meta table's name. */
public static final TableName META_TABLE_NAME =
valueOf(NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR, "meta");
@@ -93,6 +97,13 @@ public final class TableName implements Comparable<TableName> {
/**
* @return True if <code>tn</code> is the hbase:meta table name.
*/
+ public static boolean isRootTableName(final TableName tn) {
+ return tn.equals(TableName.ROOT_TABLE_NAME);
+ }
+
+ /**
+ * @return True if <code>tn</code> is the hbase:meta table name.
+ */
public static boolean isMetaTableName(final TableName tn) {
return tn.equals(TableName.META_TABLE_NAME);
}
diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
index a318515..20788cf 100644
--- a/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
+++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/TestCellComparator.java
@@ -166,6 +166,44 @@ public class TestCellComparator {
assertTrue(c.compare(x, y) < 0);
}
+ /**
+ * Test meta comparisons using our new ByteBufferKeyValue Cell type, the type we use everywhere
+ * in 2.0.
+ */
+ @Test
+ public void testRootComparisons() throws Exception {
+ long now = System.currentTimeMillis();
+
+ // Meta compares
+ Cell aaa = createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+ ",TestScanMultipleVersions,row_0500,1236020145502,1236020145502"), now));
+ Cell bbb = createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+ ",TestScanMultipleVersions,,99999999999999,99999999999999"), now));
+ CellComparator c = CellComparatorImpl.ROOT_COMPARATOR;
+ assertTrue(c.compare(bbb, aaa) < 0);
+
+ Cell ccc = createByteBufferKeyValueFromKeyValue(
+ new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+ ",TestScanMultipleVersions,,99999999999999,1236023996656"),
+ Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236024396271L,
+ (byte[])null));
+ assertTrue(c.compare(ccc, bbb) < 0);
+
+ Cell x = createByteBufferKeyValueFromKeyValue(
+ new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+ ",TestScanMultipleVersions,row_0500,1236034574162,1236034574162"),
+ Bytes.toBytes("info"), Bytes.toBytes(""), 9223372036854775807L,
+ (byte[])null));
+ Cell y = createByteBufferKeyValueFromKeyValue(
+ new KeyValue(Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+
+ ",TestScanMultipleVersions,row_0500,1236034574162,1236034574162"),
+ Bytes.toBytes("info"), Bytes.toBytes("regioninfo"), 1236034574912L,
+ (byte[])null));
+ assertTrue(c.compare(x, y) < 0);
+ }
+
private static Cell createByteBufferKeyValueFromKeyValue(KeyValue kv) {
ByteBuffer bb = ByteBuffer.wrap(kv.getBuffer());
return new ByteBufferKeyValue(bb, 0, bb.remaining());
@@ -206,6 +244,37 @@ public class TestCellComparator {
}
@Test
+ public void testRootComparisons2() {
+ long now = System.currentTimeMillis();
+ CellComparator c = CellComparatorImpl.ROOT_COMPARATOR;
+ assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now)),
+ createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now))) == 0);
+ Cell a = createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now));
+ Cell b = createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,2"), now));
+ assertTrue(c.compare(a, b) < 0);
+ assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,2"), now)),
+ createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",a,,0,1"), now))) > 0);
+ assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now)),
+ createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) == 0);
+ assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now)),
+ createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,2"), now))) < 0);
+ assertTrue(c.compare(createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,2"), now)),
+ createByteBufferKeyValueFromKeyValue(new KeyValue(
+ Bytes.toBytes(TableName.META_TABLE_NAME.getNameAsString()+",,1"), now))) > 0);
+ }
+
+ @Test
public void testBinaryKeys() throws Exception {
Set<Cell> set = new TreeSet<>(CellComparatorImpl.COMPARATOR);
final byte [] fam = Bytes.toBytes("col");
diff --git a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
index 7c59ccf..e27c5d4 100644
--- a/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
+++ b/hbase-protocol-shaded/src/main/protobuf/MasterProcedure.proto
@@ -290,6 +290,7 @@ message ServerCrashStateData {
repeated RegionInfo regions_assigned = 4;
optional bool carrying_meta = 5;
optional bool should_split_wal = 6 [default = true];
+ optional bool carrying_root = 7;
}
message RecoverMetaStateData {
@@ -314,6 +315,10 @@ enum ServerCrashState {
SERVER_CRASH_DELETE_SPLIT_WALS_DIR=13;
SERVER_CRASH_HANDLE_RIT2 = 20[deprecated=true];
SERVER_CRASH_FINISH = 100;
+ SERVER_CRASH_SPLIT_ROOT_LOGS = 21;
+ SERVER_CRASH_ASSIGN_ROOT = 22;
+ SERVER_CRASH_CHECK_CARRYING_META = 23;
+ SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR=24;
}
enum RecoverMetaState {
@@ -467,6 +472,15 @@ message ReopenTableRegionsStateData {
repeated RegionLocation region = 2;
}
+enum InitRootState{
+ INIT_ROOT_ASSIGN_ROOT = 1;
+ INIT_ROOT_LOAD_ROOT = 2;
+ INIT_ROOT_INIT_META = 3;
+}
+
+message InitRootStateData {
+}
+
enum InitMetaState {
INIT_META_WRITE_FS_LAYOUT = 1;
INIT_META_ASSIGN_META = 2;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
index e1a4949..a8eabbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/coordination/ZkSplitLogWorkerCoordination.java
@@ -401,12 +401,22 @@ public class ZkSplitLogWorkerCoordination extends ZKListener implements
Collections.shuffle(paths);
// pick meta wal firstly
int offset = 0;
+ int metaOffset = -1;
+ int rootOffset = -1;
for (int i = 0; i < paths.size(); i++) {
+ if (AbstractFSWALProvider.isRootFile(paths.get(i))) {
+ offset = rootOffset;
+ }
if (AbstractFSWALProvider.isMetaFile(paths.get(i))) {
- offset = i;
- break;
+ offset = metaOffset;
}
}
+ if (rootOffset != -1) {
+ offset = rootOffset;
+ }
+ if (metaOffset != -1) {
+ offset = metaOffset;
+ }
int numTasks = paths.size();
boolean taskGrabbed = false;
for (int i = 0; i < numTasks; i++) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
index 35fb281..fe93cd1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/FixedFileTrailer.java
@@ -596,6 +596,9 @@ public class FixedFileTrailer {
if (comparator.equals(CellComparatorImpl.class.getName())) {
return KeyValue.COMPARATOR.getClass().getName();
}
+ if (comparator.equals(CellComparatorImpl.RootCellComparator.class.getName())) {
+ return KeyValue.ROOT_COMPARATOR.getClass().getName();
+ }
if (comparator.equals(MetaCellComparator.class.getName())) {
return KeyValue.META_COMPARATOR.getClass().getName();
}
@@ -611,6 +614,11 @@ public class FixedFileTrailer {
|| comparatorClassName.equals(KeyValue.COMPARATOR.getClass().getName())
|| (comparatorClassName.equals("org.apache.hadoop.hbase.CellComparator"))) {
comparatorKlass = CellComparatorImpl.class;
+ } else if (comparatorClassName.equals(KeyValue.ROOT_COMPARATOR.getLegacyKeyComparatorName())
+ || comparatorClassName.equals(KeyValue.ROOT_COMPARATOR.getClass().getName())
+ || (comparatorClassName
+ .equals("org.apache.hadoop.hbase.CellComparator$RootCellComparator"))) {
+ comparatorKlass = CellComparatorImpl.RootCellComparator.class;
} else if (comparatorClassName.equals(KeyValue.META_COMPARATOR.getLegacyKeyComparatorName())
|| comparatorClassName.equals(KeyValue.META_COMPARATOR.getClass().getName())
|| (comparatorClassName
@@ -638,7 +646,11 @@ public class FixedFileTrailer {
} else if (comparatorClassName.equals(
CellComparatorImpl.META_COMPARATOR.getClass().getName())) {
return CellComparatorImpl.META_COMPARATOR;
+ } else if (comparatorClassName.equals(
+ CellComparatorImpl.ROOT_COMPARATOR.getClass().getName())) {
+ return CellComparatorImpl.ROOT_COMPARATOR;
}
+
try {
Class<? extends CellComparator> comparatorClass = getComparatorClass(comparatorClassName);
if (comparatorClass != null) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
index dd05963..0d099c8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileWriterImpl.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hbase.ByteBufferExtendedCell;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
+import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.CellComparatorImpl.MetaCellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
@@ -375,7 +376,8 @@ public class HFileWriterImpl implements HFile.Writer {
// If Cells from meta table, don't mess around. meta table Cells have schema
// (table,startrow,hash) so can't be treated as plain byte arrays. Just skip
// out without trying to do this optimization.
- if (comparator instanceof MetaCellComparator) {
+ if (comparator instanceof MetaCellComparator ||
+ comparator instanceof CellComparatorImpl.RootCellComparator) {
return right;
}
int diff = comparator.compareRows(left, right);
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
index d01f357..8e7aac2 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
@@ -126,7 +126,7 @@ import org.apache.hadoop.hbase.master.procedure.DeleteNamespaceProcedure;
import org.apache.hadoop.hbase.master.procedure.DeleteTableProcedure;
import org.apache.hadoop.hbase.master.procedure.DisableTableProcedure;
import org.apache.hadoop.hbase.master.procedure.EnableTableProcedure;
-import org.apache.hadoop.hbase.master.procedure.InitMetaProcedure;
+import org.apache.hadoop.hbase.master.procedure.InitRootProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureConstants;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
@@ -354,7 +354,7 @@ public class HMaster extends HRegionServer implements MasterServices {
private volatile ServerManager serverManager;
// manager of assignment nodes in zookeeper
- private AssignmentManager assignmentManager;
+ public AssignmentManager assignmentManager;
/**
* Cache for the meta region replica's locations. Also tracks their changes to avoid stale
@@ -1011,22 +1011,23 @@ public class HMaster extends HRegionServer implements MasterServices {
}
// Checking if meta needs initializing.
- status.setStatus("Initializing meta table if this is a new deploy");
- InitMetaProcedure initMetaProc = null;
+ status.setStatus("Initializing catalog tables if this is a new deploy");
// Print out state of hbase:meta on startup; helps debugging.
RegionState rs = this.assignmentManager.getRegionStates().
- getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO);
- LOG.info("hbase:meta {}", rs);
- if (rs != null && rs.isOffline()) {
- Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
- .filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
- initMetaProc = optProc.orElseGet(() -> {
- // schedule an init meta procedure if meta has not been deployed yet
- InitMetaProcedure temp = new InitMetaProcedure();
- procedureExecutor.submitProcedure(temp);
- return temp;
- });
- }
+ getRegionState(RegionInfoBuilder.ROOT_REGIONINFO);
+ LOG.info("hbase:root {}", rs);
+ InitRootProcedure initRootProc = procedureExecutor.getProcedures().stream()
+ .filter(p -> p instanceof InitRootProcedure && !p.isFinished())
+ .map(o -> (InitRootProcedure) o).findAny()
+ .orElseGet(() -> {
+ if (rs != null && rs.isOffline()) {
+ // schedule an init meta procedure if meta has not been deployed yet
+ InitRootProcedure temp = new InitRootProcedure();
+ procedureExecutor.submitProcedure(temp);
+ return temp;
+ }
+ return null;
+ });
if (this.balancer instanceof FavoredNodesPromoter) {
favoredNodesManager = new FavoredNodesManager(this);
}
@@ -1039,9 +1040,9 @@ public class HMaster extends HRegionServer implements MasterServices {
// start up all service threads.
status.setStatus("Initializing master service threads");
startServiceThreads();
- // wait meta to be initialized after we start procedure executor
- if (initMetaProc != null) {
- initMetaProc.await();
+ // wait catalog tables to be initialized after we start procedure executor
+ if (initRootProc != null) {
+ initRootProc.await();
}
// Wake up this server to check in
sleeper.skipSleepCycle();
@@ -1060,15 +1061,6 @@ public class HMaster extends HRegionServer implements MasterServices {
}
status.setStatus("Starting assignment manager");
- // FIRST HBASE:META READ!!!!
- // The below cannot make progress w/o hbase:meta being online.
- // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
- // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
- // if it is down. It may take a while to come online. So, wait here until meta if for sure
- // available. That's what waitForMetaOnline does.
- if (!waitForMetaOnline()) {
- return;
- }
this.assignmentManager.joinCluster();
// The below depends on hbase:meta being online.
this.tableStateManager.start();
@@ -1139,9 +1131,9 @@ public class HMaster extends HRegionServer implements MasterServices {
assignmentManager.checkIfShouldMoveSystemRegionAsync();
status.setStatus("Assign meta replicas");
- MasterMetaBootstrap metaBootstrap = createMetaBootstrap();
+ MasterCatalogBootstrap metaBootstrap = createCatalogBootstrap();
try {
- metaBootstrap.assignMetaReplicas();
+ metaBootstrap.assignCatalogReplicas();
} catch (IOException | KeeperException e){
LOG.error("Assigning meta replica failed: ", e);
}
@@ -1203,6 +1195,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
* and we will hold here until operator intervention.
*/
+ //TODO francis move this to AM
@VisibleForTesting
public boolean waitForMetaOnline() {
return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
@@ -1212,6 +1205,7 @@ public class HMaster extends HRegionServer implements MasterServices {
* @return True if region is online and scannable else false if an error or shutdown (Otherwise
* we just block in here holding up all forward-progess).
*/
+ //TODO francis move this to AM
private boolean isRegionOnline(RegionInfo ri) {
RetryCounter rc = null;
while (!isStopped()) {
@@ -1255,7 +1249,7 @@ public class HMaster extends HRegionServer implements MasterServices {
}
// Else there are namespace regions up in meta. Ensure they are assigned before we go on.
for (RegionInfo ri: ris) {
- if (!isRegionOnline(ri)) {
+ if (!assignmentManager.isRegionOnline(ri)) {
return false;
}
}
@@ -1304,10 +1298,10 @@ public class HMaster extends HRegionServer implements MasterServices {
* </p>
*/
@VisibleForTesting
- protected MasterMetaBootstrap createMetaBootstrap() {
+ protected MasterCatalogBootstrap createCatalogBootstrap() {
// We put this out here in a method so can do a Mockito.spy and stub it out
// w/ a mocked up MasterMetaBootstrap.
- return new MasterMetaBootstrap(this);
+ return new MasterCatalogBootstrap(this);
}
/**
@@ -2244,7 +2238,7 @@ public class HMaster extends HRegionServer implements MasterServices {
if (t instanceof NoClassDefFoundError && t.getMessage().
contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) {
// improved error message for this special case
- abort("HBase is having a problem with its Hadoop jars. You may need to recompile " +
+ abort("HBase is having a problem with its HadoopActive jars. You may need to recompile " +
"HBase against Hadoop version " + org.apache.hadoop.util.VersionInfo.getVersion() +
" or change your hadoop jars to start properly", t);
} else {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
index da8d228..d5a0f4f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterMetaBootstrap.java
@@ -21,10 +21,13 @@ package org.apache.hadoop.hbase.master;
import java.io.IOException;
import java.util.List;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.MetaTableAccessor;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
+import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
@@ -38,12 +41,12 @@ import org.slf4j.LoggerFactory;
* Used by the HMaster on startup to split meta logs and assign the meta table.
*/
@InterfaceAudience.Private
-class MasterMetaBootstrap {
- private static final Logger LOG = LoggerFactory.getLogger(MasterMetaBootstrap.class);
+class MasterCatalogBootstrap {
+ private static final Logger LOG = LoggerFactory.getLogger(MasterCatalogBootstrap.class);
private final HMaster master;
- public MasterMetaBootstrap(HMaster master) {
+ public MasterCatalogBootstrap(HMaster master) {
this.master = master;
}
@@ -52,7 +55,7 @@ class MasterMetaBootstrap {
* TODO: The way this assign runs, nothing but chance to stop all replicas showing up on same
* server as the hbase:meta region.
*/
- void assignMetaReplicas()
+ void assignCatalogReplicas()
throws IOException, InterruptedException, KeeperException {
int numReplicas = master.getConfiguration().getInt(HConstants.META_REPLICAS_NUM,
HConstants.DEFAULT_META_REPLICA_NUM);
@@ -61,23 +64,56 @@ class MasterMetaBootstrap {
throw new IllegalStateException("hbase:meta must be initialized first before we can " +
"assign out its replicas");
}
- ServerName metaServername = MetaTableLocator.getMetaRegionLocation(this.master.getZooKeeper());
+ ServerName rootServername = MetaTableLocator.getRootRegionLocation(this.master.getZooKeeper());
for (int i = 1; i < numReplicas; i++) {
- // Get current meta state for replica from zk.
- RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper(), i);
+ // Get current hbase:root state for replica from zk.
+ RegionState rootState = MetaTableLocator.getRootRegionState(master.getZooKeeper(), i);
RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, i);
- LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + metaState);
- if (metaServername.equals(metaState.getServerName())) {
- metaState = null;
+ RegionInfoBuilder.ROOT_REGIONINFO, i);
+ LOG.debug(hri.getRegionNameAsString() + " replica region state from zookeeper=" + rootState);
+ if (rootServername.equals(rootState.getServerName())) {
+ rootState = null;
+ LOG.info(hri.getRegionNameAsString() +
+ " old location is same as current hbase:root location; setting location as null...");
+ }
+ // These assigns run inline. All is blocked till they complete. Only interrupt is shutting
+ // down hosting server which calls AM#stop.
+ if (rootState != null && rootState.getServerName() != null) {
+ // Try to retain old assignment.
+ assignmentManager.assign(hri, rootState.getServerName());
+ } else {
+ assignmentManager.assign(hri);
+ }
+ }
+
+ Result metaRegionResult = MetaTableAccessor.getRegionResult(
+ master.getConnection(), RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
+ RegionLocations regionLocations = MetaTableAccessor.getRegionLocations(metaRegionResult);
+ ServerName metaRegionServerName =
+ regionLocations.getRegionLocation(0) == null ?
+ null :
+ regionLocations.getRegionLocation(0).getServerName();
+ for (int i = 1; i < numReplicas; i++) {
+ // Get current hbase:meta state for replica from zk.
+ RegionInfo hri = RegionReplicaUtil.getRegionInfoForReplica(
+ RegionInfoBuilder.FIRST_META_REGIONINFO, i);
+ LOG.debug(hri.getRegionNameAsString() + " replica region state from meta=" +
+ regionLocations.getRegionLocation(i));
+ boolean retain = true;
+ ServerName replicaServerName =
+ regionLocations.getRegionLocation(i) == null ?
+ null :
+ regionLocations.getRegionLocation(i).getServerName();
+ if (metaRegionServerName.equals(replicaServerName)) {
+ retain = false;
LOG.info(hri.getRegionNameAsString() +
" old location is same as current hbase:meta location; setting location as null...");
}
// These assigns run inline. All is blocked till they complete. Only interrupt is shutting
// down hosting server which calls AM#stop.
- if (metaState != null && metaState.getServerName() != null) {
+ if (replicaServerName != null && retain) {
// Try to retain old assignment.
- assignmentManager.assignAsync(hri, metaState.getServerName());
+ assignmentManager.assignAsync(hri, replicaServerName);
} else {
assignmentManager.assignAsync(hri);
}
@@ -94,7 +130,7 @@ class MasterMetaBootstrap {
for (String metaReplicaZnode : metaReplicaZnodes) {
int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZnode(metaReplicaZnode);
if (replicaId >= numMetaReplicasConfigured) {
- RegionState r = MetaTableLocator.getMetaRegionState(zooKeeper, replicaId);
+ RegionState r = MetaTableLocator.getRootRegionState(zooKeeper, replicaId);
LOG.info("Closing excess replica of meta region " + r.getRegion());
// send a close and wait for a max of 30 seconds
ServerManager.closeRegionSilentlyAndWait(master.getClusterConnection(),
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
index adf8062..593b0e1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterRpcServices.java
@@ -1674,7 +1674,7 @@ public class MasterRpcServices extends RSRpcServices implements
MetaTableAccessor.getRegion(master.getConnection(), regionName);
if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), regionName)) {
pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
- MetaTableLocator.getMetaRegionLocation(master.getZooKeeper()));
+ MetaTableLocator.getRootRegionLocation(master.getZooKeeper()));
}
if (pair == null) {
throw new UnknownRegionException(Bytes.toString(regionName));
@@ -2494,7 +2494,7 @@ public class MasterRpcServices extends RSRpcServices implements
// TODO: actually, a full region name can save a lot on meta scan, improve later.
encodedName = RegionInfo.encodeRegionName(spec.getValue().toByteArray());
}
- RegionInfo info = this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
+ RegionInfo info = this.master.getAssignmentManager().loadRegionFromCatalog(encodedName);
LOG.trace("region info loaded from meta table: {}", info);
RegionState prevState =
this.master.getAssignmentManager().getRegionStates().getRegionState(info);
@@ -2508,7 +2508,7 @@ public class MasterRpcServices extends RSRpcServices implements
putList.add(metaPut);
MetaTableAccessor.putsToMetaTable(this.master.getConnection(), putList);
// Loads from meta again to refresh AM cache with the new region state
- this.master.getAssignmentManager().loadRegionFromMeta(encodedName);
+ this.master.getAssignmentManager().loadRegionFromCatalog(encodedName);
builder.addStates(RegionSpecifierAndState.newBuilder().setRegionSpecifier(spec)
.setState(prevState.getState().convert()));
}
@@ -2534,7 +2534,7 @@ public class MasterRpcServices extends RSRpcServices implements
RegionState regionState = this.master.getAssignmentManager().getRegionStates().
getRegionState(encodedRegionName);
ri = regionState == null ?
- this.master.getAssignmentManager().loadRegionFromMeta(encodedRegionName) :
+ this.master.getAssignmentManager().loadRegionFromCatalog(encodedRegionName) :
regionState.getRegion();
break;
default:
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
index 58e57c4..5880b6b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterStatusServlet.java
@@ -80,7 +80,7 @@ public class MasterStatusServlet extends HttpServlet {
}
private ServerName getMetaLocationOrNull(HMaster master) {
- return MetaTableLocator.getMetaRegionLocation(master.getZooKeeper());
+ return MetaTableLocator.getRootRegionLocation(master.getZooKeeper());
}
private Map<String, Integer> getFragmentationInfo(
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
index 3f6bd8a..3930abe 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterWalManager.java
@@ -55,6 +55,16 @@ import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesti
public class MasterWalManager {
private static final Logger LOG = LoggerFactory.getLogger(MasterWalManager.class);
+ final static PathFilter ROOT_FILTER = new PathFilter() {
+ @Override
+ public boolean accept(Path p) {
+ return AbstractFSWALProvider.isRootFile(p);
+ }
+ };
+
+ /**
+ * Filter *in* WAL files that are for the hbase:meta Region.
+ */
final static PathFilter META_FILTER = new PathFilter() {
@Override
public boolean accept(Path p) {
@@ -63,10 +73,10 @@ public class MasterWalManager {
};
@VisibleForTesting
- public final static PathFilter NON_META_FILTER = new PathFilter() {
+ public final static PathFilter NON_CATALOG_FILTER = new PathFilter() {
@Override
public boolean accept(Path p) {
- return !AbstractFSWALProvider.isMetaFile(p);
+ return !AbstractFSWALProvider.isCatalogFile(p);
}
};
@@ -273,6 +283,22 @@ public class MasterWalManager {
}
/**
+ * Specialized method to handle the splitting for root WAL
+ * @param serverName logs belonging to this server will be split
+ */
+ public void splitRootLog(final ServerName serverName) throws IOException {
+ splitRootLog(Collections.<ServerName>singleton(serverName));
+ }
+
+ /**
+ * Specialized method to handle the splitting for root WAL
+ * @param serverNames logs belonging to these servers will be split
+ */
+ public void splitRootLog(final Set<ServerName> serverNames) throws IOException {
+ splitLog(serverNames, ROOT_FILTER);
+ }
+
+ /**
* Specialized method to handle the splitting for meta WAL
* @param serverName logs belonging to this server will be split
*/
@@ -332,7 +358,7 @@ public class MasterWalManager {
}
public void splitLog(final Set<ServerName> serverNames) throws IOException {
- splitLog(serverNames, NON_META_FILTER);
+ splitLog(serverNames, NON_CATALOG_FILTER);
}
/**
@@ -351,7 +377,9 @@ public class MasterWalManager {
splitTime = EnvironmentEdgeManager.currentTime() - splitTime;
if (this.metricsMasterFilesystem != null) {
- if (filter == META_FILTER) {
+ if (filter == ROOT_FILTER) {
+ //TODO francis add metrics
+ } else if (filter == META_FILTER) {
this.metricsMasterFilesystem.addMetaWALSplit(splitTime, splitLogSize);
} else {
this.metricsMasterFilesystem.addSplit(splitTime, splitLogSize);
@@ -367,13 +395,19 @@ public class MasterWalManager {
* meta log and Archiving the meta log and delete the dir.
* @param serverName the server to archive meta log
*/
- public void archiveMetaLog(final ServerName serverName) {
+ public void archiveCatalogLog(final ServerName serverName, boolean isRoot) {
+ String regionType = "meta";
+ PathFilter filterType = META_FILTER;
+ if (isRoot) {
+ regionType = "root";
+ filterType = ROOT_FILTER;
+ }
try {
Path logDir = new Path(this.rootDir,
AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
if (fs.exists(splitDir)) {
- FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, META_FILTER);
+ FileStatus[] logfiles = CommonFSUtils.listStatus(fs, splitDir, filterType);
if (logfiles != null) {
for (FileStatus status : logfiles) {
if (!status.isDir()) {
@@ -382,7 +416,7 @@ public class MasterWalManager {
if (!CommonFSUtils.renameAndSetModifyTime(fs, status.getPath(), newPath)) {
LOG.warn("Unable to move " + status.getPath() + " to " + newPath);
} else {
- LOG.debug("Archived meta log " + status.getPath() + " to " + newPath);
+ LOG.debug("Archived "+regionType+" log " + status.getPath() + " to " + newPath);
}
}
}
@@ -392,7 +426,7 @@ public class MasterWalManager {
}
}
} catch (IOException ie) {
- LOG.warn("Failed archiving meta log for server " + serverName, ie);
+ LOG.warn("Failed archiving "+regionType+" log for server " + serverName, ie);
}
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
index f4e91b5..60cdc49 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/MetaRegionLocationCache.java
@@ -149,7 +149,7 @@ public class MetaRegionLocationCache extends ZKListener {
try {
byte[] data = ZKUtil.getDataAndWatch(watcher,
watcher.getZNodePaths().getZNodeForReplica(replicaId));
- metaRegionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+ metaRegionState = ProtobufUtil.parseRootRegionStateFrom(data, replicaId);
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
index 76407e0..5ef25a1 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/SplitWALManager.java
@@ -20,7 +20,8 @@ package org.apache.hadoop.hbase.master;
import static org.apache.hadoop.hbase.HConstants.DEFAULT_HBASE_SPLIT_WAL_MAX_SPLITTER;
import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_MAX_SPLITTER;
import static org.apache.hadoop.hbase.master.MasterWalManager.META_FILTER;
-import static org.apache.hadoop.hbase.master.MasterWalManager.NON_META_FILTER;
+import static org.apache.hadoop.hbase.master.MasterWalManager.NON_CATALOG_FILTER;
+import static org.apache.hadoop.hbase.master.MasterWalManager.ROOT_FILTER;
import java.io.IOException;
import java.util.Collections;
@@ -34,6 +35,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureScheduler;
import org.apache.hadoop.hbase.master.procedure.SplitWALProcedure;
@@ -73,6 +75,22 @@ import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
public class SplitWALManager {
private static final Logger LOG = LoggerFactory.getLogger(SplitWALManager.class);
+ public enum SplitType {
+ ROOT(ROOT_FILTER),
+ META(META_FILTER),
+ USER(NON_CATALOG_FILTER);
+
+ private PathFilter filter;
+
+ SplitType(PathFilter filter) {
+ this.filter = filter;
+ }
+
+ public PathFilter getFilter() {
+ return filter;
+ }
+ }
+
private final MasterServices master;
private final SplitWorkerAssigner splitWorkerAssigner;
private final Path rootDir;
@@ -89,11 +107,19 @@ public class SplitWALManager {
}
- public List<Procedure> splitWALs(ServerName crashedServer, boolean splitMeta)
+ //TODO francis temp added this to avoid updating all tests
+ public List<Procedure> splitWALs(ServerName crashedServer, boolean splitMeta) throws IOException {
+ if (splitMeta) {
+ return splitWALs(crashedServer, SplitType.META);
+ }
+ return splitWALs(crashedServer, SplitType.USER);
+ }
+
+ public List<Procedure> splitWALs(ServerName crashedServer, SplitType splitType)
throws IOException {
try {
// 1. list all splitting files
- List<FileStatus> splittingFiles = getWALsToSplit(crashedServer, splitMeta);
+ List<FileStatus> splittingFiles = getWALsToSplit(crashedServer, splitType);
// 2. create corresponding procedures
return createSplitWALProcedures(splittingFiles, crashedServer);
} catch (IOException e) {
@@ -102,12 +128,22 @@ public class SplitWALManager {
}
}
+
+ //TODO francis temp added this to avoid updating tests
public List<FileStatus> getWALsToSplit(ServerName serverName, boolean splitMeta)
+ throws IOException {
+ if (splitMeta) {
+ return getWALsToSplit(serverName, SplitType.META);
+ }
+ return getWALsToSplit(serverName, SplitType.USER);
+ }
+
+ public List<FileStatus> getWALsToSplit(ServerName serverName, SplitType splitType)
throws IOException {
List<Path> logDirs = master.getMasterWalManager().getLogDirs(Collections.singleton(serverName));
FileStatus[] fileStatuses =
- SplitLogManager.getFileList(this.conf, logDirs, splitMeta ? META_FILTER : NON_META_FILTER);
- LOG.info("{} WAL count={}, meta={}", serverName, fileStatuses.length, splitMeta);
+ SplitLogManager.getFileList(this.conf, logDirs, splitType.getFilter());
+ LOG.info("{} WAL count={}, SplitType={}", serverName, fileStatuses.length, splitType);
return Lists.newArrayList(fileStatuses);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
index 04529f0..aa742eb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/AssignmentManager.java
@@ -25,6 +25,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
@@ -75,6 +76,8 @@ import org.apache.hadoop.hbase.regionserver.SequenceId;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.hbase.util.VersionInfo;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
@@ -150,6 +153,9 @@ public class AssignmentManager {
"hbase.metrics.rit.stuck.warning.threshold";
private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
+ private final ProcedureEvent<?> rootAssignEvent = new ProcedureEvent<>("root assign");
+ private final ProcedureEvent<?> rootLoadEvent = new ProcedureEvent<>("root load");
+
private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
@@ -230,9 +236,9 @@ public class AssignmentManager {
// here we are still in the early steps of active master startup. There is only one thread(us)
// can access AssignmentManager and create region node, so here we do not need to lock the
// region node.
- RegionState regionState = MetaTableLocator.getMetaRegionState(zkw);
+ RegionState regionState = MetaTableLocator.getRootRegionState(zkw);
RegionStateNode regionNode =
- regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ regionStates.getOrCreateRegionStateNode(RegionInfoBuilder.ROOT_REGIONINFO);
regionNode.setRegionLocation(regionState.getServerName());
regionNode.setState(regionState.getState());
if (regionNode.getProcedure() != null) {
@@ -310,6 +316,10 @@ public class AssignmentManager {
for (RegionInfo hri: getMetaRegionSet()) {
setMetaAssigned(hri, false);
}
+ rootLoadEvent.suspend();
+ for(RegionInfo hri: getRootRegionSet()) {
+ setRootAssigned(hri, false);
+ }
}
}
@@ -393,6 +403,130 @@ public class AssignmentManager {
}
// ============================================================================================
+ // ROOT Helpers
+ // ============================================================================================
+ private boolean isRootRegion(final RegionInfo regionInfo) {
+ return regionInfo.isRootRegion();
+ }
+
+ public boolean isRootRegion(final byte[] regionName) {
+ return getRootRegionFromName(regionName) != null;
+ }
+
+ public RegionInfo getRootRegionFromName(final byte[] regionName) {
+ for (RegionInfo hri: getRootRegionSet()) {
+ if (Bytes.equals(hri.getRegionName(), regionName)) {
+ return hri;
+ }
+ }
+ return null;
+ }
+
+ public boolean isCarryingRoot(final ServerName serverName) {
+ // TODO: handle multiple root
+ return isCarryingRegion(serverName, RegionInfoBuilder.ROOT_REGIONINFO);
+ }
+
+ private RegionInfo getRootForRegion(final RegionInfo regionInfo) {
+ //if (regionInfo.isRootRegion()) return regionInfo;
+ // TODO: handle multiple root. if the region provided is not root lookup
+ // which root the region belongs to.
+ return RegionInfoBuilder.ROOT_REGIONINFO;
+ }
+
+ /**
+ * Check hbase:root is up and ready for reading. For use during Master startup only.
+ * @return True if root is UP and online and startup can progress. Otherwise, root is not online
+ * and we will hold here until operator intervention.
+ */
+ public boolean waitForRootOnline() {
+ return isRegionOnline(RegionInfoBuilder.ROOT_REGIONINFO);
+ }
+
+ // TODO: handle multiple root.
+ private static final Set<RegionInfo> ROOT_REGION_SET =
+ Collections.singleton(RegionInfoBuilder.ROOT_REGIONINFO);
+ public Set<RegionInfo> getRootRegionSet() {
+ return ROOT_REGION_SET;
+ }
+
+ // ============================================================================================
+ // ROOT Event(s) helpers
+ // ============================================================================================
+ /**
+ * Notice that, this only means the root region is available on a RS, but the AM may still be
+ * loading the region states from root, so usually you need to check {@link #isRootLoaded()} first
+ * before checking this method, unless you can make sure that your piece of code can only be
+ * executed after AM builds the region states.
+ * @see #isRootLoaded()
+ */
+ public boolean isRootAssigned() {
+ return rootAssignEvent.isReady();
+ }
+
+ public boolean isRootRegionInTransition() {
+ return !isRootAssigned();
+ }
+
+ /**
+ * Notice that this event does not mean the AM has already finished region state rebuilding. See
+ * the comment of {@link #isRootAssigned()} for more details.
+ * @see #isRootAssigned()
+ */
+ public boolean waitRootAssigned(Procedure<?> proc, RegionInfo regionInfo) {
+ return getRootAssignEvent(getRootForRegion(regionInfo)).suspendIfNotReady(proc);
+ }
+
+ private void setRootAssigned(RegionInfo rootRegionInfo, boolean assigned) {
+ assert isRootRegion(rootRegionInfo) : "unexpected non-root region " + rootRegionInfo;
+ ProcedureEvent<?> rootAssignEvent = getRootAssignEvent(rootRegionInfo);
+ if (assigned) {
+ LOG.debug("Setting hbase:root region assigned: "+rootRegionInfo);
+ rootAssignEvent.wake(getProcedureScheduler());
+ } else {
+ LOG.debug("Setting hbase:root region unassigned: "+rootRegionInfo);
+ rootAssignEvent.suspend();
+ }
+ }
+
+ private ProcedureEvent<?> getRootAssignEvent(RegionInfo rootRegionInfo) {
+ assert isRootRegion(rootRegionInfo) : "unexpected non-catalog region " + rootRegionInfo;
+ // TODO: handle multiple root.
+ return rootAssignEvent;
+ }
+
+ /**
+ * Wait until AM finishes the root loading, i.e, the region states rebuilding.
+ * @see #isRootLoaded()
+ * @see #waitRootAssigned(Procedure, RegionInfo)
+ */
+ public boolean waitRootLoaded(Procedure<?> proc) {
+ if (rootLoadEvent.suspendIfNotReady(proc)) {
+ LOG.debug("Waiting for root to be loaded: "+proc);
+ return true;
+ }
+ return false;
+ }
+
+ @VisibleForTesting
+ void wakeRootLoadedEvent() {
+ synchronized (rootLoadEvent) {
+ rootLoadEvent.wake(getProcedureScheduler());
+ assert isRootLoaded() : "expected root to be loaded";
+ }
+ }
+
+ /**
+ * Return whether AM finishes the root loading, i.e, the region states rebuilding.
+ * @see #isRootAssigned()
+ * @see #waitRootLoaded(Procedure)
+ */
+ public boolean isRootLoaded() {
+ return rootLoadEvent.isReady();
+ }
+
+
+ // ============================================================================================
// META Helpers
// ============================================================================================
private boolean isMetaRegion(final RegionInfo regionInfo) {
@@ -430,6 +564,15 @@ public class AssignmentManager {
return RegionInfoBuilder.FIRST_META_REGIONINFO;
}
+ /**
+ * Check hbase:meta is up and ready for reading. For use during Master startup only.
+ * @return True if meta is UP and online and startup can progress. Otherwise, meta is not online
+ * and we will hold here until operator intervention.
+ */
+ public boolean waitForMetaOnline() {
+ return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ }
+
// TODO: handle multiple meta.
private static final Set<RegionInfo> META_REGION_SET =
Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
@@ -468,8 +611,10 @@ public class AssignmentManager {
assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
ProcedureEvent<?> metaAssignEvent = getMetaAssignEvent(metaRegionInfo);
if (assigned) {
+ LOG.debug("Setting hbase:meta region assigned: "+metaRegionInfo);
metaAssignEvent.wake(getProcedureScheduler());
} else {
+ LOG.debug("Setting hbase:meta region unassigned: "+metaRegionInfo);
metaAssignEvent.suspend();
}
}
@@ -486,7 +631,11 @@ public class AssignmentManager {
* @see #waitMetaAssigned(Procedure, RegionInfo)
*/
public boolean waitMetaLoaded(Procedure<?> proc) {
- return metaLoadEvent.suspendIfNotReady(proc);
+ if (metaLoadEvent.suspendIfNotReady(proc)) {
+ LOG.debug("Waiting for meta to be loaded: "+proc);
+ return true;
+ }
+ return false;
}
@VisibleForTesting
@@ -504,6 +653,39 @@ public class AssignmentManager {
return metaLoadEvent.isReady();
}
+
+ /**
+ * @return True if region is online and scannable else false if an error or shutdown (Otherwise
+ * we just block in here holding up all forward-progess).
+ */
+ public boolean isRegionOnline(RegionInfo ri) {
+ RetryCounter rc = null;
+ while (!master.isStopped()) {
+ RegionState rs = regionStates.getRegionState(ri);
+ if (rs.isOpened()) {
+ if (master.getServerManager().isServerOnline(rs.getServerName())) {
+ return true;
+ }
+ }
+ // Region is not OPEN.
+ Optional<Procedure<MasterProcedureEnv>> optProc =
+ master.getMasterProcedureExecutor().getProcedures().
+ stream().filter(p -> p instanceof ServerCrashProcedure).findAny();
+ // TODO: Add a page to refguide on how to do repair. Have this log message point to it.
+ // Page will talk about loss of edits, how to schedule at least the meta WAL recovery, and
+ // then how to assign including how to break region lock if one held.
+ LOG.warn("{} is NOT online; state={}; ServerCrashProcedures={}. Master startup cannot " +
+ "progress, in holding-pattern until region onlined.",
+ ri.getRegionNameAsString(), rs, optProc.isPresent());
+ // Check once-a-minute.
+ if (rc == null) {
+ rc = new RetryCounterFactory(1000).create();
+ }
+ Threads.sleep(rc.getBackoffTimeAndIncrementAttempts());
+ }
+ return false;
+ }
+
/**
* Start a new thread to check if there are region servers whose versions are higher than others.
* If so, move all system table regions to RS with the highest version to keep compatibility.
@@ -970,7 +1152,7 @@ public class AssignmentManager {
private void updateRegionTransition(ServerName serverName, TransitionCode state,
RegionInfo regionInfo, long seqId, long procId) throws IOException {
- checkMetaLoaded(regionInfo);
+ checkParentCatalogLoaded(regionInfo);
RegionStateNode regionNode = regionStates.getRegionStateNode(regionInfo);
if (regionNode == null) {
@@ -1022,7 +1204,7 @@ public class AssignmentManager {
private void updateRegionSplitTransition(final ServerName serverName, final TransitionCode state,
final RegionInfo parent, final RegionInfo hriA, final RegionInfo hriB)
throws IOException {
- checkMetaLoaded(parent);
+ checkParentCatalogLoaded(parent);
if (state != TransitionCode.READY_TO_SPLIT) {
throw new UnexpectedStateException("unsupported split regionState=" + state +
@@ -1060,7 +1242,7 @@ public class AssignmentManager {
private void updateRegionMergeTransition(final ServerName serverName, final TransitionCode state,
final RegionInfo merged, final RegionInfo hriA, final RegionInfo hriB) throws IOException {
- checkMetaLoaded(merged);
+ checkParentCatalogLoaded(merged);
if (state != TransitionCode.READY_TO_MERGE) {
throw new UnexpectedStateException("Unsupported merge regionState=" + state +
@@ -1129,7 +1311,7 @@ public class AssignmentManager {
LOG.trace("no online region found on {}", serverName);
return;
}
- if (!isMetaLoaded()) {
+ if (!isRootLoaded() || !isMetaLoaded()) {
// we are still on startup, skip checking
return;
}
@@ -1400,6 +1582,28 @@ public class AssignmentManager {
long startTime = System.nanoTime();
LOG.debug("Joining cluster...");
+ // FIRST Catalog tables READ!!!!
+ // The below cannot make progress w/o hbase:meta being online.
+ // This is the FIRST attempt at going to hbase:meta. Meta on-lining is going on in background
+ // as procedures run -- in particular SCPs for crashed servers... One should put up hbase:meta
+ // if it is down. It may take a while to come online. So, wait here until meta if for sure
+ // available. That's what waitForXXXXOnline does.
+
+
+
+ LOG.debug("Waiting for hbase:root to be online.");
+ if (!waitForRootOnline()) {
+ throw new IOException("Waited too long for hbase:root to be online");
+ }
+
+ //load hbase:root to build regionstate for hbase:meta regions
+ loadRoot();
+
+ LOG.debug("Waiting for hbase:meta to be online.");
+ if (!waitForMetaOnline()) {
+ throw new IOException("Waited too long for hbase:meta to be online");
+ }
+
// Scan hbase:meta to build list of existing regions, servers, and assignment.
// hbase:meta is online now or will be. Inside loadMeta, we keep trying. Can't make progress
// w/o meta.
@@ -1442,13 +1646,13 @@ public class AssignmentManager {
}
/* AM internal RegionStateStore.RegionStateVisitor implementation. To be used when
- * scanning META table for region rows, using RegionStateStore utility methods. RegionStateStore
+ * scanning Catalog table for region rows, using RegionStateStore utility methods. RegionStateStore
* methods will convert Result into proper RegionInfo instances, but those would still need to be
* added into AssignmentManager.regionStates in-memory cache.
* RegionMetaLoadingVisitor.visitRegionState method provides the logic for adding RegionInfo
* instances as loaded from latest META scan into AssignmentManager.regionStates.
*/
- private class RegionMetaLoadingVisitor implements RegionStateStore.RegionStateVisitor {
+ private class RegionCatalogLoadingVisitor implements RegionStateStore.RegionStateVisitor {
@Override
public void visitRegionState(Result result, final RegionInfo regionInfo, final State state,
@@ -1489,6 +1693,13 @@ public class AssignmentManager {
if (regionNode.getProcedure() != null) {
regionNode.getProcedure().stateLoaded(AssignmentManager.this, regionNode);
}
+ if (isMetaRegion(regionInfo)) {
+ if (localState.matches(State.OPEN)) {
+ setMetaAssigned(regionInfo, true);
+ } else if (localState.matches(State.CLOSING, State.CLOSED)){
+ setMetaAssigned(regionInfo, false);
+ }
+ }
}
};
@@ -1502,10 +1713,10 @@ public class AssignmentManager {
* cache, <b>null</b> otherwise.
* @throws UnknownRegionException if any errors occur while querying meta.
*/
- public RegionInfo loadRegionFromMeta(String regionEncodedName) throws UnknownRegionException {
+ public RegionInfo loadRegionFromCatalog(String regionEncodedName) throws UnknownRegionException {
try {
- RegionMetaLoadingVisitor visitor = new RegionMetaLoadingVisitor();
- regionStateStore.visitMetaForRegion(regionEncodedName, visitor);
+ RegionCatalogLoadingVisitor visitor = new RegionCatalogLoadingVisitor();
+ regionStateStore.visitCatalogForRegion(regionEncodedName, visitor);
return regionStates.getRegionState(regionEncodedName) == null ? null :
regionStates.getRegionState(regionEncodedName).getRegion();
} catch(IOException e) {
@@ -1514,9 +1725,24 @@ public class AssignmentManager {
}
}
+ public void loadRoot() throws IOException {
+ //TODO francis is the the right monitor lock to synchronize on?
+ synchronized (rootLoadEvent) {
+ if (!isRootLoaded()) {
+ // TODO: use a thread pool
+ LOG.debug("Loaded hbase:root");
+ regionStateStore.visitCatalogTable(true, new RegionCatalogLoadingVisitor());
+ wakeRootLoadedEvent();
+ } else {
+ LOG.debug("Not loading hbase:root, already loaded");
+ }
+ }
+ }
+
private void loadMeta() throws IOException {
// TODO: use a thread pool
- regionStateStore.visitMeta(new RegionMetaLoadingVisitor());
+ LOG.debug("Loaded hbase:meta");
+ regionStateStore.visitCatalogTable(false, new RegionCatalogLoadingVisitor());
// every assignment is blocked until meta is loaded.
wakeMetaLoadedEvent();
}
@@ -1528,6 +1754,25 @@ public class AssignmentManager {
* @param hri region to check if it is already rebuild
* @throws PleaseHoldException if meta has not been loaded yet
*/
+ private void checkRootLoaded(RegionInfo hri) throws PleaseHoldException {
+ if (!isRunning()) {
+ throw new PleaseHoldException("AssignmentManager not running");
+ }
+ boolean root = isRootRegion(hri);
+ boolean rootLoaded = isRootLoaded();
+ if (!root && !rootLoaded) {
+ throw new PleaseHoldException(
+ "Master not fully online; hbase:root=" + root + ", rootLoaded=" + rootLoaded);
+ }
+ }
+
+ /**
+ * Used to check if the meta loading is done.
+ * <p/>
+ * if not we throw PleaseHoldException since we are rebuilding the RegionStates
+ * @param hri region to check if it is already rebuild
+ * @throws PleaseHoldException if meta has not been loaded yet
+ */
private void checkMetaLoaded(RegionInfo hri) throws PleaseHoldException {
if (!isRunning()) {
throw new PleaseHoldException("AssignmentManager not running");
@@ -1540,6 +1785,15 @@ public class AssignmentManager {
}
}
+
+ private void checkParentCatalogLoaded(RegionInfo regionInfo) throws PleaseHoldException {
+ if (regionInfo.isMetaRegion()) {
+ checkRootLoaded(regionInfo);
+ } else if (!regionInfo.isRootRegion()) {
+ checkMetaLoaded(regionInfo);
+ }
+ }
+
// ============================================================================================
// TODO: Metrics
// ============================================================================================
@@ -1575,14 +1829,16 @@ public class AssignmentManager {
if (serverNode != null) {
serverNode.writeLock().lock();
}
+ boolean carryingRoot;
boolean carryingMeta;
long pid;
try {
ProcedureExecutor<MasterProcedureEnv> procExec = this.master.getMasterProcedureExecutor();
+ carryingRoot = isCarryingRoot(serverName);
carryingMeta = isCarryingMeta(serverName);
if (!force && serverNode != null && !serverNode.isInState(ServerState.ONLINE)) {
- LOG.info("Skip adding ServerCrashProcedure for {} (meta={}) -- running?",
- serverNode, carryingMeta);
+ LOG.info("Skip adding ServerCrashProcedure for {} (root={}, meta={}) -- running?",
+ serverNode, carryingRoot, carryingMeta);
return Procedure.NO_PROC_ID;
} else {
MasterProcedureEnv mpe = procExec.getEnvironment();
@@ -1598,13 +1854,15 @@ public class AssignmentManager {
if (force) {
pid = procExec.submitProcedure(
- new HBCKServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta));
+ new HBCKServerCrashProcedure(
+ mpe, serverName, shouldSplitWal, carryingRoot, carryingMeta));
} else {
pid = procExec.submitProcedure(
- new ServerCrashProcedure(mpe, serverName, shouldSplitWal, carryingMeta));
+ new ServerCrashProcedure(
+ mpe, serverName, shouldSplitWal, carryingRoot, carryingMeta));
}
- LOG.info("Scheduled ServerCrashProcedure pid={} for {} (carryingMeta={}){}.",
- pid, serverName, carryingMeta,
+ LOG.info("Scheduled ServerCrashProcedure pid={} for {} (carryingRoot={}, carryingMeta={}){}.",
+ pid, serverName, carryingRoot, carryingMeta,
serverNode == null? "": " " + serverNode.toString() + ", oldState=" + oldState);
}
} finally {
@@ -1767,8 +2025,11 @@ public class AssignmentManager {
transitStateAndUpdate(regionNode, State.CLOSING, STATES_EXPECTED_ON_CLOSING);
RegionInfo hri = regionNode.getRegionInfo();
- // Set meta has not initialized early. so people trying to create/edit tables will wait
- if (isMetaRegion(hri)) {
+ // Set root has not initialized early. so people trying to create/edit tables will wait
+ if (isRootRegion(hri)) {
+ setRootAssigned(hri, false);
+ } else if (isMetaRegion(hri)) {
+ // Set meta has not initialized early. so people trying to create/edit tables will wait
setMetaAssigned(hri, false);
}
regionStates.addRegionToServer(regionNode);
@@ -1834,6 +2095,13 @@ public class AssignmentManager {
// on table that contains state.
setMetaAssigned(regionInfo, true);
}
+ if (isRootRegion(regionInfo) && regionNode.getState() == State.OPEN) {
+ // Usually we'd set a table ENABLED at this stage but hbase:meta is ALWAYs enabled, it
+ // can't be disabled -- so skip the RPC (besides... enabled is managed by TableStateManager
+ // which is backed by hbase:meta... Avoid setting ENABLED to avoid having to update state
+ // on table that contains state.
+ setRootAssigned(regionInfo, true);
+ }
}
// ============================================================================================
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
index 1c90d81..caafb64 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionRemoteProcedureBase.java
@@ -148,13 +148,20 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
@Override
protected boolean waitInitialized(MasterProcedureEnv env) {
- if (TableName.isMetaTableName(getTableName())) {
+ if (TableName.isRootTableName(getTableName())) {
return false;
}
- // First we need meta to be loaded, and second, if meta is not online then we will likely to
- // fail when updating meta so we wait until it is assigned.
- AssignmentManager am = env.getAssignmentManager();
- return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, region);
+ if (TableName.isMetaTableName(getTableName())) {
+ // First we need rot to be loaded, and second, if root is not online then we will likely to
+ // fail when updating root so we wait until it is assigned.
+ AssignmentManager am = env.getAssignmentManager();
+ return am.waitRootLoaded(this) || am.waitRootAssigned(this, region);
+ } else {
+ // First we need meta to be loaded, and second, if meta is not online then we will likely to
+ // fail when updating meta so we wait until it is assigned.
+ AssignmentManager am = env.getAssignmentManager();
+ return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, region);
+ }
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
index c353161..6c88cbd 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStateStore.java
@@ -74,8 +74,10 @@ public class RegionStateStore {
ServerName regionLocation, ServerName lastHost, long openSeqNum);
}
- public void visitMeta(final RegionStateVisitor visitor) throws IOException {
- MetaTableAccessor.fullScanRegions(master.getConnection(), new MetaTableAccessor.Visitor() {
+ public void visitCatalogTable(boolean useRoot, final RegionStateVisitor visitor)
+ throws IOException {
+ MetaTableAccessor.fullScanRegions(master.getConnection(), useRoot,
+ new MetaTableAccessor.Visitor() {
final boolean isDebugEnabled = LOG.isDebugEnabled();
@Override
@@ -85,7 +87,7 @@ public class RegionStateStore {
if (LOG.isTraceEnabled()) {
st = System.currentTimeMillis();
}
- visitMetaEntry(visitor, r);
+ visitCatalogEntry(visitor, r);
if (LOG.isTraceEnabled()) {
long et = System.currentTimeMillis();
LOG.trace("[T] LOAD META PERF " + StringUtils.humanTimeDiff(et - st));
@@ -106,16 +108,16 @@ public class RegionStateStore {
* @param visitor The <code>RegionStateVisitor</code> instance to react over the query results.
* @throws IOException If some error occurs while querying META or parsing results.
*/
- public void visitMetaForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
+ public void visitCatalogForRegion(final String regionEncodedName, final RegionStateVisitor visitor)
throws IOException {
Result result = MetaTableAccessor.
scanByRegionEncodedName(master.getConnection(), regionEncodedName);
if (result != null) {
- visitMetaEntry(visitor, result);
+ visitCatalogEntry(visitor, result);
}
}
- private void visitMetaEntry(final RegionStateVisitor visitor, final Result result)
+ private void visitCatalogEntry(final RegionStateVisitor visitor, final Result result)
throws IOException {
final RegionLocations rl = MetaTableAccessor.getRegionLocations(result);
if (rl == null) return;
@@ -147,13 +149,13 @@ public class RegionStateStore {
}
void updateRegionLocation(RegionStateNode regionStateNode) throws IOException {
- if (regionStateNode.getRegionInfo().isMetaRegion()) {
- updateMetaLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
+ if (regionStateNode.getRegionInfo().isRootRegion()) {
+ updateRootLocation(regionStateNode.getRegionInfo(), regionStateNode.getRegionLocation(),
regionStateNode.getState());
} else {
long openSeqNum = regionStateNode.getState() == State.OPEN ? regionStateNode.getOpenSeqNum()
: HConstants.NO_SEQNUM;
- updateUserRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(),
+ updateRegionLocation(regionStateNode.getRegionInfo(), regionStateNode.getState(),
regionStateNode.getRegionLocation(), openSeqNum,
// The regionStateNode may have no procedure in a test scenario; allow for this.
regionStateNode.getProcedure() != null ? regionStateNode.getProcedure().getProcId()
@@ -161,7 +163,7 @@ public class RegionStateStore {
}
}
- private void updateMetaLocation(RegionInfo regionInfo, ServerName serverName, State state)
+ private void updateRootLocation(RegionInfo regionInfo, ServerName serverName, State state)
throws IOException {
try {
MetaTableLocator.setMetaLocation(master.getZooKeeper(), serverName, regionInfo.getReplicaId(),
@@ -171,15 +173,17 @@ public class RegionStateStore {
}
}
- private void updateUserRegionLocation(RegionInfo regionInfo, State state,
+ private void updateRegionLocation(RegionInfo regionInfo, State state,
ServerName regionLocation, long openSeqNum,
long pid) throws IOException {
+ TableName catalogTableName = regionInfo.getTable().equals(TableName.META_TABLE_NAME) ?
+ TableName.ROOT_TABLE_NAME : TableName.META_TABLE_NAME;
long time = EnvironmentEdgeManager.currentTime();
final int replicaId = regionInfo.getReplicaId();
- final Put put = new Put(MetaTableAccessor.getMetaKeyForRegion(regionInfo), time);
+ final Put put = new Put(MetaTableAccessor.getCatalogKeyForRegion(regionInfo), time);
MetaTableAccessor.addRegionInfo(put, regionInfo);
final StringBuilder info =
- new StringBuilder("pid=").append(pid).append(" updating hbase:meta row=")
+ new StringBuilder("pid=").append(pid).append(" updating "+catalogTableName+" row=")
.append(regionInfo.getEncodedName()).append(", regionState=").append(state);
if (openSeqNum >= 0) {
Preconditions.checkArgument(state == State.OPEN && regionLocation != null,
@@ -215,12 +219,13 @@ public class RegionStateStore {
.setValue(Bytes.toBytes(state.name()))
.build());
LOG.info(info.toString());
- updateRegionLocation(regionInfo, state, put);
+ updateRegionLocation(catalogTableName, regionInfo, state, put);
}
- private void updateRegionLocation(RegionInfo regionInfo, State state, Put put)
+ private void updateRegionLocation(TableName catalogTableName,
+ RegionInfo regionInfo, State state, Put put)
throws IOException {
- try (Table table = master.getConnection().getTable(TableName.META_TABLE_NAME)) {
+ try (Table table = master.getConnection().getTable(catalogTableName)) {
table.put(put);
} catch (IOException e) {
// TODO: Revist!!!! Means that if a server is loaded, then we will abort our host!
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
index 8d1593a..5a05d9e 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/RegionStates.java
@@ -379,6 +379,22 @@ public class RegionStates {
}
/**
+ * Call this when we start root log splitting a crashed Server.
+ * @see #metaLogSplit(ServerName)
+ */
+ public void rootLogSplitting(ServerName serverName) {
+ setServerState(serverName, ServerState.SPLITTING_ROOT);
+ }
+
+ /**
+ * Called after we've split the meta logs on a crashed Server.
+ * @see #metaLogSplitting(ServerName)
+ */
+ public void rootLogSplit(ServerName serverName) {
+ setServerState(serverName, ServerState.SPLITTING_ROOT_DONE);
+ }
+
+ /**
* Call this when we start meta log splitting a crashed Server.
* @see #metaLogSplit(ServerName)
*/
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
index c86a60e..2b1fc67 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerState.java
@@ -35,6 +35,19 @@ public enum ServerState {
CRASHED,
/**
+ * Only server which carries root can have this state. We will split wal for root and then
+ * assign root first before splitting other wals.
+ */
+ SPLITTING_ROOT,
+
+ /**
+ * Indicate that the root splitting is done. We need this state so that the UnassignProcedure
+ * for root can safely quit. See the comments in UnassignProcedure.remoteCallFailed for more
+ * details.
+ */
+ SPLITTING_ROOT_DONE,
+
+ /**
* Only server which carries meta can have this state. We will split wal for meta and then
* assign meta first before splitting other wals.
*/
@@ -57,4 +70,4 @@ public enum ServerState {
* quit. See the comments in UnassignProcedure.remoteCallFailed for more details.
*/
OFFLINE
-}
\ No newline at end of file
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
index b0a697d..b9327db 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/TransitRegionStateProcedure.java
@@ -166,13 +166,20 @@ public class TransitRegionStateProcedure
@Override
protected boolean waitInitialized(MasterProcedureEnv env) {
- if (TableName.isMetaTableName(getTableName())) {
+ if (TableName.isRootTableName(getTableName())) {
return false;
}
- // First we need meta to be loaded, and second, if meta is not online then we will likely to
- // fail when updating meta so we wait until it is assigned.
- AssignmentManager am = env.getAssignmentManager();
- return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, getRegion());
+ if (TableName.isMetaTableName(getTableName())) {
+ // First we need root to be loaded, and second, if root is not online then we will likely to
+ // fail when updating root so we wait until it is assigned.
+ AssignmentManager am = env.getAssignmentManager();
+ return am.waitRootLoaded(this) || am.waitRootAssigned(this, getRegion());
+ } else {
+ // First we need meta to be loaded, and second, if meta is not online then we will likely to
+ // fail when updating meta so we wait until it is assigned.
+ AssignmentManager am = env.getAssignmentManager();
+ return am.waitMetaLoaded(this) || am.waitMetaAssigned(this, getRegion());
+ }
}
private void queueAssign(MasterProcedureEnv env, RegionStateNode regionNode)
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
index 9690c18..fa52e9c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/HBCKServerCrashProcedure.java
@@ -65,8 +65,8 @@ public class HBCKServerCrashProcedure extends ServerCrashProcedure {
* @param carryingMeta True if carrying hbase:meta table region.
*/
public HBCKServerCrashProcedure(final MasterProcedureEnv env, final ServerName serverName,
- final boolean shouldSplitWal, final boolean carryingMeta) {
- super(env, serverName, shouldSplitWal, carryingMeta);
+ final boolean shouldSplitWal, final boolean carryingRoot, final boolean carryingMeta) {
+ super(env, serverName, shouldSplitWal, carryingRoot, carryingMeta);
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java
new file mode 100644
index 0000000..4fe216b
--- /dev/null
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitRootProcedure.java
@@ -0,0 +1,193 @@
+/**
+ * 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.master.procedure;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.concurrent.CountDownLatch;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.master.RegionState;
+import org.apache.hadoop.hbase.master.assignment.RegionStateStore;
+import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
+import org.apache.hadoop.hbase.procedure2.ProcedureStateSerializer;
+import org.apache.hadoop.hbase.procedure2.ProcedureSuspendedException;
+import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
+import org.apache.hadoop.hbase.procedure2.ProcedureYieldException;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
+import org.apache.hadoop.hbase.util.RetryCounter;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitRootState;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.MasterProcedureProtos.InitRootStateData;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ProcedureProtos;
+
+/**
+ * This procedure is used to initialize meta table for a new hbase deploy. It will just schedule an
+ * {@link TransitRegionStateProcedure} to assign meta.
+ */
+@InterfaceAudience.Private
+public class InitRootProcedure extends AbstractStateMachineTableProcedure<InitRootState> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(InitRootProcedure.class);
+
+ private CountDownLatch latch = new CountDownLatch(1);
+
+ private RetryCounter retryCounter;
+
+ @Override
+ public TableName getTableName() {
+ return TableName.ROOT_TABLE_NAME;
+ }
+
+ @Override
+ public TableOperationType getTableOperationType() {
+ return TableOperationType.CREATE;
+ }
+
+ @Override
+ protected Flow executeFromState(MasterProcedureEnv env, InitRootState state)
+ throws ProcedureSuspendedException, ProcedureYieldException, InterruptedException {
+ LOG.debug("Execute {}", this);
+ switch (state) {
+ case INIT_ROOT_ASSIGN_ROOT:
+ LOG.info("Going to assign root");
+ addChildProcedure(env.getAssignmentManager()
+ .createAssignProcedures(Arrays.asList(RegionInfoBuilder.ROOT_REGIONINFO)));
+ setNextState(MasterProcedureProtos.InitRootState.INIT_ROOT_LOAD_ROOT);
+ return Flow.HAS_MORE_STATE;
+ case INIT_ROOT_LOAD_ROOT:
+ try {
+ addMetaRegionToRoot(env);
+ env.getAssignmentManager().loadRoot();
+ } catch (IOException e) {
+ if (retryCounter == null) {
+ retryCounter = ProcedureUtil.createRetryCounter(env.getMasterConfiguration());
+ }
+ long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
+ LOG.warn("Failed to init default and system namespaces, suspend {}secs", backoff, e);
+ setTimeout(Math.toIntExact(backoff));
+ setState(ProcedureProtos.ProcedureState.WAITING_TIMEOUT);
+ skipPersistence();
+ throw new ProcedureSuspendedException();
+ }
+ case INIT_ROOT_INIT_META:
+ addChildProcedure(new InitMetaProcedure());
+ return Flow.NO_MORE_STATE;
+ default:
+ throw new UnsupportedOperationException("unhandled state=" + state);
+ }
+ }
+
+ @Override
+ protected boolean waitInitialized(MasterProcedureEnv env) {
+ // we do not need to wait for master initialized, we are part of the initialization.
+ return false;
+ }
+
+ @Override
+ protected synchronized boolean setTimeoutFailure(MasterProcedureEnv env) {
+ setState(ProcedureProtos.ProcedureState.RUNNABLE);
+ env.getProcedureScheduler().addFront(this);
+ return false;
+ }
+
+ @Override
+ protected LockState acquireLock(MasterProcedureEnv env) {
+ if (env.getProcedureScheduler().waitTableExclusiveLock(this, getTableName())) {
+ return LockState.LOCK_EVENT_WAIT;
+ }
+ return LockState.LOCK_ACQUIRED;
+ }
+
+ @Override
+ protected void rollbackState(MasterProcedureEnv env, InitRootState state)
+ throws IOException, InterruptedException {
+ throw new UnsupportedOperationException();
+ }
+
+ @Override
+ protected InitRootState getState(int stateId) {
+ return InitRootState.forNumber(stateId);
+ }
+
+ @Override
+ protected int getStateId(InitRootState state) {
+ return state.getNumber();
+ }
+
+ @Override
+ protected InitRootState getInitialState() {
+ return InitRootState.INIT_ROOT_ASSIGN_ROOT;
+ }
+
+ @Override
+ protected void serializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.serializeStateData(serializer);
+ serializer.serialize(InitRootStateData.getDefaultInstance());
+ }
+
+ @Override
+ protected void deserializeStateData(ProcedureStateSerializer serializer) throws IOException {
+ super.deserializeStateData(serializer);
+ serializer.deserialize(InitRootStateData.class);
+ }
+
+ @Override
+ protected void completionCleanup(MasterProcedureEnv env) {
+ latch.countDown();
+ }
+
+ public void await() throws InterruptedException {
+ latch.await();
+ }
+
+
+ public void addMetaRegionToRoot(MasterProcedureEnv env) throws IOException {
+ Table rooTable = env.getMasterServices().getConnection().getTable(TableName.ROOT_TABLE_NAME);
+ // The row key is the region name
+ byte[] row = HRegionInfo.FIRST_META_REGIONINFO.getRegionName();
+ Put put = new Put(row);
+ final long now = EnvironmentEdgeManager.currentTime();
+ put.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
+ HConstants.REGIONINFO_QUALIFIER,
+ now,
+ RegionInfo.toByteArray(HRegionInfo.FIRST_META_REGIONINFO)));
+ // Set into the root table the version of the meta table.
+ put.add(new KeyValue(row,
+ HConstants.CATALOG_FAMILY,
+ HConstants.META_VERSION_QUALIFIER,
+ now,
+ Bytes.toBytes(HConstants.META_VERSION)));
+ put.add(new KeyValue(row,
+ HConstants.CATALOG_FAMILY,
+ HConstants.STATE_QUALIFIER,
+ now,
+ Bytes.toBytes(RegionState.State.OFFLINE.name())));
+ rooTable.put(put);
+ }
+}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
index 23d6ecb..efe3d83 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/MasterProcedureUtil.java
@@ -180,7 +180,13 @@ public final class MasterProcedureUtil {
* server carrying meta, and 1 for others.
*/
public static int getServerPriority(ServerProcedureInterface proc) {
- return proc.hasMetaTableRegion() ? 100 : 1;
+ if (proc.hasRootTableRegion()) {
+ //TODO francis is 200 a reasonable number?
+ return 200;
+ } else if (proc.hasMetaTableRegion()) {
+ return 100;
+ }
+ return 1;
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
index a6ebbaa..9ced487 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerCrashProcedure.java
@@ -83,6 +83,7 @@ public class ServerCrashProcedure
*/
private List<RegionInfo> regionsOnCrashedServer;
+ private boolean carryingRoot = false;
private boolean carryingMeta = false;
private boolean shouldSplitWal;
private MonitoredTask status;
@@ -94,12 +95,17 @@ public class ServerCrashProcedure
* Call this constructor queuing up a Procedure.
* @param serverName Name of the crashed server.
* @param shouldSplitWal True if we should split WALs as part of crashed server processing.
- * @param carryingMeta True if carrying hbase:meta table region.
+ * @param carryingRoot True if carrying hbase:root table region.
+ * @param carryingMeta True if carrying hbase:meta table region. Although carryingMeta is
+ * determined dynamically by an SCP instance. Caller can give the current
+ * state it sees, this information might be useful to SCP down the road or
+ * for debugging.
*/
public ServerCrashProcedure(final MasterProcedureEnv env, final ServerName serverName,
- final boolean shouldSplitWal, final boolean carryingMeta) {
+ final boolean shouldSplitWal, final boolean carryingRoot, final boolean carryingMeta) {
this.serverName = serverName;
this.shouldSplitWal = shouldSplitWal;
+ this.carryingRoot = carryingRoot;
this.carryingMeta = carryingMeta;
this.setOwner(env.getRequestUser());
}
@@ -131,22 +137,81 @@ public class ServerCrashProcedure
switch (state) {
case SERVER_CRASH_START:
+ break;
+
+ //Don't block hbase:root processing states on hbase:meta being loaded
+ case SERVER_CRASH_SPLIT_ROOT_LOGS:
+ case SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR:
+ case SERVER_CRASH_ASSIGN_ROOT:
+ break;
+
+ case SERVER_CRASH_CHECK_CARRYING_META:
+ // If hbase:root is not loaded, we can't do the check so yield
+ if (env.getAssignmentManager().waitRootLoaded(this)) {
+ LOG.info("pid="+getProcId()+", waiting for root loaded: "+state+
+ ", carryingRoot="+carryingRoot+", carryingMeta="+carryingMeta);
+ throw new ProcedureSuspendedException();
+ }
+ break;
+
+ //Don't block hbase:meta processing states on hbase:meta being loaded
case SERVER_CRASH_SPLIT_META_LOGS:
case SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR:
case SERVER_CRASH_ASSIGN_META:
break;
+
default:
// If hbase:meta is not assigned, yield.
if (env.getAssignmentManager().waitMetaLoaded(this)) {
+ LOG.info("pid="+getProcId()+", waiting for meta loaded: "+state+
+ ", carryingRoot="+carryingRoot+", carryingMeta="+carryingMeta);
throw new ProcedureSuspendedException();
}
}
+
try {
switch (state) {
case SERVER_CRASH_START:
LOG.info("Start " + this);
// If carrying meta, process it first. Else, get list of regions on crashed server.
- if (this.carryingMeta) {
+ if (this.carryingRoot) {
+ setNextState(ServerCrashState.SERVER_CRASH_SPLIT_ROOT_LOGS);
+ } else {
+ setNextState(ServerCrashState.SERVER_CRASH_CHECK_CARRYING_META);
+ }
+ break;
+ case SERVER_CRASH_SPLIT_ROOT_LOGS:
+ if (env.getMasterConfiguration().getBoolean(HBASE_SPLIT_WAL_COORDINATED_BY_ZK,
+ DEFAULT_HBASE_SPLIT_COORDINATED_BY_ZK)) {
+ splitRootLogs(env);
+ setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_ROOT);
+ } else {
+ am.getRegionStates().rootLogSplitting(serverName);
+ addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.ROOT));
+ setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR);
+ }
+ break;
+ case SERVER_CRASH_DELETE_SPLIT_ROOT_WALS_DIR:
+ if(isSplittingDone(env, SplitWALManager.SplitType.ROOT)){
+ //TODO francis are we cleaning all the dirs?
+ cleanupSplitDir(env);
+ setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_ROOT);
+ am.getRegionStates().rootLogSplit(serverName);
+ } else {
+ setNextState(ServerCrashState.SERVER_CRASH_SPLIT_ROOT_LOGS);
+ }
+ break;
+ case SERVER_CRASH_ASSIGN_ROOT:
+ assignRegions(env, Arrays.asList(RegionInfoBuilder.ROOT_REGIONINFO));
+ setNextState(ServerCrashState.SERVER_CRASH_CHECK_CARRYING_META);
+ break;
+ case SERVER_CRASH_CHECK_CARRYING_META:
+ boolean currCarryingMeta = am.isCarryingMeta(serverName);
+ if (carryingMeta && !currCarryingMeta) {
+ LOG.error("pid="+getProcId()+", carryingMeta changed to false after SCP check");
+ }
+ carryingMeta = currCarryingMeta;
+ if (carryingMeta) {
setNextState(ServerCrashState.SERVER_CRASH_SPLIT_META_LOGS);
} else {
setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
@@ -159,13 +224,12 @@ public class ServerCrashProcedure
setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_META);
} else {
am.getRegionStates().metaLogSplitting(serverName);
- addChildProcedure(createSplittingWalProcedures(env, true));
+ addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.META));
setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR);
}
break;
case SERVER_CRASH_DELETE_SPLIT_META_WALS_DIR:
- if(isSplittingDone(env, true)){
- cleanupSplitDir(env);
+ if (isSplittingDone(env, SplitWALManager.SplitType.META)) {
setNextState(ServerCrashState.SERVER_CRASH_ASSIGN_META);
am.getRegionStates().metaLogSplit(serverName);
} else {
@@ -199,12 +263,12 @@ public class ServerCrashProcedure
setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
} else {
am.getRegionStates().logSplitting(this.serverName);
- addChildProcedure(createSplittingWalProcedures(env, false));
+ addChildProcedure(createSplittingWalProcedures(env, SplitWALManager.SplitType.USER));
setNextState(ServerCrashState.SERVER_CRASH_DELETE_SPLIT_WALS_DIR);
}
break;
case SERVER_CRASH_DELETE_SPLIT_WALS_DIR:
- if (isSplittingDone(env, false)) {
+ if (isSplittingDone(env, SplitWALManager.SplitType.USER)) {
cleanupSplitDir(env);
setNextState(ServerCrashState.SERVER_CRASH_ASSIGN);
am.getRegionStates().logSplit(this.serverName);
@@ -262,22 +326,23 @@ public class ServerCrashProcedure
}
}
- private boolean isSplittingDone(MasterProcedureEnv env, boolean splitMeta) {
- LOG.debug("check if splitting WALs of {} done? isMeta: {}", serverName, splitMeta);
+ private boolean isSplittingDone(MasterProcedureEnv env, SplitWALManager.SplitType splitType) {
+ LOG.debug("check if splitting WALs of {} done? splittype: {}", serverName, splitType);
SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
try {
- return splitWALManager.getWALsToSplit(serverName, splitMeta).size() == 0;
+ return splitWALManager.getWALsToSplit(serverName, splitType).size() == 0;
} catch (IOException e) {
LOG.warn("get filelist of serverName {} failed, retry...", serverName, e);
return false;
}
}
- private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env, boolean splitMeta)
+ private Procedure[] createSplittingWalProcedures(MasterProcedureEnv env,
+ SplitWALManager.SplitType splitType)
throws IOException {
- LOG.info("Splitting WALs {}, isMeta: {}", this, splitMeta);
+ LOG.info("Splitting WALs {}, SplitType: {}", this, splitType);
SplitWALManager splitWALManager = env.getMasterServices().getSplitWALManager();
- List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitMeta);
+ List<Procedure> procedures = splitWALManager.splitWALs(serverName, splitType);
return procedures.toArray(new Procedure[procedures.size()]);
}
@@ -293,6 +358,16 @@ public class ServerCrashProcedure
return hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri);
}
+ private void splitRootLogs(MasterProcedureEnv env) throws IOException {
+ LOG.debug("Splitting root WALs {}", this);
+ MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
+ AssignmentManager am = env.getMasterServices().getAssignmentManager();
+ am.getRegionStates().rootLogSplitting(serverName);
+ mwm.splitRootLog(serverName);
+ am.getRegionStates().rootLogSplit(serverName);
+ LOG.debug("Done splitting root WALs {}", this);
+ }
+
private void splitMetaLogs(MasterProcedureEnv env) throws IOException {
LOG.debug("Splitting meta WALs {}", this);
MasterWalManager mwm = env.getMasterServices().getMasterWalManager();
@@ -312,8 +387,11 @@ public class ServerCrashProcedure
// of SCPs running because big cluster crashed down.
am.getRegionStates().logSplitting(this.serverName);
mwm.splitLog(this.serverName);
+ if (!carryingRoot) {
+ mwm.archiveCatalogLog(this.serverName, true);
+ }
if (!carryingMeta) {
- mwm.archiveMetaLog(this.serverName);
+ mwm.archiveCatalogLog(this.serverName, false);
}
am.getRegionStates().logSplit(this.serverName);
LOG.debug("Done splitting WALs {}", this);
@@ -384,6 +462,8 @@ public class ServerCrashProcedure
sb.append(getProcName());
sb.append(", splitWal=");
sb.append(shouldSplitWal);
+ sb.append(", root=");
+ sb.append(carryingRoot);
sb.append(", meta=");
sb.append(carryingMeta);
}
@@ -400,6 +480,7 @@ public class ServerCrashProcedure
MasterProcedureProtos.ServerCrashStateData.Builder state =
MasterProcedureProtos.ServerCrashStateData.newBuilder().
setServerName(ProtobufUtil.toServerName(this.serverName)).
+ setCarryingRoot(this.carryingRoot).
setCarryingMeta(this.carryingMeta).
setShouldSplitWal(this.shouldSplitWal);
if (this.regionsOnCrashedServer != null && !this.regionsOnCrashedServer.isEmpty()) {
@@ -418,6 +499,7 @@ public class ServerCrashProcedure
MasterProcedureProtos.ServerCrashStateData state =
serializer.deserialize(MasterProcedureProtos.ServerCrashStateData.class);
this.serverName = ProtobufUtil.toServerName(state.getServerName());
+ this.carryingRoot = state.hasCarryingRoot()? state.getCarryingRoot(): false;
this.carryingMeta = state.hasCarryingMeta()? state.getCarryingMeta(): false;
// shouldSplitWAL has a default over in pb so this invocation will always work.
this.shouldSplitWal = state.getShouldSplitWal();
@@ -437,6 +519,11 @@ public class ServerCrashProcedure
}
@Override
+ public boolean hasRootTableRegion() {
+ return this.carryingRoot;
+ }
+
+ @Override
public boolean hasMetaTableRegion() {
return this.carryingMeta;
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
index 8162269..55c74d4 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ServerProcedureInterface.java
@@ -49,6 +49,11 @@ public interface ServerProcedureInterface {
/**
* @return True if this server has an hbase:meta table region.
*/
+ boolean hasRootTableRegion();
+
+ /**
+ * @return True if this server has an hbase:meta table region.
+ */
boolean hasMetaTableRegion();
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
index 4ae408f..bcead65 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALProcedure.java
@@ -173,6 +173,10 @@ public class SplitWALProcedure
return this.crashedServer;
}
+ @Override public boolean hasRootTableRegion() {
+ return AbstractFSWALProvider.isRootFile(new Path(walPath));
+ }
+
@Override
public boolean hasMetaTableRegion() {
return AbstractFSWALProvider.isMetaFile(new Path(walPath));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
index c829e51..c71732a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SplitWALRemoteProcedure.java
@@ -122,6 +122,10 @@ public class SplitWALRemoteProcedure extends ServerRemoteProcedure
return this.crashedServer;
}
+ @Override public boolean hasRootTableRegion() {
+ return AbstractFSWALProvider.isRootFile(new Path(walPath));
+ }
+
@Override
public boolean hasMetaTableRegion() {
return AbstractFSWALProvider.isMetaFile(new Path(walPath));
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
index 4dd84ca..6b935e9 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleProcedure.java
@@ -142,6 +142,10 @@ public class SwitchRpcThrottleProcedure
return serverName;
}
+ @Override public boolean hasRootTableRegion() {
+ return false;
+ }
+
@Override
public boolean hasMetaTableRegion() {
return false;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
index 8ce9aef..fe4cb17 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/SwitchRpcThrottleRemoteProcedure.java
@@ -89,6 +89,10 @@ public class SwitchRpcThrottleRemoteProcedure extends ServerRemoteProcedure
return targetServer;
}
+ @Override public boolean hasRootTableRegion() {
+ return false;
+ }
+
@Override
public boolean hasMetaTableRegion() {
return false;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
index 1ff4f38..fef1fa5 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java
@@ -7609,19 +7609,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
* @return true if the row is within the range specified by the RegionInfo
*/
public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
- return ((info.getStartKey().length == 0) ||
- (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
- ((info.getEndKey().length == 0) ||
- (Bytes.compareTo(info.getEndKey(), row) > 0));
+ return info.containsRow(row, 0, (short)row.length);
}
public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
final short length) {
- return ((info.getStartKey().length == 0) ||
- (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
- row, offset, length) <= 0)) &&
- ((info.getEndKey().length == 0) ||
- (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
+ return info.containsRow(row, offset, length);
}
@Override
@@ -8869,7 +8862,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public CellComparator getCellComparator() {
- return cellComparator;
+ if (this.getRegionInfo().isRootRegion()) {
+ return CellComparatorImpl.ROOT_COMPARATOR;
+ }
+ if (this.getRegionInfo().isMetaRegion()) {
+ return CellComparatorImpl.META_COMPARATOR;
+ }
+ return CellComparatorImpl.COMPARATOR;
}
public long getMemStoreFlushSize() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
index 1f064f2..a266dbf 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSTableDescriptors.java
@@ -120,20 +120,39 @@ public class FSTableDescriptors implements TableDescriptors {
}
@VisibleForTesting
- public static void tryUpdateMetaTableDescriptor(Configuration conf) throws IOException {
- tryUpdateMetaTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
+ public static void tryUpdateCatalogTableDescriptor(Configuration conf) throws IOException {
+ tryUpdateCatalogTableDescriptor(conf, CommonFSUtils.getCurrentFileSystem(conf),
CommonFSUtils.getRootDir(conf), null);
}
- public static void tryUpdateMetaTableDescriptor(Configuration conf, FileSystem fs, Path rootdir,
- Function<TableDescriptorBuilder, TableDescriptorBuilder> metaObserver) throws IOException {
+ public static void tryUpdateCatalogTableDescriptor(Configuration conf, FileSystem fs, Path rootdir,
+ Function<TableDescriptorBuilder, TableDescriptorBuilder> catalogObserver) throws IOException {
+ // see if we already have root descriptor on fs. Write one if not.
+ try {
+ getTableDescriptorFromFs(fs, rootdir, TableName.ROOT_TABLE_NAME);
+ } catch (TableInfoMissingException e) {
+ TableDescriptorBuilder builder = createRootTableDescriptorBuilder(conf);
+ if (catalogObserver != null) {
+ builder = catalogObserver.apply(builder);
+ }
+ TableDescriptor td = builder.build();
+ LOG.info("Creating new hbase:root table descriptor {}", td);
+ TableName tableName = td.getTableName();
+ Path tableDir = CommonFSUtils.getTableDir(rootdir, tableName);
+ Path p = writeTableDescriptor(fs, td, tableDir, getTableInfoPath(fs, tableDir, true));
+ if (p == null) {
+ throw new IOException("Failed update hbase:root table descriptor");
+ }
+ LOG.info("Updated hbase:root table descriptor to {}", p);
+ }
+
// see if we already have meta descriptor on fs. Write one if not.
try {
getTableDescriptorFromFs(fs, rootdir, TableName.META_TABLE_NAME);
} catch (TableInfoMissingException e) {
TableDescriptorBuilder builder = createMetaTableDescriptorBuilder(conf);
- if (metaObserver != null) {
- builder = metaObserver.apply(builder);
+ if (catalogObserver != null) {
+ builder = catalogObserver.apply(builder);
}
TableDescriptor td = builder.build();
LOG.info("Creating new hbase:meta table descriptor {}", td);
@@ -199,6 +218,45 @@ public class FSTableDescriptors implements TableDescriptors {
this.cache.clear();
}
+ private static TableDescriptorBuilder createRootTableDescriptorBuilder(final Configuration conf)
+ throws IOException {
+ // TODO We used to set CacheDataInL1 for META table. When we have BucketCache in file mode, now
+ // the META table data goes to File mode BC only. Test how that affect the system. If too much,
+ // we have to rethink about adding back the setCacheDataInL1 for META table CFs.
+ return TableDescriptorBuilder.newBuilder(TableName.ROOT_TABLE_NAME)
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.CATALOG_FAMILY)
+ .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+ HConstants.DEFAULT_HBASE_META_VERSIONS))
+ .setInMemory(true)
+ .setBlocksize(conf.getInt(HConstants.HBASE_META_BLOCK_SIZE,
+ HConstants.DEFAULT_HBASE_META_BLOCK_SIZE))
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+ // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
+ .setBloomFilterType(BloomType.NONE)
+ .build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder.newBuilder(HConstants.TABLE_FAMILY)
+ .setMaxVersions(conf.getInt(HConstants.HBASE_META_VERSIONS,
+ HConstants.DEFAULT_HBASE_META_VERSIONS))
+ .setInMemory(true)
+ .setBlocksize(8 * 1024)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+ // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
+ .setBloomFilterType(BloomType.NONE)
+ .build())
+ .setColumnFamily(ColumnFamilyDescriptorBuilder
+ .newBuilder(HConstants.REPLICATION_BARRIER_FAMILY)
+ .setMaxVersions(HConstants.ALL_VERSIONS)
+ .setInMemory(true)
+ .setScope(HConstants.REPLICATION_SCOPE_LOCAL)
+ // Disable blooms for meta. Needs work. Seems to mess w/ getClosestOrBefore.
+ .setBloomFilterType(BloomType.NONE)
+ .build())
+ .setCoprocessor(CoprocessorDescriptorBuilder.newBuilder(
+ MultiRowMutationEndpoint.class.getName())
+ .setPriority(Coprocessor.PRIORITY_SYSTEM)
+ .build());
+ }
+
@VisibleForTesting
public boolean isUsecache() {
return this.usecache;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
index ce6770f..49876ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AbstractFSWALProvider.java
@@ -247,6 +247,11 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
// should be package private; more visible for use in AbstractFSWAL
public static final String WAL_FILE_NAME_DELIMITER = ".";
+
+ /** The hbase:root region's WAL filename extension */
+ @VisibleForTesting
+ public static final String ROOT_WAL_PROVIDER_ID = ".root";
+
/** The hbase:meta region's WAL filename extension */
@VisibleForTesting
public static final String META_WAL_PROVIDER_ID = ".meta";
@@ -410,6 +415,25 @@ public abstract class AbstractFSWALProvider<T extends AbstractFSWAL<?>> implemen
return serverName;
}
+ public static boolean isCatalogFile(Path p) {
+ return isRootFile(p) || isMetaFile(p);
+ }
+
+ public static boolean isCatalogFile(String p) {
+ return isRootFile(p) || isMetaFile(p);
+ }
+
+ public static boolean isRootFile(Path p) {
+ return isRootFile(p.getName());
+ }
+
+ public static boolean isRootFile(String p) {
+ if (p != null && p.endsWith(ROOT_WAL_PROVIDER_ID)) {
+ return true;
+ }
+ return false;
+ }
+
public static boolean isMetaFile(Path p) {
return isMetaFile(p.getName());
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
index 88b0140..48397ab 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/AsyncFSWALProvider.java
@@ -67,10 +67,16 @@ public class AsyncFSWALProvider extends AbstractFSWALProvider<AsyncFSWAL> {
private Class<? extends Channel> channelClass;
@Override
protected AsyncFSWAL createWAL() throws IOException {
+ String suffix = null;
+ if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = ROOT_WAL_PROVIDER_ID;
+ } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = META_WAL_PROVIDER_ID;
+ }
return new AsyncFSWAL(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
getWALDirectoryName(factory.factoryId),
getWALArchiveDirectoryName(conf, factory.factoryId), conf, listeners, true, logPrefix,
- META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null,
+ suffix,
eventLoopGroup, channelClass);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
index ea78cf2..d1be4fb 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedRecoveredHFilesOutputSink.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.hbase.wal;
import static org.apache.hadoop.hbase.TableName.META_TABLE_NAME;
+import static org.apache.hadoop.hbase.TableName.ROOT_TABLE_NAME;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.HashMap;
@@ -76,6 +77,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
Map<String, CellSet> familyCells = new HashMap<>();
Map<String, Long> familySeqIds = new HashMap<>();
boolean isMetaTable = buffer.tableName.equals(META_TABLE_NAME);
+ boolean isRootTable = buffer.tableName.equals(ROOT_TABLE_NAME);
// First iterate all Cells to find which column families are present and to stamp Cell with
// sequence id.
for (WAL.Entry entry : buffer.entries) {
@@ -91,7 +93,9 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
familyCells
.computeIfAbsent(familyName,
key -> new CellSet(
- isMetaTable ? CellComparatorImpl.META_COMPARATOR : CellComparatorImpl.COMPARATOR))
+ isRootTable ? CellComparatorImpl.ROOT_COMPARATOR :
+ (isMetaTable ? CellComparatorImpl.META_COMPARATOR :
+ CellComparatorImpl.COMPARATOR)))
.add(cell);
familySeqIds.compute(familyName, (k, v) -> v == null ? seqId : Math.max(v, seqId));
}
@@ -102,8 +106,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
for (Map.Entry<String, CellSet> cellsEntry : familyCells.entrySet()) {
String familyName = cellsEntry.getKey();
StoreFileWriter writer = createRecoveredHFileWriter(buffer.tableName, regionName,
- familySeqIds.get(familyName), familyName, isMetaTable);
- LOG.trace("Created {}", writer.getPath());
+ familySeqIds.get(familyName), familyName, isRootTable, isMetaTable);
openingWritersNum.incrementAndGet();
try {
for (Cell cell : cellsEntry.getValue()) {
@@ -191,7 +194,7 @@ public class BoundedRecoveredHFilesOutputSink extends OutputSink {
* given hfile has metadata on how it was written.
*/
private StoreFileWriter createRecoveredHFileWriter(TableName tableName, String regionName,
- long seqId, String familyName, boolean isMetaTable) throws IOException {
+ long seqId, String familyName, boolean isRootTable, boolean isMetaTable) throws IOException {
Path outputDir = WALSplitUtil.tryCreateRecoveredHFilesDir(walSplitter.rootFS, walSplitter.conf,
tableName, regionName, familyName);
StoreFileWriter.Builder writerBuilder =
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
index 3b91c24..458133f 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/FSHLogProvider.java
@@ -101,10 +101,16 @@ public class FSHLogProvider extends AbstractFSWALProvider<FSHLog> {
@Override
protected FSHLog createWAL() throws IOException {
+ String suffix = null;
+ if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = ROOT_WAL_PROVIDER_ID;
+ } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = META_WAL_PROVIDER_ID;
+ }
return new FSHLog(CommonFSUtils.getWALFileSystem(conf), CommonFSUtils.getWALRootDir(conf),
getWALDirectoryName(factory.factoryId),
getWALArchiveDirectoryName(conf, factory.factoryId), conf, listeners, true, logPrefix,
- META_WAL_PROVIDER_ID.equals(providerId) ? META_WAL_PROVIDER_ID : null);
+ suffix);
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
index 2fd8288..f512398 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/RegionGroupingProvider.java
@@ -19,6 +19,7 @@
package org.apache.hadoop.hbase.wal;
import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.META_WAL_PROVIDER_ID;
+import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.ROOT_WAL_PROVIDER_ID;
import static org.apache.hadoop.hbase.wal.AbstractFSWALProvider.WAL_FILE_NAME_DELIMITER;
import java.io.IOException;
@@ -123,6 +124,7 @@ public class RegionGroupingProvider implements WALProvider {
public static final String DEFAULT_DELEGATE_PROVIDER = WALFactory.Providers.defaultProvider
.name();
+ private static final String ROOT_WAL_GROUP_NAME = "root";
private static final String META_WAL_GROUP_NAME = "meta";
/** A group-provider mapping, make sure one-one rather than many-one mapping */
@@ -143,8 +145,8 @@ public class RegionGroupingProvider implements WALProvider {
}
this.factory = factory;
- if (META_WAL_PROVIDER_ID.equals(providerId)) {
- // do not change the provider id if it is for meta
+ if (ROOT_WAL_PROVIDER_ID.equals(providerId) || META_WAL_PROVIDER_ID.equals(providerId)) {
+ // do not change the provider id if it is for root/meta
this.providerId = providerId;
} else {
StringBuilder sb = new StringBuilder().append(factory.factoryId);
@@ -166,11 +168,13 @@ public class RegionGroupingProvider implements WALProvider {
}
private WALProvider createProvider(String group) throws IOException {
- if (META_WAL_PROVIDER_ID.equals(providerId)) {
- return factory.createProvider(providerClass, META_WAL_PROVIDER_ID);
- } else {
- return factory.createProvider(providerClass, group);
+ String suffix = group;
+ if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = ROOT_WAL_PROVIDER_ID;
+ } else if (META_WAL_PROVIDER_ID.equals(providerId)) {
+ suffix = META_WAL_PROVIDER_ID;
}
+ return factory.createProvider(providerClass, suffix);
}
@Override
@@ -199,7 +203,9 @@ public class RegionGroupingProvider implements WALProvider {
@Override
public WAL getWAL(RegionInfo region) throws IOException {
String group;
- if (META_WAL_PROVIDER_ID.equals(this.providerId)) {
+ if (ROOT_WAL_PROVIDER_ID.equals(providerId)) {
+ group = ROOT_WAL_GROUP_NAME;
+ } else if (META_WAL_PROVIDER_ID.equals(this.providerId)) {
group = META_WAL_GROUP_NAME;
} else {
byte[] id;
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
index 30bb77e..68f6541 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WALFactory.java
@@ -81,12 +81,20 @@ public class WALFactory {
public static final String WAL_PROVIDER = "hbase.wal.provider";
static final String DEFAULT_WAL_PROVIDER = Providers.defaultProvider.name();
+ public static final String ROOT_WAL_PROVIDER = "hbase.wal.root_provider";
+
public static final String META_WAL_PROVIDER = "hbase.wal.meta_provider";
public static final String WAL_ENABLED = "hbase.regionserver.hlog.enabled";
final String factoryId;
private final WALProvider provider;
+
+ // The root updates are written to a different wal. If this
+ // regionserver holds root regions, then this ref will be non-null.
+ // lazily intialized; most RegionServers don't deal with ROOT
+ private final AtomicReference<WALProvider> rootProvider = new AtomicReference<>();
+
// The meta updates are written to a different wal. If this
// regionserver holds meta regions, then this ref will be non-null.
// lazily intialized; most RegionServers don't deal with META
@@ -212,6 +220,10 @@ public class WALFactory {
* factory.
*/
public void close() throws IOException {
+ final WALProvider rootProvider = this.rootProvider.get();
+ if (null != rootProvider) {
+ rootProvider.close();
+ }
final WALProvider metaProvider = this.metaProvider.get();
if (null != metaProvider) {
metaProvider.close();
@@ -230,6 +242,14 @@ public class WALFactory {
*/
public void shutdown() throws IOException {
IOException exception = null;
+ final WALProvider rootProvider = this.rootProvider.get();
+ if (null != rootProvider) {
+ try {
+ rootProvider.shutdown();
+ } catch(IOException ioe) {
+ exception = ioe;
+ }
+ }
final WALProvider metaProvider = this.metaProvider.get();
if (null != metaProvider) {
try {
@@ -249,6 +269,35 @@ public class WALFactory {
}
@VisibleForTesting
+ WALProvider getRootProvider() throws IOException {
+ for (;;) {
+ WALProvider provider = this.rootProvider.get();
+ if (provider != null) {
+ return provider;
+ }
+ Class<? extends WALProvider> clz = null;
+ if (conf.get(ROOT_WAL_PROVIDER) == null) {
+ try {
+ clz = conf.getClass(WAL_PROVIDER, Providers.defaultProvider.clazz, WALProvider.class);
+ } catch (Throwable t) {
+ // the WAL provider should be an enum. Proceed
+ }
+ }
+ if (clz == null){
+ clz = getProviderClass(ROOT_WAL_PROVIDER, conf.get(WAL_PROVIDER, DEFAULT_WAL_PROVIDER));
+ }
+ provider = createProvider(clz, AbstractFSWALProvider.ROOT_WAL_PROVIDER_ID);
+ if (rootProvider.compareAndSet(null, provider)) {
+ return provider;
+ } else {
+ // someone is ahead of us, close and try again.
+ provider.close();
+ }
+ }
+ }
+
+
+ @VisibleForTesting
WALProvider getMetaProvider() throws IOException {
for (;;) {
WALProvider provider = this.metaProvider.get();
@@ -280,8 +329,11 @@ public class WALFactory {
* @param region the region which we want to get a WAL for it. Could be null.
*/
public WAL getWAL(RegionInfo region) throws IOException {
- // use different WAL for hbase:meta
- if (region != null && region.isMetaRegion() &&
+ // use different WAL for hbase:root and hbase:meta
+ if (region != null && region.isRootRegion() &&
+ region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
+ return getRootProvider().getWAL(region);
+ } else if (region != null && region.isMetaRegion() &&
region.getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
return getMetaProvider().getWAL(region);
} else {
@@ -482,6 +534,10 @@ public class WALFactory {
return this.provider;
}
+ public final WALProvider getRootWALProvider() {
+ return this.rootProvider.get();
+ }
+
public final WALProvider getMetaWALProvider() {
return this.metaProvider.get();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
index 23ab048..b49cea3 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseCluster.java
@@ -372,6 +372,13 @@ public abstract class HBaseCluster implements Closeable, Configurable {
public boolean restoreClusterMetrics(ClusterMetrics desiredStatus) throws IOException {
return true;
}
+ /**
+ * Get the ServerName of region server serving the first hbase:meta region
+ */
+ public ServerName getServerHoldingRoot() throws IOException {
+ return getServerHoldingRegion(TableName.ROOT_TABLE_NAME,
+ RegionInfoBuilder.ROOT_REGIONINFO.getRegionName());
+ }
/**
* Get the ServerName of region server serving the first hbase:meta region
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
index d6782fa..df8867b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableAccessor.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.ipc.CallRunner;
import org.apache.hadoop.hbase.ipc.DelegatingRpcScheduler;
@@ -62,6 +63,7 @@ import org.apache.hadoop.hbase.regionserver.SimpleRpcSchedulerFactory;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
import org.apache.hadoop.hbase.util.Pair;
@@ -106,6 +108,8 @@ public class TestMetaTableAccessor {
c.setLong("hbase.client.pause", 1000);
c.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10);
connection = ConnectionFactory.createConnection(c);
+ LOG.info("-->FS dump");
+ CommonFSUtils.logFileSystemState(UTIL.getTestFileSystem(), UTIL.getDefaultRootDirPath(), LOG);
}
@AfterClass public static void afterClass() throws Exception {
@@ -182,7 +186,7 @@ public class TestMetaTableAccessor {
@Test
public void testIsMetaWhenAllHealthy() throws InterruptedException {
HMaster m = UTIL.getMiniHBaseCluster().getMaster();
- assertTrue(m.waitForMetaOnline());
+ assertTrue(m.assignmentManager.waitForMetaOnline());
}
@Test
@@ -191,7 +195,7 @@ public class TestMetaTableAccessor {
int index = UTIL.getMiniHBaseCluster().getServerWithMeta();
HRegionServer rsWithMeta = UTIL.getMiniHBaseCluster().getRegionServer(index);
rsWithMeta.abort("TESTING");
- assertTrue(m.waitForMetaOnline());
+ assertTrue(m.assignmentManager.waitForMetaOnline());
}
/**
@@ -251,6 +255,13 @@ public class TestMetaTableAccessor {
startTime + timeOut < System.currentTimeMillis());
if (index != -1){
+ ServerName rootServerName = UTIL.getMiniHBaseCluster().getRegionServer(index).getServerName();
+ UTIL.getAdmin().move(HRegionInfo.ROOT_REGIONINFO.getEncodedNameAsBytes(),
+ rootServerName.toString().getBytes());
+ while (!UTIL.getConnection().getRegionLocator(TableName.ROOT_TABLE_NAME)
+ .getAllRegionLocations().get(0).getServerName().equals(rootServerName)) {
+ Thread.sleep(100);
+ }
UTIL.getMiniHBaseCluster().abortRegionServer(index);
UTIL.getMiniHBaseCluster().waitOnRegionServer(index);
}
@@ -1000,5 +1011,20 @@ public class TestMetaTableAccessor {
encodedName);
assertNull(result);
}
+
+ @Test
+ public void scanUserTable() throws IOException {
+ Connection conn = UTIL.getConnection();
+ TableName tableName = TableName.valueOf("foo");
+ byte[] familyName = Bytes.toBytes("f");
+ HTableDescriptor desc = new HTableDescriptor(tableName);
+ desc.addFamily(new HColumnDescriptor(familyName));
+ conn.getAdmin().createTable(desc);
+ Table table = conn.getTable(tableName);
+ table.put(new Put(new byte[]{'r'}).addColumn(familyName, new byte[]{'q'}, new byte[]{'v'}));
+ for (Result res : table.getScanner(new Scan())) {
+ System.out.println("-->"+res);
+ }
+ }
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
index 9274fa0..b5ffaad 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestMetaTableLocator.java
@@ -118,25 +118,25 @@ public class TestMetaTableLocator {
Mockito.when(client.get((RpcController) Mockito.any(), (GetRequest) Mockito.any()))
.thenReturn(GetResponse.newBuilder().build());
- assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
+ assertNull(MetaTableLocator.getRootRegionLocation(this.watcher));
for (RegionState.State state : RegionState.State.values()) {
if (state.equals(RegionState.State.OPEN)) {
continue;
}
MetaTableLocator.setMetaLocation(this.watcher, SN, state);
- assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
- assertEquals(state, MetaTableLocator.getMetaRegionState(this.watcher).getState());
+ assertNull(MetaTableLocator.getRootRegionLocation(this.watcher));
+ assertEquals(state, MetaTableLocator.getRootRegionState(this.watcher).getState());
}
MetaTableLocator.setMetaLocation(this.watcher, SN, RegionState.State.OPEN);
- assertEquals(SN, MetaTableLocator.getMetaRegionLocation(this.watcher));
+ assertEquals(SN, MetaTableLocator.getRootRegionLocation(this.watcher));
assertEquals(RegionState.State.OPEN,
- MetaTableLocator.getMetaRegionState(this.watcher).getState());
+ MetaTableLocator.getRootRegionState(this.watcher).getState());
MetaTableLocator.deleteMetaLocation(this.watcher);
- assertNull(MetaTableLocator.getMetaRegionState(this.watcher).getServerName());
+ assertNull(MetaTableLocator.getRootRegionState(this.watcher).getServerName());
assertEquals(RegionState.State.OFFLINE,
- MetaTableLocator.getMetaRegionState(this.watcher).getState());
- assertNull(MetaTableLocator.getMetaRegionLocation(this.watcher));
+ MetaTableLocator.getRootRegionState(this.watcher).getState());
+ assertNull(MetaTableLocator.getRootRegionLocation(this.watcher));
}
@Test(expected = NotAllMetaRegionsOnlineException.class)
@@ -149,7 +149,7 @@ public class TestMetaTableLocator {
*/
@Test
public void testNoTimeoutWaitForMeta() throws IOException, InterruptedException, KeeperException {
- ServerName hsa = MetaTableLocator.getMetaRegionLocation(watcher);
+ ServerName hsa = MetaTableLocator.getRootRegionLocation(watcher);
assertNull(hsa);
// Now test waiting on meta location getting set.
@@ -161,7 +161,7 @@ public class TestMetaTableLocator {
// Join the thread... should exit shortly.
t.join();
// Now meta is available.
- assertTrue(MetaTableLocator.getMetaRegionLocation(watcher).equals(hsa));
+ assertTrue(MetaTableLocator.getRootRegionLocation(watcher).equals(hsa));
}
private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
index 8e562bd..6d9bc9d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/RegionReplicaTestHelper.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.util.JVMClusterUtil;
+import org.apache.hadoop.hbase.util.test.LoadTestDataGenerator;
public final class RegionReplicaTestHelper {
@@ -102,7 +103,7 @@ public final class RegionReplicaTestHelper {
.map(t -> t.getRegionServer().getServerName()).filter(sn -> !sn.equals(serverName)).findAny()
.get();
util.getAdmin().move(regionInfo.getEncodedNameAsBytes(), newServerName);
- util.waitFor(30000, new ExplainingPredicate<Exception>() {
+ util.waitFor(30000, new ExplainingPredicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
similarity index 98%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
index 70f867a..38f48d2 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocator.java
@@ -57,11 +57,11 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncNonMetaRegionLocator {
+public class TestAsyncNonRootRegionLocator {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocator.class);
+ HBaseClassTestRule.forClass(TestAsyncNonRootRegionLocator.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -71,7 +71,7 @@ public class TestAsyncNonMetaRegionLocator {
private static AsyncConnectionImpl CONN;
- private static AsyncNonMetaRegionLocator LOCATOR;
+ private static AsyncNonRootRegionLocator LOCATOR;
private static byte[][] SPLIT_KEYS;
@@ -83,7 +83,7 @@ public class TestAsyncNonMetaRegionLocator {
ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
registry.getClusterId().get(), User.getCurrent());
- LOCATOR = new AsyncNonMetaRegionLocator(CONN);
+ LOCATOR = new AsyncNonRootRegionLocator(CONN);
SPLIT_KEYS = new byte[8][];
for (int i = 111; i < 999; i += 111) {
SPLIT_KEYS[i / 111 - 1] = Bytes.toBytes(String.format("%03d", i));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
similarity index 95%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
index 52242f1..e3e2184 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonMetaRegionLocatorConcurrenyLimit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncNonRootRegionLocatorConcurrenyLimit.java
@@ -18,7 +18,7 @@
package org.apache.hadoop.hbase.client;
import static java.util.stream.Collectors.toList;
-import static org.apache.hadoop.hbase.client.AsyncNonMetaRegionLocator.MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE;
+import static org.apache.hadoop.hbase.client.AsyncNonRootRegionLocator.MAX_CONCURRENT_LOCATE_REQUEST_PER_TABLE;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStartRow;
import static org.apache.hadoop.hbase.client.ConnectionUtils.isEmptyStopRow;
import static org.apache.hadoop.hbase.coprocessor.CoprocessorHost.REGION_COPROCESSOR_CONF_KEY;
@@ -56,11 +56,11 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
+public class TestAsyncNonRootRegionLocatorConcurrenyLimit {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncNonMetaRegionLocatorConcurrenyLimit.class);
+ HBaseClassTestRule.forClass(TestAsyncNonRootRegionLocatorConcurrenyLimit.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
@@ -70,7 +70,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
private static AsyncConnectionImpl CONN;
- private static AsyncNonMetaRegionLocator LOCATOR;
+ private static AsyncNonRootRegionLocator LOCATOR;
private static byte[][] SPLIT_KEYS;
@@ -127,7 +127,7 @@ public class TestAsyncNonMetaRegionLocatorConcurrenyLimit {
ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
CONN = new AsyncConnectionImpl(TEST_UTIL.getConfiguration(), registry,
registry.getClusterId().get(), User.getCurrent());
- LOCATOR = new AsyncNonMetaRegionLocator(CONN);
+ LOCATOR = new AsyncNonRootRegionLocator(CONN);
SPLIT_KEYS = IntStream.range(1, 256).mapToObj(i -> Bytes.toBytes(String.format("%02x", i)))
.toArray(byte[][]::new);
TEST_UTIL.createTable(TABLE_NAME, FAMILY, SPLIT_KEYS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
similarity index 91%
rename from hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
rename to hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
index 003bef3..2e7372c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncMetaRegionLocator.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRootRegionLocator.java
@@ -36,17 +36,17 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
@Category({ MediumTests.class, ClientTests.class })
-public class TestAsyncMetaRegionLocator {
+public class TestAsyncRootRegionLocator {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestAsyncMetaRegionLocator.class);
+ HBaseClassTestRule.forClass(TestAsyncRootRegionLocator.class);
private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
private static ConnectionRegistry REGISTRY;
- private static AsyncMetaRegionLocator LOCATOR;
+ private static AsyncRootRegionLocator LOCATOR;
@BeforeClass
public static void setUp() throws Exception {
@@ -56,7 +56,7 @@ public class TestAsyncMetaRegionLocator {
REGISTRY = ConnectionRegistryFactory.getRegistry(TEST_UTIL.getConfiguration());
RegionReplicaTestHelper.waitUntilAllMetaReplicasAreReady(TEST_UTIL, REGISTRY);
TEST_UTIL.getAdmin().balancerSwitch(false, true);
- LOCATOR = new AsyncMetaRegionLocator(REGISTRY);
+ LOCATOR = new AsyncRootRegionLocator(REGISTRY);
}
@AfterClass
@@ -67,7 +67,7 @@ public class TestAsyncMetaRegionLocator {
@Test
public void test() throws Exception {
- testLocator(TEST_UTIL, TableName.META_TABLE_NAME, new Locator() {
+ testLocator(TEST_UTIL, TableName.ROOT_TABLE_NAME, new Locator() {
@Override
public void updateCachedLocationOnError(HRegionLocation loc, Throwable error)
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
index 6c6bb98..e4672073 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocatePrefetch.java
@@ -49,16 +49,16 @@ public class TestAsyncTableLocatePrefetch {
private static AsyncConnection CONN;
- private static AsyncNonMetaRegionLocator LOCATOR;
+ private static AsyncNonRootRegionLocator LOCATOR;
@BeforeClass
public static void setUp() throws Exception {
- TEST_UTIL.getConfiguration().setInt(AsyncNonMetaRegionLocator.LOCATE_PREFETCH_LIMIT, 100);
+ TEST_UTIL.getConfiguration().setInt(AsyncNonRootRegionLocator.LOCATE_PREFETCH_LIMIT, 100);
TEST_UTIL.startMiniCluster(3);
TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAMILY);
TEST_UTIL.waitTableAvailable(TABLE_NAME);
CONN = ConnectionFactory.createAsyncConnection(TEST_UTIL.getConfiguration()).get();
- LOCATOR = new AsyncNonMetaRegionLocator((AsyncConnectionImpl) CONN);
+ LOCATOR = new AsyncNonRootRegionLocator((AsyncConnectionImpl) CONN);
}
@AfterClass
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocateRegionForDeletedTable.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocateRegionForDeletedTable.java
index 6ccd9bc..210afca 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocateRegionForDeletedTable.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableLocateRegionForDeletedTable.java
@@ -35,7 +35,7 @@ import org.junit.Test;
import org.junit.experimental.categories.Category;
/**
- * Fix an infinite loop in {@link AsyncNonMetaRegionLocator}, see the comments on HBASE-21943 for
+ * Fix an infinite loop in {@link AsyncNonRootRegionLocator}, see the comments on HBASE-21943 for
* more details.
*/
@Category({ MediumTests.class, ClientTests.class })
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
index 461bf1b..53ea788 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableRSCrashPublish.java
@@ -84,7 +84,7 @@ public class TestAsyncTableRSCrashPublish {
public void test() throws IOException, ExecutionException, InterruptedException {
Configuration conf = UTIL.getHBaseCluster().getMaster().getConfiguration();
try (AsyncConnection connection = ConnectionFactory.createAsyncConnection(conf).get()) {
- AsyncNonMetaRegionLocator locator =
+ AsyncNonRootRegionLocator locator =
((AsyncConnectionImpl) connection).getLocator().getNonMetaRegionLocator();
connection.getTable(TABLE_NAME).get(new Get(Bytes.toBytes(0))).join();
ServerName serverName =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
index abaf092..c8cf45f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaRegionLocationCache.java
@@ -75,7 +75,7 @@ public class TestMetaRegionLocationCache {
for (String znode: zk.getMetaReplicaNodes()) {
String path = ZNodePaths.joinZNode(zk.getZNodePaths().baseZNode, znode);
int replicaId = zk.getZNodePaths().getMetaReplicaIdFromPath(path);
- RegionState state = MetaTableLocator.getMetaRegionState(zk, replicaId);
+ RegionState state = MetaTableLocator.getRootRegionState(zk, replicaId);
result.add(new HRegionLocation(state.getRegion(), state.getServerName()));
}
return result;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaWithReplicas.java
new file mode 100644
index 0000000..e69de29
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
index 4016e7b..858e0dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/http/TestInfoServersACL.java
@@ -141,7 +141,7 @@ public class TestInfoServersACL {
CLUSTER = new LocalHBaseCluster(conf, 1);
CLUSTER.startup();
- CLUSTER.getActiveMaster().waitForMetaOnline();
+ CLUSTER.getActiveMaster().getAssignmentManager().waitForMetaOnline();
}
/**
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
index f78d490..5ddc018 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDeadServer.java
@@ -108,7 +108,7 @@ public class TestDeadServer {
HMaster master = TEST_UTIL.getHBaseCluster().getMaster();
final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();
ServerCrashProcedure proc = new ServerCrashProcedure(
- pExecutor.getEnvironment(), hostname123, false, false);
+ pExecutor.getEnvironment(), hostname123, false, false, false);
ProcedureTestingUtility.submitAndWait(pExecutor, proc);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
index a27936d..676e125 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterFailover.java
@@ -187,7 +187,7 @@ public class TestMasterFailover {
LOG.info("Master has aborted");
// meta should remain where it was
- RegionState metaState = MetaTableLocator.getMetaRegionState(hrs.getZooKeeper());
+ RegionState metaState = MetaTableLocator.getRootRegionState(hrs.getZooKeeper());
assertEquals("hbase:meta should be online on RS",
metaState.getServerName(), metaServerName);
assertEquals("hbase:meta should be online on RS", State.OPEN, metaState.getState());
@@ -200,7 +200,7 @@ public class TestMasterFailover {
LOG.info("Master is ready");
// ensure meta is still deployed on RS
- metaState = MetaTableLocator.getMetaRegionState(activeMaster.getZooKeeper());
+ metaState = MetaTableLocator.getRootRegionState(activeMaster.getZooKeeper());
assertEquals("hbase:meta should be online on RS",
metaState.getServerName(), metaServerName);
assertEquals("hbase:meta should be online on RS", State.OPEN, metaState.getState());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
index 9b5c591..f6bf68e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterNoCluster.java
@@ -275,10 +275,10 @@ public class TestMasterNoCluster {
HMaster master = new HMaster(conf) {
@Override
- protected MasterMetaBootstrap createMetaBootstrap() {
- return new MasterMetaBootstrap(this) {
+ protected MasterCatalogBootstrap createCatalogBootstrap() {
+ return new MasterCatalogBootstrap(this) {
@Override
- protected void assignMetaReplicas()
+ protected void assignCatalogReplicas()
throws IOException, InterruptedException, KeeperException {
// Nothing to do.
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
index ea532da..43c4de4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaShutdownHandler.java
@@ -98,7 +98,7 @@ public class TestMetaShutdownHandler {
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
metaServerName = regionStates.getRegionServerOfRegion(HRegionInfo.FIRST_META_REGIONINFO);
}
- RegionState metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
+ RegionState metaState = MetaTableLocator.getRootRegionState(master.getZooKeeper());
assertEquals("Wrong state for meta!", RegionState.State.OPEN, metaState.getState());
assertNotEquals("Meta is on master!", metaServerName, master.getServerName());
@@ -125,7 +125,7 @@ public class TestMetaShutdownHandler {
assertTrue("Meta should be assigned",
regionStates.isRegionOnline(HRegionInfo.FIRST_META_REGIONINFO));
// Now, make sure meta is registered in zk
- metaState = MetaTableLocator.getMetaRegionState(master.getZooKeeper());
+ metaState = MetaTableLocator.getRootRegionState(master.getZooKeeper());
assertEquals("Meta should not be in transition", RegionState.State.OPEN,
metaState.getState());
assertEquals("Meta should be assigned", metaState.getServerName(),
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
index 6db21fe..5244c35 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestRegionsRecoveryChore.java
@@ -592,6 +592,11 @@ public class TestRegionsRecoveryChore {
}
@Override
+ public boolean isRootRegion() {
+ return false;
+ }
+
+ @Override
public boolean isMetaRegion() {
return false;
}
@@ -606,6 +611,10 @@ public class TestRegionsRecoveryChore {
return false;
}
+ @Override public boolean containsRow(byte[] row, int offset, short length) {
+ return false;
+ }
+
};
return regionInfo;
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
index 7edb011..b927ddb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestSplitWALManager.java
@@ -158,7 +158,7 @@ public class TestSplitWALManager {
Assert.assertFalse(TEST_UTIL.getTestFileSystem().exists(wals[0].getPath()));
// Test splitting wal
- wals = TEST_UTIL.getTestFileSystem().listStatus(metaWALDir, MasterWalManager.NON_META_FILTER);
+ wals = TEST_UTIL.getTestFileSystem().listStatus(metaWALDir, MasterWalManager.NON_CATALOG_FILTER);
Assert.assertEquals(1, wals.length);
testProcedures =
splitWALManager.createSplitWALProcedures(Lists.newArrayList(wals[0]), metaServer);
@@ -281,6 +281,10 @@ public class TestSplitWALManager {
return serverName;
}
+ @Override public boolean hasRootTableRegion() {
+ return false;
+ }
+
@Override
public boolean hasMetaTableRegion() {
return false;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
index 0f4e97f..adcb73c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManager.java
@@ -224,7 +224,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
* back any response, which cause master startup hangs forever
*/
@Test
- public void testAssignMetaAndCrashBeforeResponse() throws Exception {
+ public void testAssignRootAndCrashBeforeResponse() throws Exception {
tearDown();
// See setUp(), start HBase until set up meta
util = new HBaseTestingUtility();
@@ -237,8 +237,8 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
// Assign meta
rsDispatcher.setMockRsExecutor(new HangThenRSRestartExecutor());
- am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
- assertEquals(true, am.isMetaAssigned());
+ am.assign(RegionInfoBuilder.ROOT_REGIONINFO);
+ assertEquals(true, am.isRootAssigned());
// set it back as default, see setUpMeta()
am.wakeMetaLoadedEvent();
@@ -302,7 +302,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
MetaTableAccessor.addRegionToMeta(this.util.getConnection(), hri);
assertNull("RegionInfo was manually added in META, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
- hri = am.loadRegionFromMeta(hri.getEncodedName());
+ hri = am.loadRegionFromCatalog(hri.getEncodedName());
assertEquals(hri.getEncodedName(),
am.getRegionStates().getRegionState(hri).getRegion().getEncodedName());
}finally {
@@ -321,7 +321,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
assertNull("RegionInfo was just instantiated by the test, but "
+ "shouldn't be in AM regionStates yet.", am.getRegionStates().getRegionState(hri));
assertNull("RegionInfo was never added in META, should had returned null.",
- am.loadRegionFromMeta(hri.getEncodedName()));
+ am.loadRegionFromCatalog(hri.getEncodedName()));
}finally {
this.util.killMiniHBaseCluster();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
index 2451f41..61989aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentManagerBase.java
@@ -172,6 +172,8 @@ public abstract class TestAssignmentManagerBase {
protected void setUpMeta() throws Exception {
rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
+ am.assign(RegionInfoBuilder.ROOT_REGIONINFO);
+ am.wakeRootLoadedEvent();
am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
am.wakeMetaLoadedEvent();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
index 75f73e5..aa6eb59 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestCloseRegionWhileRSCrash.java
@@ -88,6 +88,10 @@ public class TestCloseRegionWhileRSCrash {
return serverName;
}
+ @Override public boolean hasRootTableRegion() {
+ return false;
+ }
+
@Override
public boolean hasMetaTableRegion() {
return false;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
index fea362f..fbc64a5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestRegionStateStore.java
@@ -77,7 +77,7 @@ public class TestRegionStateStore {
final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
getAssignmentManager().getRegionStateStore();
final AtomicBoolean visitorCalled = new AtomicBoolean(false);
- regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+ regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum) {
@@ -108,7 +108,7 @@ public class TestRegionStateStore {
}
final AtomicBoolean visitorCalled = new AtomicBoolean(false);
- regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+ regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, RegionInfo regionInfo,
RegionState.State state, ServerName regionLocation,
@@ -127,7 +127,7 @@ public class TestRegionStateStore {
final RegionStateStore regionStateStore = UTIL.getHBaseCluster().getMaster().
getAssignmentManager().getRegionStateStore();
final AtomicBoolean visitorCalled = new AtomicBoolean(false);
- regionStateStore.visitMetaForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
+ regionStateStore.visitCatalogForRegion(encodedName, new RegionStateStore.RegionStateVisitor() {
@Override
public void visitRegionState(Result result, RegionInfo regionInfo, RegionState.State state,
ServerName regionLocation, ServerName lastHost, long openSeqNum) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
index 1845771..222db70 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestSCP.java
@@ -71,7 +71,7 @@ public class TestSCP extends TestSCPBase {
HMaster master = util.getHBaseCluster().getMaster();
final ProcedureExecutor<MasterProcedureEnv> pExecutor = master.getMasterProcedureExecutor();
ServerCrashProcedure procB =
- new ServerCrashProcedure(pExecutor.getEnvironment(), rsToKill, false, false);
+ new ServerCrashProcedure(pExecutor.getEnvironment(), rsToKill, false, false, false);
AssignmentTestingUtil.killRs(util, rsToKill);
long procId = getSCPProcId(pExecutor);
Procedure<?> procA = pExecutor.getProcedure(procId);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
index 14342d4..98f4739 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/procedure/TestServerRemoteProcedure.java
@@ -218,6 +218,10 @@ public class TestServerRemoteProcedure {
return targetServer;
}
+ @Override public boolean hasRootTableRegion() {
+ return false;
+ }
+
@Override
public boolean hasMetaTableRegion() {
return false;
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
index 69e656f..e7b5525 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/protobuf/TestProtobufUtil.java
@@ -369,14 +369,14 @@ public class TestProtobufUtil {
ProtobufUtil.prependPBMagic(data);
// Deserialize
RegionState regionStateNew =
- org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(data, 1);
+ org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseRootRegionStateFrom(data, 1);
assertEquals(regionState.getServerName(), regionStateNew.getServerName());
assertEquals(regionState.getState(), regionStateNew.getState());
}
// old style.
RegionState rs =
- org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseMetaRegionStateFrom(
- serverName.getVersionedBytes(), 1);
+ org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.parseRootRegionStateFrom((
+ serverName.getVersionedBytes()), 1);
assertEquals(serverName, rs.getServerName());
assertEquals(rs.getState(), RegionState.State.OPEN);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
index 3e00540..f41d1b6 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionInfo.java
@@ -31,7 +31,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
@@ -170,7 +169,7 @@ public class TestHRegionInfo {
Path basedir = htu.getDataTestDir();
// Create a region. That'll write the .regioninfo file.
FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
- FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration());
+ FSTableDescriptors.tryUpdateCatalogTableDescriptor(htu.getConfiguration());
HRegion r = HBaseTestingUtility.createRegionAndWAL(hri, basedir, htu.getConfiguration(),
fsTableDescriptors.get(TableName.META_TABLE_NAME));
// Get modtime on the file.
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
index 77e4079..5a1a4fc 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerNoMaster.java
@@ -111,7 +111,7 @@ public class TestRegionServerNoMaster {
HRegionServer hrs = HTU.getHBaseCluster()
.getLiveRegionServerThreads().get(0).getRegionServer();
ZKWatcher zkw = hrs.getZooKeeper();
- ServerName sn = MetaTableLocator.getMetaRegionLocation(zkw);
+ ServerName sn = MetaTableLocator.getRootRegionLocation(zkw);
if (sn != null && !masterAddr.equals(sn)) {
return;
}
@@ -119,7 +119,7 @@ public class TestRegionServerNoMaster {
ProtobufUtil.openRegion(null, hrs.getRSRpcServices(),
hrs.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
while (true) {
- sn = MetaTableLocator.getMetaRegionLocation(zkw);
+ sn = MetaTableLocator.getRootRegionLocation(zkw);
if (sn != null && sn.equals(hrs.getServerName())
&& hrs.getOnlineRegions().containsKey(
HRegionInfo.FIRST_META_REGIONINFO.getEncodedName())) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
index a8b6490..440d6aa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
@@ -92,7 +92,7 @@ public class TestLogRollingNoCluster {
conf.set(WALFactory.WAL_PROVIDER, "filesystem");
CommonFSUtils.setRootDir(conf, dir);
FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(TEST_UTIL.getConfiguration());
- FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
+ FSTableDescriptors.tryUpdateCatalogTableDescriptor(TEST_UTIL.getConfiguration());
TableDescriptor metaTableDescriptor = fsTableDescriptors.get(TableName.META_TABLE_NAME);
conf.set("hbase.regionserver.hlog.writer.impl", HighLatencySyncWriter.class.getName());
final WALFactory wals = new WALFactory(conf, TestLogRollingNoCluster.class.getName());
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
index c7b45fe..dbb6bef 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/MetaTableLocator.java
@@ -73,10 +73,10 @@ public final class MetaTableLocator {
*/
public static List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
int replicaId) {
- ServerName serverName = getMetaRegionLocation(zkw, replicaId);
+ ServerName serverName = getRootRegionLocation(zkw, replicaId);
List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
+ RegionInfoBuilder.ROOT_REGIONINFO, replicaId), serverName));
return list;
}
@@ -120,9 +120,9 @@ public final class MetaTableLocator {
* @param zkw zookeeper connection to use
* @return server name or null if we failed to get the data.
*/
- public static ServerName getMetaRegionLocation(final ZKWatcher zkw) {
+ public static ServerName getRootRegionLocation(final ZKWatcher zkw) {
try {
- RegionState state = getMetaRegionState(zkw);
+ RegionState state = getRootRegionState(zkw);
return state.isOpened() ? state.getServerName() : null;
} catch (KeeperException ke) {
return null;
@@ -135,9 +135,9 @@ public final class MetaTableLocator {
* @param replicaId the ID of the replica
* @return server name
*/
- public static ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
+ public static ServerName getRootRegionLocation(final ZKWatcher zkw, int replicaId) {
try {
- RegionState state = getMetaRegionState(zkw, replicaId);
+ RegionState state = getRootRegionState(zkw, replicaId);
return state.isOpened() ? state.getServerName() : null;
} catch (KeeperException ke) {
return null;
@@ -248,8 +248,8 @@ public final class MetaTableLocator {
/**
* Load the meta region state from the meta server ZNode.
*/
- public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException {
- return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
+ public static RegionState getRootRegionState(ZKWatcher zkw) throws KeeperException {
+ return getRootRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
@@ -260,12 +260,12 @@ public final class MetaTableLocator {
* @return regionstate
* @throws KeeperException if a ZooKeeper operation fails
*/
- public static RegionState getMetaRegionState(ZKWatcher zkw, int replicaId)
+ public static RegionState getRootRegionState(ZKWatcher zkw, int replicaId)
throws KeeperException {
RegionState regionState = null;
try {
byte[] data = ZKUtil.getData(zkw, zkw.getZNodePaths().getZNodeForReplica(replicaId));
- regionState = ProtobufUtil.parseMetaRegionStateFrom(data, replicaId);
+ regionState = ProtobufUtil.parseRootRegionStateFrom(data, replicaId);
} catch (DeserializationException e) {
throw ZKUtil.convert(e);
} catch (InterruptedException e) {
@@ -331,7 +331,7 @@ public final class MetaTableLocator {
}
for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
// return all replica locations for the meta
- servers.add(getMetaRegionLocation(zkw, replicaId));
+ servers.add(getRootRegionLocation(zkw, replicaId));
}
return servers;
}
@@ -369,7 +369,7 @@ public final class MetaTableLocator {
long startTime = System.currentTimeMillis();
ServerName sn = null;
while (true) {
- sn = getMetaRegionLocation(zkw, replicaId);
+ sn = getRootRegionLocation(zkw, replicaId);
if (sn != null ||
(System.currentTimeMillis() - startTime) > timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
break;
diff --git a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
index 19d11d0..f78b05a 100644
--- a/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
+++ b/hbase-zookeeper/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java
@@ -1860,13 +1860,13 @@ public final class ZKUtil {
}
}
sb.append("\nRegion server holding hbase:meta: "
- + MetaTableLocator.getMetaRegionLocation(zkw));
+ + MetaTableLocator.getRootRegionLocation(zkw));
Configuration conf = HBaseConfiguration.create();
int numMetaReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
HConstants.DEFAULT_META_REPLICA_NUM);
for (int i = 1; i < numMetaReplicas; i++) {
sb.append("\nRegion server holding hbase:meta, replicaId " + i + " "
- + MetaTableLocator.getMetaRegionLocation(zkw, i));
+ + MetaTableLocator.getRootRegionLocation(zkw, i));
}
sb.append("\nRegion servers:");
final List<String> rsChildrenNoWatchList =