You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2020/07/01 08:14:36 UTC
[hbase] branch HBASE-11288.splittable-meta updated: HBASE-24390
Remove RegionInfoBuilder.FIRST_META_REGIONINFO (#1877)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch HBASE-11288.splittable-meta
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/HBASE-11288.splittable-meta by this push:
new c034cc7 HBASE-24390 Remove RegionInfoBuilder.FIRST_META_REGIONINFO (#1877)
c034cc7 is described below
commit c034cc7c9e885baff229f8cc99b718c5b948a140
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Wed Jul 1 16:14:21 2020 +0800
HBASE-24390 Remove RegionInfoBuilder.FIRST_META_REGIONINFO (#1877)
Signed-off-by: stack <st...@apache.org>
---
.../hadoop/hbase/client/RawAsyncHBaseAdmin.java | 7 -
.../hadoop/hbase/client/RegionInfoBuilder.java | 23 +--
.../hadoop/hbase/client/ZKConnectionRegistry.java | 11 +-
.../hadoop/hbase/shaded/protobuf/ProtobufUtil.java | 8 +-
.../hadoop/hbase/client/TestRegionInfoBuilder.java | 6 +-
.../hadoop/hbase/DistributedHBaseCluster.java | 13 ++
.../org/apache/hadoop/hbase/master/HMaster.java | 29 ++--
.../hadoop/hbase/master/MasterStatusServlet.java | 15 +-
.../hbase/master/assignment/AssignmentManager.java | 120 ++--------------
.../assignment/RegionRemoteProcedureBase.java | 9 +-
.../hbase/master/assignment/ServerStateNode.java | 7 +
.../assignment/TransitRegionStateProcedure.java | 10 +-
.../master/procedure/HBCKServerCrashProcedure.java | 2 +-
.../hbase/master/procedure/InitMetaProcedure.java | 15 +-
.../master/procedure/MasterProcedureUtil.java | 11 ++
.../master/procedure/ServerCrashProcedure.java | 8 +-
.../hadoop/hbase/regionserver/HRegionServer.java | 5 +-
.../java/org/apache/hadoop/hbase/util/FSUtils.java | 11 +-
.../org/apache/hadoop/hbase/util/HBaseFsck.java | 17 +--
.../main/resources/hbase-webapps/master/table.jsp | 8 +-
.../java/org/apache/hadoop/hbase/HBaseCluster.java | 6 +-
.../org/apache/hadoop/hbase/MiniHBaseCluster.java | 36 ++++-
.../hadoop/hbase/TestClientClusterMetrics.java | 13 +-
.../org/apache/hadoop/hbase/TestHBaseMetaEdit.java | 10 +-
.../apache/hadoop/hbase/TestHRegionLocation.java | 24 ++--
.../hbase/client/MetaWithReplicasTestBase.java | 12 +-
.../org/apache/hadoop/hbase/client/TestAdmin2.java | 2 +-
.../client/TestAsyncTableGetMultiThreaded.java | 3 +-
.../client/TestFailedMetaReplicaAssigment.java | 8 +-
.../client/TestMetaReplicasAddressChange.java | 4 +-
.../client/TestMetaTableAccessorNoCluster.java | 6 +-
.../hbase/client/TestSeparateClientZKCluster.java | 6 +-
.../TestRegionObserverPreFlushAndPreCompact.java | 3 +-
.../hadoop/hbase/master/AbstractTestDLS.java | 10 +-
.../hadoop/hbase/master/MockRegionServer.java | 7 +-
.../org/apache/hadoop/hbase/master/TestMaster.java | 3 +-
.../hadoop/hbase/master/TestMasterQosFunction.java | 2 +-
.../hbase/master/TestMetaFixerNoCluster.java | 2 +-
.../hbase/master/TestMetaShutdownHandler.java | 38 +++--
.../master/assignment/AssignmentTestingUtil.java | 15 +-
.../master/assignment/TestAssignmentManager.java | 6 +-
.../assignment/TestAssignmentManagerBase.java | 11 +-
.../master/assignment/TestAssignmentOnRSCrash.java | 4 -
.../hbase/master/assignment/TestHbckChore.java | 5 +-
.../master/balancer/TestBaseLoadBalancer.java | 5 -
.../hbase/regionserver/TestCleanupMetaWAL.java | 6 +-
.../hbase/regionserver/TestDefaultMemStore.java | 6 +-
.../hbase/regionserver/TestDefaultStoreEngine.java | 9 +-
.../regionserver/TestGetClosestAtOrBefore.java | 4 +-
.../hadoop/hbase/regionserver/TestPriorityRpc.java | 12 +-
.../TestReadAndWriteRegionInfoFile.java | 2 +-
.../hadoop/hbase/regionserver/TestRegionInfo.java | 158 +++++++++------------
.../hbase/regionserver/TestStripeStoreEngine.java | 4 +-
.../compactions/TestStripeCompactionPolicy.java | 7 +-
.../regionserver/wal/TestLogRollingNoCluster.java | 9 +-
.../TestReplicationWALEntryFilters.java | 6 +-
.../org/apache/hadoop/hbase/util/TestFSUtils.java | 16 +--
.../org/apache/hadoop/hbase/wal/TestWALSplit.java | 14 +-
58 files changed, 366 insertions(+), 463 deletions(-)
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 cbdc462..e85b924 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
@@ -2402,13 +2402,6 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
return failedFuture(new IllegalArgumentException("Passed region name can't be null"));
}
- if (Bytes.equals(regionNameOrEncodedRegionName,
- RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName()) ||
- Bytes.equals(regionNameOrEncodedRegionName,
- RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes())) {
- return CompletableFuture.completedFuture(RegionInfoBuilder.FIRST_META_REGIONINFO);
- }
-
CompletableFuture<RegionInfo> future = new CompletableFuture<>();
addListener(getRegionLocation(regionNameOrEncodedRegionName), (location, err) -> {
if (err != null) {
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..aad3ef5 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
@@ -37,18 +37,6 @@ public class RegionInfoBuilder {
//TODO: Move NO_HASH to HStoreFile which is really the only place it is used.
public static final String NO_HASH = null;
- /**
- * 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
- * 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).
- */
- // TODO: How come Meta regions still do not have encoded region names? Fix.
- // hbase:meta,,1.1588230740 should be the hbase:meta first region name.
- public static final RegionInfo FIRST_META_REGIONINFO =
- new MutableRegionInfo(1L, TableName.META_TABLE_NAME, RegionInfo.DEFAULT_REPLICA_ID);
-
private final TableName tableName;
private byte[] startKey = HConstants.EMPTY_START_ROW;
private byte[] endKey = HConstants.EMPTY_END_ROW;
@@ -194,15 +182,6 @@ public class RegionInfoBuilder {
return regionId;
}
- /**
- * Private constructor used constructing MutableRegionInfo for the
- * first meta regions
- */
- private MutableRegionInfo(long regionId, TableName tableName, int replicaId) {
- this(tableName, HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false, regionId,
- replicaId, false);
- }
-
MutableRegionInfo(final TableName tableName, final byte[] startKey, final byte[] endKey,
final boolean split, final long regionId, final int replicaId, boolean offLine) {
this.tableName = checkTableName(tableName);
@@ -320,7 +299,7 @@ public class RegionInfoBuilder {
/** @return true if this region is a meta region */
@Override
public boolean isMetaRegion() {
- return tableName.equals(FIRST_META_REGIONINFO.getTable());
+ return TableName.isMetaTableName(tableName);
}
/**
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 04ea535..e8acbda 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,9 +18,6 @@
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.RegionReplicaUtil.getRegionInfoForDefaultReplica;
-import static org.apache.hadoop.hbase.client.RegionReplicaUtil.getRegionInfoForReplica;
import static org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil.lengthOfPBMagic;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import static org.apache.hadoop.hbase.zookeeper.ZKMetadata.removeMetaData;
@@ -35,6 +32,7 @@ import org.apache.hadoop.hbase.ClusterId;
import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.Pair;
@@ -161,7 +159,8 @@ 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());
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setRegionId(1).build(),
+ stateAndServerName.getSecond());
tryComplete(remaining, locs, future);
});
} else {
@@ -183,8 +182,8 @@ class ZKConnectionRegistry implements ConnectionRegistry {
locs[replicaId] = null;
} else {
locs[replicaId] =
- new HRegionLocation(getRegionInfoForReplica(FIRST_META_REGIONINFO, replicaId),
- stateAndServerName.getSecond());
+ new HRegionLocation(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME)
+ .setRegionId(1).setReplicaId(replicaId).build(), stateAndServerName.getSecond());
}
}
tryComplete(remaining, locs, future);
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 1f6e057..b4b657c 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
@@ -82,7 +82,6 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionLoadStats;
import org.apache.hadoop.hbase.client.RegionLocateType;
-import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -3155,8 +3154,8 @@ public final class ProtobufUtil {
if (serverName == null) {
state = RegionState.State.OFFLINE;
}
- return new RegionState(RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), state, serverName);
+ return new RegionState(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setRegionId(1)
+ .setReplicaId(replicaId).build(), state, serverName);
}
/**
@@ -3272,9 +3271,6 @@ public final class ProtobufUtil {
long regionId = proto.getRegionId();
int defaultReplicaId = org.apache.hadoop.hbase.client.RegionInfo.DEFAULT_REPLICA_ID;
int replicaId = proto.hasReplicaId()? proto.getReplicaId(): defaultReplicaId;
- if (tableName.equals(TableName.META_TABLE_NAME) && replicaId == defaultReplicaId) {
- return RegionInfoBuilder.FIRST_META_REGIONINFO;
- }
byte[] startKey = null;
byte[] endKey = null;
if (proto.hasStartKey()) {
diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java
index 0d2b7cc..308df36 100644
--- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java
+++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestRegionInfoBuilder.java
@@ -82,10 +82,10 @@ public class TestRegionInfoBuilder {
@Test
public void testPb() throws DeserializationException {
- RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ RegionInfo ri = RegionInfoBuilder.newBuilder(name.getTableName()).build();
byte[] bytes = RegionInfo.toByteArray(ri);
RegionInfo pbri = RegionInfo.parseFrom(bytes);
- assertTrue(RegionInfo.COMPARATOR.compare(ri, pbri) == 0);
+ assertEquals(0, RegionInfo.COMPARATOR.compare(ri, pbri));
}
@Test
@@ -148,7 +148,7 @@ public class TestRegionInfoBuilder {
@Test
public void testMetaTables() {
- assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaRegion());
+ assertTrue(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build().isMetaRegion());
}
@Test
diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
index 796bc1f..72f8fe5 100644
--- a/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
+++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/DistributedHBaseCluster.java
@@ -301,6 +301,19 @@ public class DistributedHBaseCluster extends HBaseCluster {
}
@Override
+ public ServerName getServerHoldingMeta() throws IOException {
+ HRegionLocation regionLoc = null;
+ try (RegionLocator locator = connection.getRegionLocator(TableName.META_TABLE_NAME)) {
+ regionLoc = locator.getRegionLocation(HConstants.EMPTY_START_ROW, true);
+ }
+ if (regionLoc == null) {
+ LOG.warn("Cannot find region server holding first meta region");
+ return null;
+ }
+ return regionLoc.getServerName();
+ }
+
+ @Override
public ServerName getServerHoldingRegion(TableName tn, byte[] regionName) throws IOException {
byte[] startKey = RegionInfo.getStartKey(regionName);
HRegionLocation regionLoc = null;
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 2e59f0d..e142fd0 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
@@ -90,8 +90,8 @@ import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.MasterSwitchType;
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.RegionLocateType;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -921,13 +921,16 @@ public class HMaster extends HRegionServer implements MasterServices {
}
}
// start migrating
- byte[] row = CatalogFamilyFormat.getMetaKeyForRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
- Put put = new Put(row);
+ Put put = null;
List<String> metaReplicaNodes = zooKeeper.getMetaReplicaNodes();
StringBuilder info = new StringBuilder("Migrating meta location:");
for (String metaReplicaNode : metaReplicaNodes) {
int replicaId = zooKeeper.getZNodePaths().getMetaReplicaIdFromZnode(metaReplicaNode);
RegionState state = getMetaRegionState(zooKeeper, replicaId);
+ if (put == null) {
+ byte[] row = CatalogFamilyFormat.getMetaKeyForRegion(state.getRegion());
+ put = new Put(row);
+ }
info.append(" ").append(state);
put.setTimestamp(state.getStamp());
MetaTableAccessor.addRegionInfo(put, state.getRegion());
@@ -939,9 +942,10 @@ public class HMaster extends HRegionServer implements MasterServices {
.setQualifier(RegionStateStore.getStateColumn(replicaId)).setTimestamp(put.getTimestamp())
.setType(Cell.Type.Put).setValue(Bytes.toBytes(state.getState().name())).build());
}
- if (!put.isEmpty()) {
+ if (put != null) {
LOG.info(info.toString());
- masterRegion.update(r -> r.put(put));
+ final Put p = put;
+ masterRegion.update(r -> r.put(p));
} else {
LOG.info("No meta location avaiable on zookeeper, skip migrating...");
}
@@ -1123,11 +1127,7 @@ 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;
- // 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()) {
+ if (!this.assignmentManager.getRegionStates().hasTableRegionStates(TableName.META_TABLE_NAME)) {
Optional<InitMetaProcedure> optProc = procedureExecutor.getProcedures().stream()
.filter(p -> p instanceof InitMetaProcedure).map(o -> (InitMetaProcedure) o).findAny();
initMetaProc = optProc.orElseGet(() -> {
@@ -1306,11 +1306,14 @@ public class HMaster extends HRegionServer implements MasterServices {
/**
* 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.
+ * and we will hold here until operator intervention.
*/
@VisibleForTesting
- public boolean waitForMetaOnline() {
- return isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ public boolean waitForMetaOnline() throws InterruptedException {
+ Optional<RegionInfo> firstMetaRegion =
+ this.assignmentManager.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).stream()
+ .filter(RegionInfo::isFirst).filter(RegionReplicaUtil::isDefaultReplica).findFirst();
+ return firstMetaRegion.isPresent() ? isRegionOnline(firstMetaRegion.get()) : false;
}
/**
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 198208f..415f928 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
@@ -26,8 +26,11 @@ import javax.servlet.http.HttpServlet;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
+import org.apache.hadoop.hbase.RegionLocations;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionLocateType;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.yetus.audience.InterfaceAudience;
@@ -79,9 +82,13 @@ public class MasterStatusServlet extends HttpServlet {
tmpl.render(response.getWriter(), master);
}
- private ServerName getMetaLocationOrNull(HMaster master) {
- return master.getAssignmentManager().getRegionStates()
- .getRegionState(RegionInfoBuilder.FIRST_META_REGIONINFO).getServerName();
+ private ServerName getMetaLocationOrNull(HMaster master) throws IOException {
+ RegionLocations locs = master.locateMeta(HConstants.EMPTY_START_ROW, RegionLocateType.CURRENT);
+ if (locs == null) {
+ return null;
+ }
+ HRegionLocation loc = locs.getDefaultRegionLocation();
+ return loc != null ? loc.getServerName() : null;
}
private Map<String, Integer> getFragmentationInfo(
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 41bc4d0..53c305f 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
@@ -45,8 +45,6 @@ import org.apache.hadoop.hbase.UnknownRegionException;
import org.apache.hadoop.hbase.client.DoNotRetryRegionException;
import org.apache.hadoop.hbase.client.MasterSwitchType;
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.RegionStatesCount;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -153,7 +151,6 @@ public class AssignmentManager {
"hbase.metrics.rit.stuck.warning.threshold";
private static final int DEFAULT_RIT_STUCK_WARNING_THRESHOLD = 60 * 1000;
- private final ProcedureEvent<?> metaAssignEvent = new ProcedureEvent<>("meta assign");
private final ProcedureEvent<?> metaLoadEvent = new ProcedureEvent<>("meta load");
private final MetricsAssignmentManager metrics;
@@ -256,10 +253,6 @@ public class AssignmentManager {
if (regionLocation != null) {
regionStates.addRegionToServer(regionNode);
}
- if (RegionReplicaUtil.isDefaultReplica(regionInfo)) {
- setMetaAssigned(regionInfo, state == State.OPEN);
- }
-
if (regionInfo.isFirst()) {
// for compatibility, mirror the meta region state to zookeeper
try {
@@ -271,10 +264,6 @@ public class AssignmentManager {
}
}, result);
} while (moreRows);
- if (!regionStates.hasTableRegionStates(TableName.META_TABLE_NAME)) {
- // no meta regions yet, create the region node for the first meta region
- regionStates.createRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
- }
}
}
@@ -340,9 +329,6 @@ public class AssignmentManager {
// Update meta events (for testing)
if (hasProcExecutor) {
metaLoadEvent.suspend();
- for (RegionInfo hri: getMetaRegionSet()) {
- setMetaAssigned(hri, false);
- }
}
}
@@ -404,6 +390,14 @@ public class AssignmentManager {
return serverInfo.getRegionInfoList();
}
+ public List<RegionInfo> getDefaultMetaRegionsOnServer(ServerName serverName) {
+ ServerStateNode serverInfo = regionStates.getServerNode(serverName);
+ if (serverInfo == null) {
+ return Collections.emptyList();
+ }
+ return serverInfo.getDefaultMetaRegionInfoList();
+ }
+
public RegionStateStore getRegionStateStore() {
return regionStateStore;
}
@@ -433,89 +427,17 @@ public class AssignmentManager {
// ============================================================================================
// META Helpers
// ============================================================================================
- private boolean isMetaRegion(final RegionInfo regionInfo) {
- return regionInfo.isMetaRegion();
- }
-
- public boolean isMetaRegion(final byte[] regionName) {
- return getMetaRegionFromName(regionName) != null;
- }
-
- public RegionInfo getMetaRegionFromName(final byte[] regionName) {
- for (RegionInfo hri: getMetaRegionSet()) {
- if (Bytes.equals(hri.getRegionName(), regionName)) {
- return hri;
- }
- }
- return null;
- }
-
- public boolean isCarryingMeta(final ServerName serverName) {
- // TODO: handle multiple meta
- return isCarryingRegion(serverName, RegionInfoBuilder.FIRST_META_REGIONINFO);
- }
-
- private boolean isCarryingRegion(final ServerName serverName, final RegionInfo regionInfo) {
- // TODO: check for state?
- final RegionStateNode node = regionStates.getRegionStateNode(regionInfo);
- return(node != null && serverName.equals(node.getRegionLocation()));
- }
-
- private RegionInfo getMetaForRegion(final RegionInfo regionInfo) {
- //if (regionInfo.isMetaRegion()) return regionInfo;
- // TODO: handle multiple meta. if the region provided is not meta lookup
- // which meta the region belongs to.
- return RegionInfoBuilder.FIRST_META_REGIONINFO;
- }
-
- // TODO: handle multiple meta.
- private static final Set<RegionInfo> META_REGION_SET =
- Collections.singleton(RegionInfoBuilder.FIRST_META_REGIONINFO);
- public Set<RegionInfo> getMetaRegionSet() {
- return META_REGION_SET;
+ public boolean isCarryingMeta(ServerName serverName) {
+ return regionStates.getTableRegionStateNodes(TableName.META_TABLE_NAME).stream()
+ .map(RegionStateNode::getRegionLocation).anyMatch(serverName::equals);
}
// ============================================================================================
// META Event(s) helpers
// ============================================================================================
- /**
- * Notice that, this only means the meta region is available on a RS, but the AM may still be
- * loading the region states from meta, so usually you need to check {@link #isMetaLoaded()} 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 #isMetaLoaded()
- */
- public boolean isMetaAssigned() {
- return metaAssignEvent.isReady();
- }
-
public boolean isMetaRegionInTransition() {
- return !isMetaAssigned();
- }
-
- /**
- * Notice that this event does not mean the AM has already finished region state rebuilding. See
- * the comment of {@link #isMetaAssigned()} for more details.
- * @see #isMetaAssigned()
- */
- public boolean waitMetaAssigned(Procedure<?> proc, RegionInfo regionInfo) {
- return getMetaAssignEvent(getMetaForRegion(regionInfo)).suspendIfNotReady(proc);
- }
-
- private void setMetaAssigned(RegionInfo metaRegionInfo, boolean assigned) {
- assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
- ProcedureEvent<?> metaAssignEvent = getMetaAssignEvent(metaRegionInfo);
- if (assigned) {
- metaAssignEvent.wake(getProcedureScheduler());
- } else {
- metaAssignEvent.suspend();
- }
- }
-
- private ProcedureEvent<?> getMetaAssignEvent(RegionInfo metaRegionInfo) {
- assert isMetaRegion(metaRegionInfo) : "unexpected non-meta region " + metaRegionInfo;
- // TODO: handle multiple meta.
- return metaAssignEvent;
+ return regionStates.getRegionsInTransition().stream().map(RegionStateNode::getRegionInfo)
+ .anyMatch(RegionInfo::isMetaRegion);
}
/**
@@ -1576,7 +1498,7 @@ public class AssignmentManager {
if (!isRunning()) {
throw new PleaseHoldException("AssignmentManager not running");
}
- boolean meta = isMetaRegion(hri);
+ boolean meta = hri.isMetaRegion();
boolean metaLoaded = isMetaLoaded();
if (!meta && !metaLoaded) {
throw new PleaseHoldException(
@@ -1809,12 +1731,6 @@ public class AssignmentManager {
// should be called under the RegionStateNode lock
void regionClosing(RegionStateNode regionNode) throws IOException {
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)) {
- setMetaAssigned(hri, false);
- }
regionStates.addRegionToServer(regionNode);
// update the operation count metrics
metrics.incrementOperationCounter();
@@ -1870,14 +1786,6 @@ public class AssignmentManager {
void persistToMeta(RegionStateNode regionNode) throws IOException {
regionStateStore.updateRegionLocation(regionNode);
- RegionInfo regionInfo = regionNode.getRegionInfo();
- if (isMetaRegion(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.
- setMetaAssigned(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..bb71fe0 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
@@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.exceptions.UnexpectedStateException;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.TableProcedureInterface;
import org.apache.hadoop.hbase.procedure2.FailedRemoteDispatchException;
import org.apache.hadoop.hbase.procedure2.Procedure;
@@ -148,13 +149,7 @@ public abstract class RegionRemoteProcedureBase extends Procedure<MasterProcedur
@Override
protected boolean waitInitialized(MasterProcedureEnv env) {
- if (TableName.isMetaTableName(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);
+ return MasterProcedureUtil.waitInitialized(this, env, getTableName());
}
@Override
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
index 33f6b1a..76c7469 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/assignment/ServerStateNode.java
@@ -26,6 +26,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.yetus.audience.InterfaceAudience;
/**
@@ -73,6 +74,12 @@ public class ServerStateNode implements Comparable<ServerStateNode> {
return regions.stream().map(RegionStateNode::getRegionInfo).collect(Collectors.toList());
}
+ public List<RegionInfo> getDefaultMetaRegionInfoList() {
+ System.err.println("================" + regions);
+ return regions.stream().map(RegionStateNode::getRegionInfo).filter(RegionInfo::isMetaRegion)
+ .filter(RegionReplicaUtil::isDefaultReplica).collect(Collectors.toList());
+ }
+
public List<RegionInfo> getSystemRegionInfoList() {
return regions.stream().filter(RegionStateNode::isSystemTable)
.map(RegionStateNode::getRegionInfo).collect(Collectors.toList());
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..fed00c0 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
@@ -21,7 +21,6 @@ import edu.umd.cs.findbugs.annotations.Nullable;
import java.io.IOException;
import org.apache.hadoop.hbase.HBaseIOException;
import org.apache.hadoop.hbase.ServerName;
-import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.client.RetriesExhaustedException;
@@ -29,6 +28,7 @@ import org.apache.hadoop.hbase.master.MetricsAssignmentManager;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.procedure.AbstractStateMachineRegionProcedure;
import org.apache.hadoop.hbase.master.procedure.MasterProcedureEnv;
+import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil;
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
@@ -166,13 +166,7 @@ public class TransitRegionStateProcedure
@Override
protected boolean waitInitialized(MasterProcedureEnv env) {
- if (TableName.isMetaTableName(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());
+ return MasterProcedureUtil.waitInitialized(this, env, getTableName());
}
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 725a138..28e1901 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
@@ -84,7 +84,7 @@ public class HBCKServerCrashProcedure extends ServerCrashProcedure {
@Override
@edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NP_NULL_ON_SOME_PATH_EXCEPTION",
justification="FindBugs seems confused on ps in below.")
- List<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env) {
+ protected List<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env) {
// Super will return an immutable list (empty if nothing on this server).
List<RegionInfo> ris = super.getRegionsOnCrashedServer(env);
if (!ris.isEmpty()) {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
index cdf78cc..29aee81 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/InitMetaProcedure.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HConstants;
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.client.TableDescriptor;
import org.apache.hadoop.hbase.master.assignment.TransitRegionStateProcedure;
@@ -58,6 +59,14 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
private static final Logger LOG = LoggerFactory.getLogger(InitMetaProcedure.class);
+ /**
+ * Used to create meta table when bootstraping a new hbase cluster.
+ * <p/>
+ * Setting region id to 1 is for keeping compatible with old clients.
+ */
+ private static final RegionInfo BOOTSTRAP_META_REGIONINFO =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setRegionId(1).build();
+
private CountDownLatch latch = new CountDownLatch(1);
private RetryCounter retryCounter;
@@ -87,9 +96,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
builder -> builder.setRegionReplication(
conf.getInt(HConstants.META_REPLICAS_NUM, HConstants.DEFAULT_META_REPLICA_NUM)));
TableDescriptor metaDescriptor = new FSTableDescriptors(conf).get(TableName.META_TABLE_NAME);
- HRegion
- .createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, rootDir, conf, metaDescriptor, null)
- .close();
+ HRegion.createHRegion(BOOTSTRAP_META_REGIONINFO, rootDir, conf, metaDescriptor, null).close();
}
@Override
@@ -107,7 +114,7 @@ public class InitMetaProcedure extends AbstractStateMachineTableProcedure<InitMe
case INIT_META_ASSIGN_META:
LOG.info("Going to assign meta");
addChildProcedure(env.getAssignmentManager()
- .createAssignProcedures(Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+ .createAssignProcedures(Arrays.asList(BOOTSTRAP_META_REGIONINFO)));
setNextState(InitMetaState.INIT_META_CREATE_NAMESPACES);
return Flow.HAS_MORE_STATE;
case INIT_META_CREATE_NAMESPACES:
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 c6e77fd..0766b5c 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
@@ -26,6 +26,7 @@ import org.apache.hadoop.hbase.NamespaceDescriptor;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.constraint.ConstraintException;
import org.apache.hadoop.hbase.master.MasterServices;
+import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureException;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
@@ -241,4 +242,14 @@ public final class MasterProcedureUtil {
return Optional
.ofNullable(namespaceDesc.getConfigurationValue(RSGroupInfo.NAMESPACE_DESC_PROP_GROUP));
}
+
+ public static boolean waitInitialized(Procedure<MasterProcedureEnv> proc, MasterProcedureEnv env,
+ TableName tableName) {
+ if (TableName.isMetaTableName(tableName)) {
+ return false;
+ }
+ // we need meta to be loaded
+ AssignmentManager am = env.getAssignmentManager();
+ return am.waitMetaLoaded(proc);
+ }
}
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..9fc0914 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
@@ -22,12 +22,10 @@ import static org.apache.hadoop.hbase.HConstants.HBASE_SPLIT_WAL_COORDINATED_BY_
import java.io.IOException;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.List;
import org.apache.hadoop.hbase.DoNotRetryIOException;
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.TableState;
import org.apache.hadoop.hbase.master.MasterServices;
@@ -173,7 +171,9 @@ public class ServerCrashProcedure
}
break;
case SERVER_CRASH_ASSIGN_META:
- assignRegions(env, Arrays.asList(RegionInfoBuilder.FIRST_META_REGIONINFO));
+ // notice that, here we will only assign the primary meta regions, secondary meta replicas
+ // will be assigned below
+ assignRegions(env, env.getAssignmentManager().getDefaultMetaRegionsOnServer(serverName));
setNextState(ServerCrashState.SERVER_CRASH_GET_REGIONS);
break;
case SERVER_CRASH_GET_REGIONS:
@@ -249,7 +249,7 @@ public class ServerCrashProcedure
/**
* @return List of Regions on crashed server.
*/
- List<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env) {
+ protected List<RegionInfo> getRegionsOnCrashedServer(MasterProcedureEnv env) {
return env.getMasterServices().getAssignmentManager().getRegionsOnServer(serverName);
}
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 104c9b6..1472b45 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -91,7 +91,7 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.ConnectionUtils;
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.locking.EntityLock;
import org.apache.hadoop.hbase.client.locking.LockServiceClient;
import org.apache.hadoop.hbase.conf.ConfigurationManager;
@@ -1172,7 +1172,8 @@ public class HRegionServer extends Thread implements
}
private boolean containsMetaTableRegions() {
- return onlineRegions.containsKey(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ return onlineRegions.values().stream().map(Region::getRegionInfo)
+ .anyMatch(ri -> ri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(ri));
}
private boolean areAllUserRegionsOffline() {
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
index a75d31c..79a8ea6 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSUtils.java
@@ -69,7 +69,6 @@ import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
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.fs.HFileSystem;
import org.apache.hadoop.hbase.io.HFileLink;
@@ -405,7 +404,7 @@ public final class FSUtils {
String version = getVersion(fs, rootdir);
String msg;
if (version == null) {
- if (!metaRegionExists(fs, rootdir)) {
+ if (!metaTableExists(fs, rootdir)) {
// rootDir is empty (no version file and no root region)
// just create new version file (HBASE-1195)
setVersion(fs, rootdir, wait, retries);
@@ -706,14 +705,14 @@ public final class FSUtils {
}
/**
- * Checks if meta region exists
+ * Checks if meta table exists
* @param fs file system
* @param rootDir root directory of HBase installation
* @return true if exists
*/
- public static boolean metaRegionExists(FileSystem fs, Path rootDir) throws IOException {
- Path metaRegionDir = getRegionDirFromRootDir(rootDir, RegionInfoBuilder.FIRST_META_REGIONINFO);
- return fs.exists(metaRegionDir);
+ private static boolean metaTableExists(FileSystem fs, Path rootDir) throws IOException {
+ Path metaTableDir = CommonFSUtils.getTableDir(rootDir, TableName.META_TABLE_NAME);
+ return fs.exists(metaTableDir);
}
/**
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
index c42ec70..a5f670a 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java
@@ -2715,19 +2715,10 @@ public class HBaseFsck extends Configured implements Closeable {
zkw.getZNodePaths().getZNodeForReplica(hi.getMetaEntry().getRegionInfo().getReplicaId()));
}
- private void assignMetaReplica(int replicaId)
- throws IOException, KeeperException, InterruptedException {
- errors.reportError(ERROR_CODE.NO_META_REGION, "hbase:meta, replicaId " +
- replicaId +" is not found on any region.");
- if (shouldFixAssignments()) {
- errors.print("Trying to fix a problem with hbase:meta..");
- setShouldRerun();
- // try to fix it (treat it as unassigned region)
- RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
- HBaseFsckRepair.fixUnassigned(admin, h);
- HBaseFsckRepair.waitUntilAssigned(admin, h);
- }
+ private void assignMetaReplica(int replicaId) {
+ errors.reportError(ERROR_CODE.NO_META_REGION,
+ "hbase:meta, replicaId " + replicaId + " is not found on any region.");
+ throw new UnsupportedOperationException("fix meta region is not allowed");
}
/**
diff --git a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
index 68c1c63..48cef11 100644
--- a/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
+++ b/hbase-server/src/main/resources/hbase-webapps/master/table.jsp
@@ -261,7 +261,7 @@ if (fqtn != null && master.isInitialized()) {
// NOTE: Presumes meta with one or more replicas
for (int j = 0; j < numMetaReplicas; j++) {
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, j);
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(), j);
RegionState regionState = master.getAssignmentManager().getRegionStates().getRegionState(meta);
ServerName metaLocation = regionState != null ? regionState.getServerName() : null;
for (int i = 0; i < 1; i++) {
@@ -333,7 +333,7 @@ if (fqtn != null && master.isInitialized()) {
// NOTE: Presumes meta with one or more replicas
for (int j = 0; j < numMetaReplicas; j++) {
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, j);
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(), j);
RegionState regionState = master.getAssignmentManager().getRegionStates().getRegionState(meta);
ServerName metaLocation = regionState != null ? regionState.getServerName() : null;
for (int i = 0; i < 1; i++) {
@@ -382,8 +382,8 @@ if (fqtn != null && master.isInitialized()) {
// NOTE: Presumes meta with one or more replicas
for (int j = 0; j < numMetaReplicas; j++) {
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
- RegionInfoBuilder.FIRST_META_REGIONINFO, j);
- RegionState regionState = master.getAssignmentManager().getRegionStates().getRegionState(meta);
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(), j);
+ RegionState regionState = master.getAssignmentManager().getRegionStates().getRegionState(meta);
ServerName metaLocation = regionState != null ? regionState.getServerName() : null;
for (int i = 0; i < 1; i++) {
String hostAndPort = "";
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 85dff35..c31868d 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
@@ -21,7 +21,6 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
@@ -353,10 +352,7 @@ public abstract class HBaseCluster implements Closeable, Configurable {
/**
* Get the ServerName of region server serving the first hbase:meta region
*/
- public ServerName getServerHoldingMeta() throws IOException {
- return getServerHoldingRegion(TableName.META_TABLE_NAME,
- RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
- }
+ public abstract ServerName getServerHoldingMeta() throws IOException;
/**
* Get the ServerName of region server serving the specified region
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
index 4d40ca4..258a61c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/MiniHBaseCluster.java
@@ -26,7 +26,8 @@ import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
+import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.regionserver.HRegion;
import org.apache.hadoop.hbase.regionserver.HRegion.FlushResult;
@@ -827,7 +828,20 @@ public class MiniHBaseCluster extends HBaseCluster {
* of HRS carrying regionName. Returns -1 if none found.
*/
public int getServerWithMeta() {
- return getServerWith(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
+ int index = 0;
+ for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
+ HRegionServer hrs = rst.getRegionServer();
+ if (!hrs.isStopped()) {
+ for (Region region : hrs.getRegions(TableName.META_TABLE_NAME)) {
+ RegionInfo ri = region.getRegionInfo();
+ if (ri.isFirst() && RegionReplicaUtil.isDefaultReplica(ri)) {
+ return index;
+ }
+ }
+ }
+ index++;
+ }
+ return -1;
}
/**
@@ -852,8 +866,24 @@ public class MiniHBaseCluster extends HBaseCluster {
}
@Override
+ public ServerName getServerHoldingMeta() throws IOException {
+ for (JVMClusterUtil.RegionServerThread rst : getRegionServerThreads()) {
+ HRegionServer hrs = rst.getRegionServer();
+ if (!hrs.isStopped()) {
+ for (Region region : hrs.getRegions(TableName.META_TABLE_NAME)) {
+ RegionInfo ri = region.getRegionInfo();
+ if (ri.isFirst() && RegionReplicaUtil.isDefaultReplica(ri)) {
+ return hrs.getServerName();
+ }
+ }
+ }
+ }
+ return null;
+ }
+
+ @Override
public ServerName getServerHoldingRegion(final TableName tn, byte[] regionName)
- throws IOException {
+ throws IOException {
// Assume there is only one master thread which is the active master.
// If there are multiple master threads, the backup master threads
// should hold some regions. Please refer to #countServedRegions
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
index dd3bc73..d1dba10 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestClientClusterMetrics.java
@@ -25,7 +25,6 @@ import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ClusterMetrics.Option;
import org.apache.hadoop.hbase.Waiter.Predicate;
@@ -37,7 +36,6 @@ import org.apache.hadoop.hbase.client.Connection;
import org.apache.hadoop.hbase.client.ConnectionFactory;
import org.apache.hadoop.hbase.client.Get;
import org.apache.hadoop.hbase.client.Put;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.RegionStatesCount;
import org.apache.hadoop.hbase.client.Result;
import org.apache.hadoop.hbase.client.Scan;
@@ -389,11 +387,12 @@ public class TestClientClusterMetrics {
private RegionMetrics getMetaMetrics() throws IOException {
for (ServerMetrics serverMetrics : ADMIN.getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
- .getLiveServerMetrics().values()) {
- RegionMetrics metaMetrics = serverMetrics.getRegionMetrics()
- .get(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
- if (metaMetrics != null) {
- return metaMetrics;
+ .getLiveServerMetrics().values()) {
+ for (RegionMetrics metrics : serverMetrics.getRegionMetrics().values()) {
+ if (CatalogFamilyFormat.parseRegionInfoFromRegionName(metrics.getRegionName())
+ .isMetaRegion()) {
+ return metrics;
+ }
}
}
Assert.fail("Should have find meta metrics");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
index 6977452..41eae30 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseMetaEdit.java
@@ -26,7 +26,6 @@ import java.util.Collections;
import org.apache.hadoop.hbase.client.Admin;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
import org.apache.hadoop.hbase.regionserver.Region;
@@ -114,11 +113,12 @@ public class TestHBaseMetaEdit {
String encoding = descriptor.getColumnFamily(HConstants.CATALOG_FAMILY).getConfiguration().
get(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING);
assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString());
- Region r = UTIL.getHBaseCluster().getRegionServer(0).
- getRegion(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ Region r =
+ UTIL.getHBaseCluster().getRegionServer(0).getRegions(TableName.META_TABLE_NAME).get(0);
assertEquals(oldVersions + 1,
- r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor().getMaxVersions());
- encoding = r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor().
+ r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor().getMaxVersions());
+ encoding = r.getStore(HConstants.CATALOG_FAMILY).getColumnFamilyDescriptor()
+ .
getConfigurationValue(ColumnFamilyDescriptorBuilder.DATA_BLOCK_ENCODING);
assertEquals(encoding, DataBlockEncoding.ROW_INDEX_V1.toString());
assertTrue(r.getStore(extraColumnFamilyName) != null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
index 038ced6..9f1e54d 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestHRegionLocation.java
@@ -22,16 +22,21 @@ import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotSame;
import static org.junit.Assert.assertTrue;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
@Category({ MiscTests.class, SmallTests.class })
public class TestHRegionLocation {
+ private static final Logger LOG = LoggerFactory.getLogger(TestHRegionLocation.class);
+
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestHRegionLocation.class);
@@ -43,17 +48,18 @@ public class TestHRegionLocation {
@Test
public void testHashAndEqualsCode() {
ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
- HRegionLocation hrl1 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa1);
- HRegionLocation hrl2 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa1);
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+ HRegionLocation hrl1 = new HRegionLocation(ri, hsa1);
+ HRegionLocation hrl2 = new HRegionLocation(ri, hsa1);
assertEquals(hrl1.hashCode(), hrl2.hashCode());
assertTrue(hrl1.equals(hrl2));
- HRegionLocation hrl3 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa1);
+ HRegionLocation hrl3 = new HRegionLocation(ri, hsa1);
assertNotSame(hrl1, hrl3);
// They are equal because they have same location even though they are
// carrying different regions or timestamp.
assertTrue(hrl1.equals(hrl3));
ServerName hsa2 = ServerName.valueOf("localhost", 12345, -1L);
- HRegionLocation hrl4 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa2);
+ HRegionLocation hrl4 = new HRegionLocation(ri, hsa2);
// These have same HRI but different locations so should be different.
assertFalse(hrl3.equals(hrl4));
HRegionLocation hrl5 =
@@ -64,17 +70,19 @@ public class TestHRegionLocation {
@Test
public void testToString() {
ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
- HRegionLocation hrl1 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa1);
- System.out.println(hrl1.toString());
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+ HRegionLocation hrl1 = new HRegionLocation(ri, hsa1);
+ LOG.info(hrl1.toString());
}
@SuppressWarnings("SelfComparison")
@Test
public void testCompareTo() {
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
ServerName hsa1 = ServerName.valueOf("localhost", 1234, -1L);
- HRegionLocation hsl1 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa1);
+ HRegionLocation hsl1 = new HRegionLocation(ri, hsa1);
ServerName hsa2 = ServerName.valueOf("localhost", 1235, -1L);
- HRegionLocation hsl2 = new HRegionLocation(RegionInfoBuilder.FIRST_META_REGIONINFO, hsa2);
+ HRegionLocation hsl2 = new HRegionLocation(ri, hsa2);
assertEquals(0, hsl1.compareTo(hsl1));
assertEquals(0, hsl2.compareTo(hsl2));
int compare1 = hsl1.compareTo(hsl2);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
index 977a274..0ea1df4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/MetaWithReplicasTestBase.java
@@ -27,6 +27,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.hbase.Abortable;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionLocation;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.StartMiniClusterOption;
import org.apache.hadoop.hbase.TableName;
@@ -64,16 +65,18 @@ public class MetaWithReplicasTestBase {
TEST_UTIL.startMiniCluster(option);
AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
Set<ServerName> sns = new HashSet<ServerName>();
+ RegionInfo metaRegionInfo;
ServerName hbaseMetaServerName;
try (RegionLocator locator =
TEST_UTIL.getConnection().getRegionLocator(TableName.META_TABLE_NAME)) {
- hbaseMetaServerName = locator.getRegionLocation(HConstants.EMPTY_START_ROW).getServerName();
+ HRegionLocation loc = locator.getRegionLocation(HConstants.EMPTY_START_ROW);
+ metaRegionInfo = loc.getRegion();
+ hbaseMetaServerName = loc.getServerName();
}
LOG.info("HBASE:META DEPLOY: on " + hbaseMetaServerName);
sns.add(hbaseMetaServerName);
for (int replicaId = 1; replicaId < 3; replicaId++) {
- RegionInfo h = RegionReplicaUtil
- .getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId);
+ RegionInfo h = RegionReplicaUtil.getRegionInfoForReplica(metaRegionInfo, replicaId);
AssignmentTestingUtil.waitForAssignment(am, h);
ServerName sn = am.getRegionStates().getRegionServerOfRegion(h);
assertNotNull(sn);
@@ -97,8 +100,7 @@ public class MetaWithReplicasTestBase {
ServerName metaServerName =
TEST_UTIL.getHBaseCluster().getRegionServer(metaServerIndex).getServerName();
assertNotEquals(destinationServerName, metaServerName);
- TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
- destinationServerName);
+ TEST_UTIL.getAdmin().move(metaRegionInfo.getEncodedNameAsBytes(), destinationServerName);
}
// Disable the balancer
LoadBalancerTracker l =
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
index 09c9ab2..9feab47 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAdmin2.java
@@ -738,7 +738,7 @@ public class TestAdmin2 extends TestAdminBase {
testGetWithRegionName(sn, ri, ri.getEncodedNameAsBytes());
testGetWithRegionName(sn, ri, ri.getRegionName());
// Try querying meta encoded name.
- ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ ri = ADMIN.getRegions(TableName.META_TABLE_NAME).get(0);
testGetWithRegionName(sn, ri, ri.getEncodedNameAsBytes());
testGetWithRegionName(sn, ri, ri.getRegionName());
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
index 94adce7..217ee42 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncTableGetMultiThreaded.java
@@ -205,7 +205,8 @@ public class TestAsyncTableGetMultiThreaded {
.map(t -> t.getRegionServer().getServerName()).filter(s -> !s.equals(metaServer))
.findAny().get();
LOG.info("====== Moving meta from {} to {} ======", metaServer, newMetaServer);
- admin.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), newMetaServer);
+ RegionInfo meta = admin.getRegions(TableName.META_TABLE_NAME).get(0);
+ admin.move(meta.getEncodedNameAsBytes(), newMetaServer);
LOG.info("====== Move meta done ======");
Thread.sleep(5000);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFailedMetaReplicaAssigment.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFailedMetaReplicaAssigment.java
index 9a8230b..d242932 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFailedMetaReplicaAssigment.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestFailedMetaReplicaAssigment.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.MasterServices;
import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
@@ -74,9 +75,9 @@ public class TestFailedMetaReplicaAssigment {
TEST_UTIL.waitFor(30000, () -> master.isInitialized());
AssignmentManager am = master.getAssignmentManager();
+ RegionInfo metaHri = am.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).get(0);
// showing one of the replicas got assigned
- RegionInfo metaReplicaHri =
- RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, 1);
+ RegionInfo metaReplicaHri = RegionReplicaUtil.getRegionInfoForReplica(metaHri, 1);
// we use assignAsync so we need to wait a bit
TEST_UTIL.waitFor(30000, () -> {
RegionStateNode metaReplicaRegionNode =
@@ -84,8 +85,7 @@ public class TestFailedMetaReplicaAssigment {
return metaReplicaRegionNode.getRegionLocation() != null;
});
// showing one of the replicas failed to be assigned
- RegionInfo metaReplicaHri2 =
- RegionReplicaUtil.getRegionInfoForReplica(RegionInfoBuilder.FIRST_META_REGIONINFO, 2);
+ RegionInfo metaReplicaHri2 = RegionReplicaUtil.getRegionInfoForReplica(metaHri, 2);
RegionStateNode metaReplicaRegionNode2 =
am.getRegionStates().getOrCreateRegionStateNode(metaReplicaHri2);
// wait for several seconds to make sure that it is not assigned
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaReplicasAddressChange.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaReplicasAddressChange.java
index fe10584..f2bf5c9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaReplicasAddressChange.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaReplicasAddressChange.java
@@ -77,8 +77,8 @@ public class TestMetaReplicasAddressChange extends MetaWithReplicasTestBase {
final TableName tableName = name.getTableName();
TEST_UTIL.createTable(tableName, "f");
assertTrue(TEST_UTIL.getAdmin().tableExists(tableName));
- TEST_UTIL.getAdmin().move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
- moveToServer);
+ RegionInfo metaRegionInfo = TEST_UTIL.getAdmin().getRegions(TableName.META_TABLE_NAME).get(0);
+ TEST_UTIL.getAdmin().move(metaRegionInfo.getEncodedNameAsBytes(), moveToServer);
assertNotEquals(currentServer, moveToServer);
LOG.debug("CurrentServer={}, moveToServer={}", currentServer, moveToServer);
TEST_UTIL.waitFor(60000, () -> {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
index e160fb3..8faebaa 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestMetaTableAccessorNoCluster.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.junit.After;
@@ -80,10 +81,11 @@ public class TestMetaTableAccessorNoCluster {
assertTrue(hri == null);
// OK, give it what it expects
kvs.clear();
+ RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY, f, HConstants.REGIONINFO_QUALIFIER,
- RegionInfo.toByteArray(RegionInfoBuilder.FIRST_META_REGIONINFO)));
+ RegionInfo.toByteArray(metaRegionInfo)));
hri = CatalogFamilyFormat.getRegionInfo(Result.create(kvs));
assertNotNull(hri);
- assertTrue(RegionInfo.COMPARATOR.compare(hri, RegionInfoBuilder.FIRST_META_REGIONINFO) == 0);
+ assertTrue(RegionInfo.COMPARATOR.compare(hri, metaRegionInfo) == 0);
}
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
index 4665e84..c655261 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestSeparateClientZKCluster.java
@@ -172,7 +172,8 @@ public class TestSeparateClientZKCluster {
break;
}
}
- admin.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(), destServerName);
+ RegionInfo metaRegion = admin.getRegions(TableName.META_TABLE_NAME).get(0);
+ admin.move(metaRegion.getEncodedNameAsBytes(), destServerName);
LOG.debug("Finished moving meta");
// invalidate client cache
RegionInfo region = locator.getRegionLocation(row).getRegion();
@@ -210,6 +211,7 @@ public class TestSeparateClientZKCluster {
Put put = new Put(row);
put.addColumn(family, qualifier, value);
table.put(put);
+ RegionInfo metaRegion = admin.getRegions(TableName.META_TABLE_NAME).get(0);
// invalid connection cache
conn.clearRegionLocationCache();
// stop client zk cluster
@@ -225,7 +227,7 @@ public class TestSeparateClientZKCluster {
}
// wait for meta region online
AssignmentTestingUtil.waitForAssignment(cluster.getMaster().getAssignmentManager(),
- RegionInfoBuilder.FIRST_META_REGIONINFO);
+ metaRegion);
// wait some long time to make sure we will retry sync data to client ZK until data set
Thread.sleep(10000);
clientZkCluster.startup(clientZkDir);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java
index 734d4e0..ef8bc1b 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestRegionObserverPreFlushAndPreCompact.java
@@ -24,6 +24,7 @@ import java.util.Optional;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
+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.client.TableDescriptor;
@@ -109,7 +110,7 @@ public class TestRegionObserverPreFlushAndPreCompact {
// Make up an HRegion instance. Use the hbase:meta first region as our RegionInfo. Use
// hbase:meta table name for building the TableDescriptor our mock returns when asked schema
// down inside RegionCoprocessorHost. Pass in mocked RegionServerServices too.
- RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
HRegion mockedHRegion = Mockito.mock(HRegion.class);
Mockito.when(mockedHRegion.getRegionInfo()).thenReturn(ri);
TableDescriptor td = TableDescriptorBuilder.newBuilder(ri.getTable()).build();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
index f5d5d41..9f1a0bb 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/AbstractTestDLS.java
@@ -62,7 +62,6 @@ import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
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.RegionLocator;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.coordination.ZKSplitLogManagerCoordination;
@@ -223,7 +222,6 @@ public abstract class AbstractTestDLS {
int count = 0;
for (RegionInfo hri : regions) {
- @SuppressWarnings("deprecation")
Path editsdir = WALSplitUtil
.getRegionDirRecoveredEditsDir(CommonFSUtils.getWALRegionDir(conf,
tableName, hri.getEncodedName()));
@@ -299,7 +297,7 @@ public abstract class AbstractTestDLS {
LOG.info("Current Open Regions After Master Node Starts Up:" +
HBaseTestingUtility.getAllOnlineRegions(cluster).size());
- assertEquals(numLogLines, TEST_UTIL.countRows(ht));
+ assertEquals(numLogLines, HBaseTestingUtility.countRows(ht));
}
}
@@ -400,7 +398,7 @@ public abstract class AbstractTestDLS {
TEST_UTIL.waitUntilAllRegionsAssigned(tableName);
int rows;
try {
- rows = TEST_UTIL.countRows(table);
+ rows = HBaseTestingUtility.countRows(table);
} catch (Exception e) {
Threads.printThreadInfo(System.out, "Thread dump before fail");
throw e;
@@ -566,9 +564,7 @@ public abstract class AbstractTestDLS {
public void makeWAL(HRegionServer hrs, List<RegionInfo> regions, int numEdits, int editSize,
boolean cleanShutdown) throws IOException {
- // remove root and meta region
- regions.remove(RegionInfoBuilder.FIRST_META_REGIONINFO);
-
+ // remove meta and system regions
for (Iterator<RegionInfo> iter = regions.iterator(); iter.hasNext();) {
RegionInfo regionInfo = iter.next();
if (regionInfo.getTable().isSystemTable()) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
index 1b2d86e..5516a4f 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/MockRegionServer.java
@@ -431,10 +431,11 @@ class MockRegionServer implements AdminProtos.AdminService.BlockingInterface,
}
@Override
- public GetRegionInfoResponse getRegionInfo(RpcController controller,
- GetRegionInfoRequest request) throws ServiceException {
+ public GetRegionInfoResponse getRegionInfo(RpcController controller, GetRegionInfoRequest request)
+ throws ServiceException {
GetRegionInfoResponse.Builder builder = GetRegionInfoResponse.newBuilder();
- builder.setRegionInfo(ProtobufUtil.toRegionInfo(RegionInfoBuilder.FIRST_META_REGIONINFO));
+ builder.setRegionInfo(ProtobufUtil.toRegionInfo(
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setRegionId(1).build()));
return builder.build();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
index e653234..ff8bbd4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMaster.java
@@ -176,7 +176,8 @@ public class TestMaster {
HMaster m = cluster.getMaster();
try {
m.setInitialized(false); // fake it, set back later
- RegionInfo meta = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ RegionInfo meta = m.getAssignmentManager().getRegionStates()
+ .getRegionsOfTable(TableName.META_TABLE_NAME).get(0);
m.move(meta.getEncodedNameAsBytes(), null);
fail("Region should not be moved since master is not initialized");
} catch (IOException ioe) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
index 3e7f605..43c23dd 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMasterQosFunction.java
@@ -67,7 +67,7 @@ public class TestMasterQosFunction extends QosTestHelper {
public void testRegionInTransition() throws IOException {
// Check ReportRegionInTransition
HBaseProtos.RegionInfo meta_ri =
- ProtobufUtil.toRegionInfo(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ ProtobufUtil.toRegionInfo(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build());
HBaseProtos.RegionInfo normal_ri =
ProtobufUtil.toRegionInfo(RegionInfoBuilder.newBuilder(TableName.valueOf("test:table"))
.setStartKey(Bytes.toBytes("a")).setEndKey(Bytes.toBytes("b")).build());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java
index 5b80f89..3cf8f8c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestMetaFixerNoCluster.java
@@ -53,7 +53,7 @@ public class TestMetaFixerNoCluster {
private static byte [] B = Bytes.toBytes("b");
private static byte [] C = Bytes.toBytes("c");
private static byte [] D = Bytes.toBytes("d");
- private static RegionInfo ALL = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ private static RegionInfo ALL = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
private static RegionInfo _ARI = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
setEndKey(A).build();
private static RegionInfo _BRI = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
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 efb784e..45d1f98 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
@@ -28,8 +28,9 @@ import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster.MiniHBaseClusterRegionServer;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.StartMiniClusterOption;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.Bytes;
@@ -70,13 +71,10 @@ public class TestMetaShutdownHandler {
}
/**
- * This test will test the expire handling of a meta-carrying
- * region server.
- * After HBaseMiniCluster is up, we will delete the ephemeral
- * node of the meta-carrying region server, which will trigger
- * the expire of this region server on the master.
- * On the other hand, we will slow down the abort process on
- * the region server so that it is still up during the master SSH.
+ * This test will test the expire handling of a meta-carrying region server. After
+ * HBaseMiniCluster is up, we will delete the ephemeral node of the meta-carrying region server,
+ * which will trigger the expire of this region server on the master. On the other hand, we will
+ * slow down the abort process on the region server so that it is still up during the master SSH.
* We will check that the master SSH is still successfully done.
*/
@Test
@@ -84,26 +82,24 @@ public class TestMetaShutdownHandler {
MiniHBaseCluster cluster = TEST_UTIL.getHBaseCluster();
HMaster master = cluster.getMaster();
RegionStates regionStates = master.getAssignmentManager().getRegionStates();
- ServerName metaServerName =
- regionStates.getRegionServerOfRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ RegionInfo firstMetaRegion = regionStates.getRegionsOfTable(TableName.META_TABLE_NAME).get(0);
+ ServerName metaServerName = regionStates.getRegionServerOfRegion(firstMetaRegion);
if (master.getServerName().equals(metaServerName) || metaServerName == null ||
!metaServerName.equals(cluster.getServerHoldingMeta())) {
// Move meta off master
metaServerName =
cluster.getLiveRegionServerThreads().get(0).getRegionServer().getServerName();
- master.move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
+ master.move(firstMetaRegion.getEncodedNameAsBytes(),
Bytes.toBytes(metaServerName.getServerName()));
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
- metaServerName =
- regionStates.getRegionServerOfRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ metaServerName = regionStates.getRegionServerOfRegion(firstMetaRegion);
}
assertNotEquals("Meta is on master!", metaServerName, master.getServerName());
// Delete the ephemeral node of the meta-carrying region server.
// This is trigger the expire of this region server on the master.
- String rsEphemeralNodePath =
- ZNodePaths.joinZNode(master.getZooKeeper().getZNodePaths().rsZNode,
- metaServerName.toString());
+ String rsEphemeralNodePath = ZNodePaths.joinZNode(master.getZooKeeper().getZNodePaths().rsZNode,
+ metaServerName.toString());
ZKUtil.deleteNode(master.getZooKeeper(), rsEphemeralNodePath);
LOG.info("Deleted the znode for the RegionServer hosting hbase:meta; waiting on SSH");
// Wait for SSH to finish
@@ -112,18 +108,16 @@ public class TestMetaShutdownHandler {
TEST_UTIL.waitFor(120000, 200, new Waiter.Predicate<Exception>() {
@Override
public boolean evaluate() throws Exception {
- return !serverManager.isServerOnline(priorMetaServerName)
- && !serverManager.areDeadServersInProgress();
+ return !serverManager.isServerOnline(priorMetaServerName) &&
+ !serverManager.areDeadServersInProgress();
}
});
LOG.info("Past wait on RIT");
TEST_UTIL.waitUntilNoRegionsInTransition(60000);
// Now, make sure meta is assigned
- assertTrue("Meta should be assigned",
- regionStates.isRegionOnline(RegionInfoBuilder.FIRST_META_REGIONINFO));
+ assertTrue("Meta should be assigned", regionStates.isRegionOnline(firstMetaRegion));
// Now, make sure meta is registered in zk
- ServerName newMetaServerName =
- regionStates.getRegionServerOfRegion(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ ServerName newMetaServerName = regionStates.getRegionServerOfRegion(firstMetaRegion);
assertNotEquals("Meta should be assigned on a different server", newMetaServerName,
metaServerName);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
index 65001e2..3efa147 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/AssignmentTestingUtil.java
@@ -21,15 +21,18 @@ import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;
import java.io.IOException;
+import java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.Waiter;
import org.apache.hadoop.hbase.Waiter.ExplainingPredicate;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.RegionState.State;
import org.apache.hadoop.hbase.master.procedure.ProcedureSyncWait;
+import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
@@ -111,16 +114,16 @@ public final class AssignmentTestingUtil {
public static boolean isServerHoldingMeta(final HBaseTestingUtility util,
final ServerName serverName) throws Exception {
- for (RegionInfo hri: getMetaRegions(util)) {
- if (serverName.equals(getServerHoldingRegion(util, hri))) {
- return true;
- }
+ HRegionServer server = util.getMiniHBaseCluster().getRegionServer(serverName);
+ if (server == null) {
+ return false;
}
- return false;
+ return !server.getRegions(TableName.META_TABLE_NAME).isEmpty();
}
public static Set<RegionInfo> getMetaRegions(final HBaseTestingUtility util) {
- return getMaster(util).getAssignmentManager().getMetaRegionSet();
+ return new HashSet<>(getMaster(util).getAssignmentManager().getRegionStates()
+ .getTableRegionsInfo(TableName.META_TABLE_NAME));
}
private static HMaster getMaster(final HBaseTestingUtility util) {
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..2d2fbf7 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.MetaTableAccessor;
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.master.RegionState.State;
import org.apache.hadoop.hbase.procedure2.ProcedureTestingUtility;
import org.apache.hadoop.hbase.procedure2.util.StringUtils;
@@ -236,10 +235,7 @@ public class TestAssignmentManager extends TestAssignmentManagerBase {
am = master.getAssignmentManager();
// Assign meta
- rsDispatcher.setMockRsExecutor(new HangThenRSRestartExecutor());
- am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
- assertEquals(true, am.isMetaAssigned());
-
+ setUpMeta(new HangThenRSRestartExecutor());
// set it back as default, see setUpMeta()
am.wakeMetaLoadedEvent();
}
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..252ea61 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
@@ -57,7 +57,6 @@ import org.apache.hadoop.hbase.master.procedure.RSProcedureDispatcher;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureMetrics;
import org.apache.hadoop.hbase.procedure2.ProcedureUtil;
-import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
import org.apache.hadoop.hbase.util.Bytes;
@@ -140,8 +139,6 @@ public abstract class TestAssignmentManagerBase {
protected void setupConfiguration(Configuration conf) throws Exception {
CommonFSUtils.setRootDir(conf, util.getDataTestDir());
- conf.setBoolean(WALProcedureStore.USE_HSYNC_CONF_KEY, false);
- conf.setInt(WALProcedureStore.SYNC_WAIT_MSEC_CONF_KEY, 10);
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, PROC_NTHREADS);
conf.setInt(RSProcedureDispatcher.RS_RPC_STARTUP_WAIT_TIME_CONF_KEY, 1000);
conf.setInt(AssignmentManager.ASSIGN_MAX_ATTEMPTS, getAssignMaxAttempts());
@@ -167,12 +164,12 @@ public abstract class TestAssignmentManagerBase {
reopenProcMetrics = am.getAssignmentManagerMetrics().getReopenProcMetrics();
openProcMetrics = am.getAssignmentManagerMetrics().getOpenProcMetrics();
closeProcMetrics = am.getAssignmentManagerMetrics().getCloseProcMetrics();
- setUpMeta();
+ setUpMeta(new GoodRsExecutor());
}
- protected void setUpMeta() throws Exception {
- rsDispatcher.setMockRsExecutor(new GoodRsExecutor());
- am.assign(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ protected final void setUpMeta(MockRSExecutor mockRsExec) throws Exception {
+ rsDispatcher.setMockRsExecutor(mockRsExec);
+ am.assign(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setRegionId(1).build());
am.wakeMetaLoadedEvent();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
index 5673ed8..28b4c25 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestAssignmentOnRSCrash.java
@@ -41,8 +41,6 @@ import org.junit.Before;
import org.junit.ClassRule;
import org.junit.Test;
import org.junit.experimental.categories.Category;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
@Category({MasterTests.class, LargeTests.class})
public class TestAssignmentOnRSCrash {
@@ -51,8 +49,6 @@ public class TestAssignmentOnRSCrash {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestAssignmentOnRSCrash.class);
- private static final Logger LOG = LoggerFactory.getLogger(TestAssignmentOnRSCrash.class);
-
private static final TableName TEST_TABLE = TableName.valueOf("testb");
private static final String FAMILY_STR = "f";
private static final byte[] FAMILY = Bytes.toBytes(FAMILY_STR);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
index 6c5a811..c06959e 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/assignment/TestHbckChore.java
@@ -64,8 +64,9 @@ public class TestHbckChore extends TestAssignmentManagerBase {
@Test
public void testForMeta() {
- byte[] metaRegionNameAsBytes = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName();
- String metaRegionName = RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionNameAsString();
+ RegionInfo meta = am.getRegionStates().getRegionsOfTable(TableName.META_TABLE_NAME).get(0);
+ byte[] metaRegionNameAsBytes = meta.getRegionName();
+ String metaRegionName = meta.getRegionNameAsString();
List<ServerName> serverNames = master.getServerManager().getOnlineServersList();
assertEquals(NSERVERS, serverNames.size());
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
index 0bdbbd0..1212aba 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestBaseLoadBalancer.java
@@ -154,13 +154,8 @@ public class TestBaseLoadBalancer extends BalancerTestBase {
public void testBulkAssignment() throws Exception {
List<ServerName> tmp = getListOfServerNames(randomServers(5, 0));
List<RegionInfo> hris = randomRegions(20);
- hris.add(RegionInfoBuilder.FIRST_META_REGIONINFO);
tmp.add(master);
Map<ServerName, List<RegionInfo>> plans = loadBalancer.roundRobinAssignment(hris, tmp);
- if (LoadBalancer.isTablesOnMaster(loadBalancer.getConf())) {
- assertTrue(plans.get(master).contains(RegionInfoBuilder.FIRST_META_REGIONINFO));
- assertEquals(1, plans.get(master).size());
- }
int totalRegion = 0;
for (List<RegionInfo> regions: plans.values()) {
totalRegion += regions.size();
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
index fd1e533..5288ff8 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCleanupMetaWAL.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.TableName;
-import org.apache.hadoop.hbase.client.RegionInfoBuilder;
+import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.master.MasterFileSystem;
import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
import org.apache.hadoop.hbase.testclassification.MediumTests;
@@ -64,8 +64,8 @@ public class TestCleanupMetaWAL {
TEST_UTIL.createTable(TableName.valueOf("test"), "cf");
HRegionServer serverWithMeta = TEST_UTIL.getMiniHBaseCluster()
.getRegionServer(TEST_UTIL.getMiniHBaseCluster().getServerWithMeta());
- TEST_UTIL.getAdmin()
- .move(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
+ RegionInfo metaInfo = TEST_UTIL.getAdmin().getRegions(TableName.META_TABLE_NAME).get(0);
+ TEST_UTIL.getAdmin().move(metaInfo.getEncodedNameAsBytes());
LOG.info("KILL");
TEST_UTIL.getMiniHBaseCluster().killRegionServer(serverWithMeta.getServerName());
LOG.info("WAIT");
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
index 27b301f..8b1abe9 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java
@@ -951,9 +951,9 @@ public class TestDefaultMemStore {
WALFactory wFactory = new WALFactory(conf, "1234");
TableDescriptors tds = new FSTableDescriptors(conf);
FSTableDescriptors.tryUpdateMetaTableDescriptor(conf);
- HRegion meta = HRegion.createHRegion(RegionInfoBuilder.FIRST_META_REGIONINFO, testDir,
- conf, tds.get(TableName.META_TABLE_NAME),
- wFactory.getWAL(RegionInfoBuilder.FIRST_META_REGIONINFO));
+ RegionInfo metaRegionInfo = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+ HRegion meta = HRegion.createHRegion(metaRegionInfo, testDir, conf,
+ tds.get(TableName.META_TABLE_NAME), wFactory.getWAL(metaRegionInfo));
// parameterized tests add [#] suffix get rid of [ and ].
TableDescriptor desc = TableDescriptorBuilder
.newBuilder(TableName.valueOf(name.getMethodName().replaceAll("[\\[\\]]", "_")))
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
index e832c47..49d9d66 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultStoreEngine.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.regionserver.compactions.DefaultCompactor;
import org.apache.hadoop.hbase.regionserver.compactions.RatioBasedCompactionPolicy;
@@ -62,11 +63,11 @@ public class TestDefaultStoreEngine {
Configuration conf = HBaseConfiguration.create();
conf.set(DefaultStoreEngine.DEFAULT_COMPACTOR_CLASS_KEY, DummyCompactor.class.getName());
conf.set(DefaultStoreEngine.DEFAULT_COMPACTION_POLICY_CLASS_KEY,
- DummyCompactionPolicy.class.getName());
- conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY,
- DummyStoreFlusher.class.getName());
+ DummyCompactionPolicy.class.getName());
+ conf.set(DefaultStoreEngine.DEFAULT_STORE_FLUSHER_CLASS_KEY, DummyStoreFlusher.class.getName());
HStore mockStore = Mockito.mock(HStore.class);
- Mockito.when(mockStore.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ Mockito.when(mockStore.getRegionInfo())
+ .thenReturn(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build());
StoreEngine<?, ?, ?, ?> se = StoreEngine.create(mockStore, conf, CellComparatorImpl.COMPARATOR);
Assert.assertTrue(se instanceof DefaultStoreEngine);
Assert.assertTrue(se.getCompactionPolicy() instanceof DummyCompactionPolicy);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
index 79f0291..3d01124 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestGetClosestAtOrBefore.java
@@ -94,8 +94,8 @@ public class TestGetClosestAtOrBefore {
FSTableDescriptors.tryUpdateMetaTableDescriptor(UTIL.getConfiguration());
TableDescriptor td = tds.get(TableName.META_TABLE_NAME);
td = TableDescriptorBuilder.newBuilder(td).setMemStoreFlushSize(64 * 1024 * 1024).build();
- HRegion mr = HBaseTestingUtility.createRegionAndWAL(RegionInfoBuilder.FIRST_META_REGIONINFO,
- rootdir, conf, td);
+ HRegion mr = HBaseTestingUtility.createRegionAndWAL(
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build(), rootdir, conf, td);
try {
// Write rows for three tables 'A', 'B', and 'C'.
for (char c = 'A'; c < 'D'; c++) {
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
index d75d055..083854c 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPriorityRpc.java
@@ -59,6 +59,9 @@ public class TestPriorityRpc {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestPriorityRpc.class);
+ private static final RegionInfo FIRST_META_REGIONINFO =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+
private Configuration conf;
private HRegionServer regionServer = null;
private PriorityFunction priority = null;
@@ -85,8 +88,7 @@ public class TestPriorityRpc {
GetRequest.Builder getRequestBuilder = GetRequest.newBuilder();
RegionSpecifier.Builder regionSpecifierBuilder = RegionSpecifier.newBuilder();
regionSpecifierBuilder.setType(RegionSpecifierType.REGION_NAME);
- ByteString name = UnsafeByteOperations.unsafeWrap(
- RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName());
+ ByteString name = UnsafeByteOperations.unsafeWrap(FIRST_META_REGIONINFO.getRegionName());
regionSpecifierBuilder.setValue(name);
RegionSpecifier regionSpecifier = regionSpecifierBuilder.build();
getRequestBuilder.setRegion(regionSpecifier);
@@ -102,8 +104,7 @@ public class TestPriorityRpc {
RegionInfo mockRegionInfo = Mockito.mock(RegionInfo.class);
Mockito.when(mockRpc.getRegion(Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
- Mockito.when(mockRegionInfo.getTable())
- .thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable());
+ Mockito.when(mockRegionInfo.getTable()).thenReturn(FIRST_META_REGIONINFO.getTable());
// Presume type.
((AnnotationReadingPriorityFunction)priority).setRegionServer(mockRS);
assertEquals(
@@ -157,8 +158,7 @@ public class TestPriorityRpc {
Mockito.when(mockRegionScanner.getRegionInfo()).thenReturn(mockRegionInfo);
Mockito.when(mockRpc.getRegion((RegionSpecifier)Mockito.any())).thenReturn(mockRegion);
Mockito.when(mockRegion.getRegionInfo()).thenReturn(mockRegionInfo);
- Mockito.when(mockRegionInfo.getTable())
- .thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO.getTable());
+ Mockito.when(mockRegionInfo.getTable()).thenReturn(FIRST_META_REGIONINFO.getTable());
// Presume type.
((AnnotationReadingPriorityFunction)priority).setRegionServer(mockRS);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
index 103fb73..c90aaf7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestReadAndWriteRegionInfoFile.java
@@ -68,7 +68,7 @@ public class TestReadAndWriteRegionInfoFile {
@Test
public void testReadAndWriteRegionInfoFile() throws IOException, InterruptedException {
- RegionInfo ri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
// Create a region. That'll write the .regioninfo file.
FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(FS, ROOT_DIR);
FSTableDescriptors.tryUpdateMetaTableDescriptor(CONF, FS, ROOT_DIR, null);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
index 1bd8da6..800a4a4 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionInfo.java
@@ -55,70 +55,58 @@ import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
-@Category({RegionServerTests.class, SmallTests.class})
+@Category({ RegionServerTests.class, SmallTests.class })
public class TestRegionInfo {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
- HBaseClassTestRule.forClass(TestRegionInfo.class);
+ HBaseClassTestRule.forClass(TestRegionInfo.class);
@Rule
public TestName name = new TestName();
@Test
public void testIsStart() {
- assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst());
- org.apache.hadoop.hbase.client.RegionInfo ri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(Bytes.toBytes("not_start")).build();
+ assertTrue(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build().isFirst());
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME)
+ .setStartKey(Bytes.toBytes("not_start")).build();
assertFalse(ri.isFirst());
}
@Test
public void testIsEnd() {
- assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isFirst());
- org.apache.hadoop.hbase.client.RegionInfo ri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setEndKey(Bytes.toBytes("not_end")).build();
+ assertTrue(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build().isLast());
+ RegionInfo ri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME)
+ .setEndKey(Bytes.toBytes("not_end")).build();
assertFalse(ri.isLast());
}
@Test
public void testIsNext() {
- byte [] bytes = Bytes.toBytes("row");
- org.apache.hadoop.hbase.client.RegionInfo ri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setEndKey(bytes).build();
- org.apache.hadoop.hbase.client.RegionInfo ri2 =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(bytes).build();
- assertFalse(ri.isNext(RegionInfoBuilder.FIRST_META_REGIONINFO));
+ byte[] bytes = Bytes.toBytes("row");
+ RegionInfo ri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(bytes).build();
+ RegionInfo ri2 =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(bytes).build();
+ assertFalse(ri.isNext(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build()));
assertTrue(ri.isNext(ri2));
}
@Test
public void testIsOverlap() {
- byte [] a = Bytes.toBytes("a");
- byte [] b = Bytes.toBytes("b");
- byte [] c = Bytes.toBytes("c");
- byte [] d = Bytes.toBytes("d");
- org.apache.hadoop.hbase.client.RegionInfo all =
- RegionInfoBuilder.FIRST_META_REGIONINFO;
- org.apache.hadoop.hbase.client.RegionInfo ari =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setEndKey(a).build();
- org.apache.hadoop.hbase.client.RegionInfo abri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(a).setEndKey(b).build();
- org.apache.hadoop.hbase.client.RegionInfo adri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(a).setEndKey(d).build();
- org.apache.hadoop.hbase.client.RegionInfo cdri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(c).setEndKey(d).build();
- org.apache.hadoop.hbase.client.RegionInfo dri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(d).build();
+ byte[] a = Bytes.toBytes("a");
+ byte[] b = Bytes.toBytes("b");
+ byte[] c = Bytes.toBytes("c");
+ byte[] d = Bytes.toBytes("d");
+ RegionInfo all = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
+ RegionInfo ari = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(a).build();
+ RegionInfo abri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(a).setEndKey(b).build();
+ RegionInfo adri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(a).setEndKey(d).build();
+ RegionInfo cdri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(c).setEndKey(d).build();
+ RegionInfo dri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(d).build();
assertTrue(all.isOverlap(all));
assertTrue(all.isOverlap(abri));
assertFalse(abri.isOverlap(cdri));
@@ -144,21 +132,14 @@ public class TestRegionInfo {
byte[] d = Bytes.toBytes("d");
byte[] e = Bytes.toBytes("e");
byte[] f = Bytes.toBytes("f");
- org.apache.hadoop.hbase.client.RegionInfo ari =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setEndKey(a).build();
- org.apache.hadoop.hbase.client.RegionInfo abri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(a).setEndKey(b).build();
- org.apache.hadoop.hbase.client.RegionInfo eri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setEndKey(e).build();
- org.apache.hadoop.hbase.client.RegionInfo cdri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(c).setEndKey(d).build();
- org.apache.hadoop.hbase.client.RegionInfo efri =
- org.apache.hadoop.hbase.client.RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).
- setStartKey(e).setEndKey(f).build();
+ RegionInfo ari = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(a).build();
+ RegionInfo abri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(a).setEndKey(b).build();
+ RegionInfo eri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setEndKey(e).build();
+ RegionInfo cdri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(c).setEndKey(d).build();
+ RegionInfo efri =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).setStartKey(e).setEndKey(f).build();
assertFalse(ari.isOverlap(abri));
assertTrue(abri.isOverlap(eri));
assertFalse(cdri.isOverlap(efri));
@@ -167,8 +148,9 @@ public class TestRegionInfo {
@Test
public void testPb() throws DeserializationException {
- RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
- byte [] bytes = RegionInfo.toByteArray(hri);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.valueOf("test"))
+ .setStartKey(Bytes.toBytes("start")).build();
+ byte[] bytes = RegionInfo.toByteArray(hri);
RegionInfo pbhri = RegionInfo.parseFrom(bytes);
assertTrue(hri.equals(pbhri));
}
@@ -176,28 +158,26 @@ public class TestRegionInfo {
@Test
public void testReadAndWriteHRegionInfoFile() throws IOException, InterruptedException {
HBaseTestingUtility htu = new HBaseTestingUtility();
- RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
Path basedir = htu.getDataTestDir();
- // Create a region. That'll write the .regioninfo file.
+ // Create a region. That'll write the .regioninfo file.
FSTableDescriptors fsTableDescriptors = new FSTableDescriptors(htu.getConfiguration());
FSTableDescriptors.tryUpdateMetaTableDescriptor(htu.getConfiguration());
HRegion r = HBaseTestingUtility.createRegionAndWAL(hri, basedir, htu.getConfiguration(),
- fsTableDescriptors.get(TableName.META_TABLE_NAME));
+ fsTableDescriptors.get(TableName.META_TABLE_NAME));
// Get modtime on the file.
long modtime = getModTime(r);
HBaseTestingUtility.closeRegionAndWAL(r);
Thread.sleep(1001);
- r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(TableName.META_TABLE_NAME),
- null, htu.getConfiguration());
+ r = HRegion.openHRegion(basedir, hri, fsTableDescriptors.get(TableName.META_TABLE_NAME), null,
+ htu.getConfiguration());
// Ensure the file is not written for a second time.
long modtime2 = getModTime(r);
assertEquals(modtime, modtime2);
// Now load the file.
- org.apache.hadoop.hbase.client.RegionInfo deserializedHri =
- HRegionFileSystem.loadRegionInfoFileContent(
- r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
- assertEquals(0,
- org.apache.hadoop.hbase.client.RegionInfo.COMPARATOR.compare(hri, deserializedHri));
+ RegionInfo deserializedHri = HRegionFileSystem.loadRegionInfoFileContent(
+ r.getRegionFileSystem().getFileSystem(), r.getRegionFileSystem().getRegionDir());
+ assertEquals(0, RegionInfo.COMPARATOR.compare(hri, deserializedHri));
HBaseTestingUtility.closeRegionAndWAL(r);
}
@@ -217,19 +197,16 @@ public class TestRegionInfo {
String id = "id";
// old format region name
- byte [] name = RegionInfo.createRegionName(tn, sk, id, false);
+ byte[] name = RegionInfo.createRegionName(tn, sk, id, false);
String nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id, nameStr);
-
// new format region name.
String md5HashInHex = MD5Hash.getMD5AsHex(name);
assertEquals(RegionInfo.MD5_HEX_LENGTH, md5HashInHex.length());
name = RegionInfo.createRegionName(tn, sk, id, true);
nameStr = Bytes.toString(name);
- assertEquals(tableName + "," + startKey + ","
- + id + "." + md5HashInHex + ".",
- nameStr);
+ assertEquals(tableName + "," + startKey + "," + id + "." + md5HashInHex + ".", nameStr);
}
@Test
@@ -274,7 +251,8 @@ public class TestRegionInfo {
@Test
public void testMetaTables() {
- assertTrue(RegionInfoBuilder.FIRST_META_REGIONINFO.isMetaRegion());
+ assertTrue(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build().isMetaRegion());
+ assertFalse(RegionInfoBuilder.newBuilder(TableName.valueOf("test")).build().isMetaRegion());
}
@SuppressWarnings("SelfComparison")
@@ -292,7 +270,7 @@ public class TestRegionInfo {
RegionInfo b = RegionInfoBuilder.newBuilder(TableName.valueOf("b")).build();
assertNotEquals(0, a.compareTo(b));
TableName t = TableName.valueOf("t");
- byte [] midway = Bytes.toBytes("midway");
+ byte[] midway = Bytes.toBytes("midway");
a = RegionInfoBuilder.newBuilder(t).setEndKey(midway).build();
b = RegionInfoBuilder.newBuilder(t).setStartKey(midway).build();
assertTrue(a.compareTo(b) < 0);
@@ -328,21 +306,22 @@ public class TestRegionInfo {
// assert with only the region name without encoding
// primary, replicaId = 0
- byte [] name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0, false);
+ byte[] name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0, false);
String nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id, nameStr);
// replicaId = 1
name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 1, false);
nameStr = Bytes.toString(name);
- assertEquals(tableName + "," + startKey + "," + id + "_" +
- String.format(RegionInfo.REPLICA_ID_FORMAT, 1), nameStr);
+ assertEquals(
+ tableName + "," + startKey + "," + id + "_" + String.format(RegionInfo.REPLICA_ID_FORMAT, 1),
+ nameStr);
// replicaId = max
name = RegionInfo.createRegionName(tn, sk, Bytes.toBytes(id), 0xFFFF, false);
nameStr = Bytes.toString(name);
assertEquals(tableName + "," + startKey + "," + id + "_" +
- String.format(RegionInfo.REPLICA_ID_FORMAT, 0xFFFF), nameStr);
+ String.format(RegionInfo.REPLICA_ID_FORMAT, 0xFFFF), nameStr);
}
@Test
@@ -356,21 +335,20 @@ public class TestRegionInfo {
byte[] regionName = RegionInfo.createRegionName(tableName, startKey, regionId, false);
byte[][] fields = RegionInfo.parseRegionName(regionName);
- assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
- assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
- assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
+ assertArrayEquals(Bytes.toString(fields[0]), tableName.getName(), fields[0]);
+ assertArrayEquals(Bytes.toString(fields[1]), startKey, fields[1]);
+ assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)), fields[2]);
assertEquals(3, fields.length);
// test with replicaId
- regionName = RegionInfo.createRegionName(tableName, startKey, regionId,
- replicaId, false);
+ regionName = RegionInfo.createRegionName(tableName, startKey, regionId, replicaId, false);
fields = RegionInfo.parseRegionName(regionName);
- assertArrayEquals(Bytes.toString(fields[0]),tableName.getName(), fields[0]);
- assertArrayEquals(Bytes.toString(fields[1]),startKey, fields[1]);
- assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)),fields[2]);
- assertArrayEquals(Bytes.toString(fields[3]), Bytes.toBytes(
- String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)), fields[3]);
+ assertArrayEquals(Bytes.toString(fields[0]), tableName.getName(), fields[0]);
+ assertArrayEquals(Bytes.toString(fields[1]), startKey, fields[1]);
+ assertArrayEquals(Bytes.toString(fields[2]), Bytes.toBytes(Long.toString(regionId)), fields[2]);
+ assertArrayEquals(Bytes.toString(fields[3]),
+ Bytes.toBytes(String.format(RegionInfo.REPLICA_ID_FORMAT, replicaId)), fields[3]);
}
@Test
@@ -406,10 +384,11 @@ public class TestRegionInfo {
assertEquals(expectedHri, convertedHri);
}
+
@Test
public void testRegionDetailsForDisplay() throws IOException {
- byte[] startKey = new byte[] {0x01, 0x01, 0x02, 0x03};
- byte[] endKey = new byte[] {0x01, 0x01, 0x02, 0x04};
+ byte[] startKey = new byte[] { 0x01, 0x01, 0x02, 0x03 };
+ byte[] endKey = new byte[] { 0x01, 0x01, 0x02, 0x04 };
Configuration conf = new Configuration();
conf.setBoolean("hbase.display.keys", false);
RegionInfo h = RegionInfoBuilder.newBuilder(TableName.valueOf(name.getMethodName()))
@@ -472,4 +451,3 @@ public class TestRegionInfo {
}
}
}
-
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
index f5330f6..dbab4e7 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestStripeStoreEngine.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequestImpl;
@@ -119,7 +120,8 @@ public class TestStripeStoreEngine {
private static TestStoreEngine createEngine(Configuration conf) throws Exception {
HStore store = mock(HStore.class);
- when(store.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ when(store.getRegionInfo())
+ .thenReturn(RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build());
CellComparatorImpl kvComparator = mock(CellComparatorImpl.class);
return (TestStoreEngine) StoreEngine.create(store, conf, kvComparator);
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
index 79b0a83..0285f14 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/compactions/TestStripeCompactionPolicy.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hbase.CellComparatorImpl;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.ColumnFamilyDescriptorBuilder;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -98,6 +99,8 @@ public class TestStripeCompactionPolicy {
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestStripeCompactionPolicy.class);
+ private static final RegionInfo FIRST_META_REGIONINFO =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
private static final byte[] KEY_A = Bytes.toBytes("aaa");
private static final byte[] KEY_B = Bytes.toBytes("bbb");
private static final byte[] KEY_C = Bytes.toBytes("ccc");
@@ -167,7 +170,7 @@ public class TestStripeCompactionPolicy {
conf.setInt(StripeStoreConfig.MAX_FILES_KEY, 4);
conf.setLong(StripeStoreConfig.SIZE_TO_SPLIT_KEY, 1000); // make sure the are no splits
StoreConfigInformation sci = mock(StoreConfigInformation.class);
- when(sci.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ when(sci.getRegionInfo()).thenReturn(FIRST_META_REGIONINFO);
StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
StripeCompactionPolicy policy = new StripeCompactionPolicy(conf, sci, ssc) {
@Override
@@ -484,7 +487,7 @@ public class TestStripeCompactionPolicy {
conf.setInt(StripeStoreConfig.INITIAL_STRIPE_COUNT_KEY, initialCount);
StoreConfigInformation sci = mock(StoreConfigInformation.class);
when(sci.getStoreFileTtl()).thenReturn(hasTtl ? defaultTtl : Long.MAX_VALUE);
- when(sci.getRegionInfo()).thenReturn(RegionInfoBuilder.FIRST_META_REGIONINFO);
+ when(sci.getRegionInfo()).thenReturn(FIRST_META_REGIONINFO);
StripeStoreConfig ssc = new StripeStoreConfig(conf, sci);
return new StripeCompactionPolicy(conf, sci, ssc);
}
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 ac89c92..02cb102 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
@@ -29,7 +29,6 @@ import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
-import org.apache.hadoop.hbase.TableDescriptors;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
@@ -158,9 +157,8 @@ public class TestLogRollingNoCluster {
this.log.info(getName() +" started");
final MultiVersionConcurrencyControl mvcc = new MultiVersionConcurrencyControl();
try {
- TableDescriptors tds = new FSTableDescriptors(TEST_UTIL.getConfiguration());
FSTableDescriptors.tryUpdateMetaTableDescriptor(TEST_UTIL.getConfiguration());
- TableDescriptor htd = tds.get(TableName.META_TABLE_NAME);
+ RegionInfo hri = RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
for (int i = 0; i < this.count; i++) {
long now = System.currentTimeMillis();
// Roll every ten edits
@@ -170,7 +168,6 @@ public class TestLogRollingNoCluster {
WALEdit edit = new WALEdit();
byte[] bytes = Bytes.toBytes(i);
edit.add(new KeyValue(bytes, bytes, bytes, now, EMPTY_1K_ARRAY));
- RegionInfo hri = RegionInfoBuilder.FIRST_META_REGIONINFO;
NavigableMap<byte[], Integer> scopes = new TreeMap<>(Bytes.BYTES_COMPARATOR);
for(byte[] fam: this.metaTableDescriptor.getColumnFamilyNames()) {
scopes.put(fam, 0);
@@ -198,8 +195,4 @@ public class TestLogRollingNoCluster {
}
}
}
-
- //@org.junit.Rule
- //public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
- // new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
}
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
index 3a6cfd4..c253392 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationWALEntryFilters.java
@@ -66,9 +66,9 @@ public class TestReplicationWALEntryFilters {
SystemTableWALEntryFilter filter = new SystemTableWALEntryFilter();
// meta
- WALKeyImpl key1 =
- new WALKeyImpl(RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes(),
- TableName.META_TABLE_NAME, System.currentTimeMillis());
+ WALKeyImpl key1 = new WALKeyImpl(
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build().getEncodedNameAsBytes(),
+ TableName.META_TABLE_NAME, System.currentTimeMillis());
Entry metaEntry = new Entry(key1, null);
assertNull(filter.filter(metaEntry));
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
index 24ad6ef..d70c409 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestFSUtils.java
@@ -28,7 +28,6 @@ import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Random;
-
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -42,6 +41,7 @@ import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.exceptions.DeserializationException;
import org.apache.hadoop.hbase.fs.HFileSystem;
@@ -230,8 +230,8 @@ public class TestFSUtils {
Path versionFile = new Path(rootdir, HConstants.VERSION_FILE_NAME);
assertTrue(CommonFSUtils.isExists(fs, versionFile));
assertTrue(CommonFSUtils.delete(fs, versionFile, true));
- Path metaRegionDir =
- FSUtils.getRegionDirFromRootDir(rootdir, RegionInfoBuilder.FIRST_META_REGIONINFO);
+ Path metaRegionDir = FSUtils.getRegionDirFromRootDir(rootdir,
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build());
FsPermission defaultPerms = CommonFSUtils.getFilePermissions(fs, this.conf,
HConstants.DATA_FILE_UMASK_KEY);
CommonFSUtils.create(fs, metaRegionDir, defaultPerms, false);
@@ -295,7 +295,7 @@ public class TestFSUtils {
assertEquals(new FsPermission("700"), filePerm);
// then that the correct file is created
- Path p = new Path("target" + File.separator + htu.getRandomUUID().toString());
+ Path p = new Path("target" + File.separator + HBaseTestingUtility.getRandomUUID().toString());
try {
FSDataOutputStream out = FSUtils.create(conf, fs, p, filePerm, null);
out.close();
@@ -314,7 +314,7 @@ public class TestFSUtils {
conf.setBoolean(HConstants.ENABLE_DATA_FILE_UMASK, true);
FsPermission perms = CommonFSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
// then that the correct file is created
- String file = htu.getRandomUUID().toString();
+ String file = HBaseTestingUtility.getRandomUUID().toString();
Path p = new Path(htu.getDataTestDir(), "temptarget" + File.separator + file);
Path p1 = new Path(htu.getDataTestDir(), "temppath" + File.separator + file);
try {
@@ -355,7 +355,7 @@ public class TestFSUtils {
FileSystem fs = FileSystem.get(conf);
Path testDir = htu.getDataTestDirOnTestFS("testArchiveFile");
- String file = htu.getRandomUUID().toString();
+ String file = HBaseTestingUtility.getRandomUUID().toString();
Path p = new Path(testDir, file);
FSDataOutputStream out = fs.create(p);
@@ -369,7 +369,7 @@ public class TestFSUtils {
mockEnv.setValue(expect);
EnvironmentEdgeManager.injectEdge(mockEnv);
try {
- String dstFile = htu.getRandomUUID().toString();
+ String dstFile = HBaseTestingUtility.getRandomUUID().toString();
Path dst = new Path(testDir , dstFile);
assertTrue(CommonFSUtils.renameAndSetModifyTime(fs, p, dst));
@@ -451,7 +451,7 @@ public class TestFSUtils {
conf.get(HConstants.WAL_STORAGE_POLICY, HConstants.DEFAULT_WAL_STORAGE_POLICY);
CommonFSUtils.setStoragePolicy(fs, testDir, storagePolicy);
- String file =htu.getRandomUUID().toString();
+ String file = HBaseTestingUtility.getRandomUUID().toString();
Path p = new Path(testDir, file);
WriteDataToHDFS(fs, p, 4096);
HFileSystem hfs = new HFileSystem(fs);
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
index 43cf81f..c9208e5 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/TestWALSplit.java
@@ -147,6 +147,8 @@ public class TestWALSplit {
private static String ROBBER;
private static String ZOMBIE;
private static String [] GROUP = new String [] {"supergroup"};
+ private static RegionInfo FIRST_META_REGIONINFO =
+ RegionInfoBuilder.newBuilder(TableName.META_TABLE_NAME).build();
static enum Corruptions {
INSERT_GARBAGE_ON_FIRST_LINE,
@@ -385,7 +387,7 @@ public class TestWALSplit {
public void testRecoveredEditsPathForMeta() throws IOException {
Path p = createRecoveredEditsPathForRegion();
String parentOfParent = p.getParent().getParent().getName();
- assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ assertEquals(parentOfParent, FIRST_META_REGIONINFO.getEncodedName());
}
/**
@@ -397,18 +399,18 @@ public class TestWALSplit {
Path p = createRecoveredEditsPathForRegion();
Path tdir = CommonFSUtils.getTableDir(HBASEDIR, TableName.META_TABLE_NAME);
Path regiondir = new Path(tdir,
- RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ FIRST_META_REGIONINFO.getEncodedName());
fs.mkdirs(regiondir);
Path parent = WALSplitUtil.getRegionDirRecoveredEditsDir(regiondir);
assertEquals(HConstants.RECOVERED_EDITS_DIR, parent.getName());
fs.createNewFile(parent); // create a recovered.edits file
String parentOfParent = p.getParent().getParent().getName();
- assertEquals(parentOfParent, RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedName());
+ assertEquals(parentOfParent, FIRST_META_REGIONINFO.getEncodedName());
WALFactory.createRecoveredEditsWriter(fs, p, conf).close();
}
private Path createRecoveredEditsPathForRegion() throws IOException {
- byte[] encoded = RegionInfoBuilder.FIRST_META_REGIONINFO.getEncodedNameAsBytes();
+ byte[] encoded = FIRST_META_REGIONINFO.getEncodedNameAsBytes();
long now = System.currentTimeMillis();
Entry entry = new Entry(
new WALKeyImpl(encoded, TableName.META_TABLE_NAME, 1, now, HConstants.DEFAULT_CLUSTER_ID),
@@ -422,10 +424,10 @@ public class TestWALSplit {
@Test
public void testHasRecoveredEdits() throws IOException {
Path p = createRecoveredEditsPathForRegion();
- assertFalse(WALSplitUtil.hasRecoveredEdits(conf, RegionInfoBuilder.FIRST_META_REGIONINFO));
+ assertFalse(WALSplitUtil.hasRecoveredEdits(conf, FIRST_META_REGIONINFO));
String renamedEdit = p.getName().split("-")[0];
fs.createNewFile(new Path(p.getParent(), renamedEdit));
- assertTrue(WALSplitUtil.hasRecoveredEdits(conf, RegionInfoBuilder.FIRST_META_REGIONINFO));
+ assertTrue(WALSplitUtil.hasRecoveredEdits(conf, FIRST_META_REGIONINFO));
}
private void useDifferentDFSClient() throws IOException {