HBASE-26285 Remove MetaTableLocator usages in non-migration code (#3686)

Signed-off-by: Yulin Niu <niuyulin@apache.org>
This commit is contained in:
Duo Zhang 2021-09-21 22:22:00 +08:00 committed by GitHub
parent 4614c7bc57
commit 4ea23c3baf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 111 additions and 199 deletions

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.client.MasterSwitchType;
import org.apache.hadoop.hbase.client.NormalizeTableFilterParams;
import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.client.TableDescriptor;
import org.apache.hadoop.hbase.client.TableState;
@ -64,6 +63,7 @@ import org.apache.hadoop.hbase.ipc.RpcServer;
import org.apache.hadoop.hbase.ipc.RpcServer.BlockingServiceAndInterface;
import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
import org.apache.hadoop.hbase.ipc.ServerRpcController;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.master.janitor.MetaFixer;
import org.apache.hadoop.hbase.master.locking.LockProcedure;
@ -114,7 +114,6 @@ import org.apache.hadoop.hbase.util.DNS.ServerType;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.ForeignExceptionUtil;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
@ -1752,17 +1751,13 @@ public class MasterRpcServices extends HBaseRpcServicesBase<HMaster>
LOG.warn("unassignRegion specifier type: expected: " + RegionSpecifierType.REGION_NAME
+ " actual: " + type);
}
Pair<RegionInfo, ServerName> pair =
MetaTableAccessor.getRegion(server.getConnection(), regionName);
if (Bytes.equals(RegionInfoBuilder.FIRST_META_REGIONINFO.getRegionName(), regionName)) {
pair = new Pair<>(RegionInfoBuilder.FIRST_META_REGIONINFO,
MetaTableLocator.getMetaRegionLocation(server.getZooKeeper()));
}
if (pair == null) {
RegionStateNode rsn =
server.getAssignmentManager().getRegionStates().getRegionStateNodeFromName(regionName);
if (rsn == null) {
throw new UnknownRegionException(Bytes.toString(regionName));
}
RegionInfo hri = pair.getFirst();
RegionInfo hri = rsn.getRegionInfo();
if (server.cpHost != null) {
server.cpHost.preUnassign(hri);
}

View File

@ -32,6 +32,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.CatalogFamilyFormat;
import org.apache.hadoop.hbase.DoNotRetryIOException;
@ -439,6 +440,39 @@ public class AssignmentManager {
return serverInfo.getRegionInfoList();
}
private RegionInfo getRegionInfo(RegionStateNode rsn) {
if (rsn.isSplit() && !rsn.getRegionInfo().isSplit()) {
// see the comments in markRegionAsSplit on why we need to do this converting.
return RegionInfoBuilder.newBuilder(rsn.getRegionInfo()).setSplit(true).setOffline(true)
.build();
} else {
return rsn.getRegionInfo();
}
}
private Stream<RegionStateNode> getRegionStateNodes(TableName tableName,
boolean excludeOfflinedSplitParents) {
Stream<RegionStateNode> stream = regionStates.getTableRegionStateNodes(tableName).stream();
if (excludeOfflinedSplitParents) {
return stream.filter(rsn -> !rsn.isSplit());
} else {
return stream;
}
}
public List<RegionInfo> getTableRegions(TableName tableName,
boolean excludeOfflinedSplitParents) {
return getRegionStateNodes(tableName, excludeOfflinedSplitParents)
.map(this::getRegionInfo).collect(Collectors.toList());
}
public List<Pair<RegionInfo, ServerName>> getTableRegionsAndLocations(TableName tableName,
boolean excludeOfflinedSplitParents) {
return getRegionStateNodes(tableName, excludeOfflinedSplitParents)
.map(rsn -> Pair.newPair(getRegionInfo(rsn), rsn.getRegionLocation()))
.collect(Collectors.toList());
}
public RegionStateStore getRegionStateStore() {
return regionStateStore;
}

View File

@ -164,6 +164,18 @@ public class RegionStateNode implements Comparable<RegionStateNode> {
return getProcedure() != null;
}
/**
* Return whether the region has been split and not online.
* <p/>
* In this method we will test both region info and state, and will return true if either of the
* test returns true. Please see the comments in
* {@link AssignmentManager#markRegionAsSplit(RegionInfo, ServerName, RegionInfo, RegionInfo)} for
* more details on why we need to test two conditions.
*/
public boolean isSplit() {
return regionInfo.isSplit() || isInState(State.SPLIT);
}
public long getLastUpdate() {
TransitRegionStateProcedure proc = this.procedure;
if (proc != null) {

View File

@ -139,7 +139,7 @@ public class RegionStates {
return node != null ? node : createRegionStateNode(regionInfo);
}
RegionStateNode getRegionStateNodeFromName(byte[] regionName) {
public RegionStateNode getRegionStateNodeFromName(byte[] regionName) {
return regionsMap.get(regionName);
}

View File

@ -27,11 +27,13 @@ import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.master.HMaster;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.tmpl.master.MasterStatusTmpl;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
/**
@ -82,7 +84,9 @@ public class MasterStatusServlet extends HttpServlet {
}
private ServerName getMetaLocationOrNull(HMaster master) {
return MetaTableLocator.getMetaRegionLocation(master.getZooKeeper());
RegionStateNode rsn = master.getAssignmentManager().getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
return rsn.isInState(RegionState.State.OPEN) ? rsn.getRegionLocation() : null;
}
private Map<String, Integer> getFragmentationInfo(

View File

@ -26,15 +26,18 @@ import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
import org.apache.hadoop.hbase.client.RegionInfo;
import org.apache.hadoop.hbase.client.RegionInfoBuilder;
import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.master.assignment.RegionStateNode;
import org.apache.hadoop.hbase.master.assignment.RegionStates;
import org.apache.hadoop.hbase.procedure2.Procedure;
import org.apache.hadoop.hbase.procedure2.ProcedureExecutor;
import org.apache.hadoop.hbase.quotas.MasterQuotaManager;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
@ -225,9 +228,17 @@ public final class ProcedureSyncWait {
protected static void waitMetaRegions(final MasterProcedureEnv env) throws IOException {
int timeout = env.getMasterConfiguration().getInt("hbase.client.catalog.timeout", 10000);
try {
if (MetaTableLocator.waitMetaRegionLocation(env.getMasterServices().getZooKeeper(),
timeout) == null) {
throw new NotAllMetaRegionsOnlineException();
long start = EnvironmentEdgeManager.currentTime();
for (;;) {
RegionStateNode rsn = env.getAssignmentManager().getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
if (rsn != null && rsn.isInState(RegionState.State.OPEN)) {
return;
}
if (EnvironmentEdgeManager.currentTime() - start >= timeout) {
throw new NotAllMetaRegionsOnlineException();
}
Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);
}
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException().initCause(e);

View File

@ -23,7 +23,6 @@ import java.util.Map;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
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.RegionReplicaUtil;
@ -36,7 +35,6 @@ import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.yetus.audience.InterfaceStability;
import org.slf4j.Logger;
@ -157,12 +155,7 @@ public final class MasterSnapshotVerifier {
* @throws IOException if we can't reach hbase:meta or read the files from the FS
*/
private void verifyRegions(final SnapshotManifest manifest) throws IOException {
List<RegionInfo> regions;
if (TableName.META_TABLE_NAME.equals(tableName)) {
regions = MetaTableLocator.getMetaRegions(services.getZooKeeper());
} else {
regions = MetaTableAccessor.getTableRegions(services.getConnection(), tableName);
}
List<RegionInfo> regions = services.getAssignmentManager().getTableRegions(tableName, false);
// Remove the non-default regions
RegionReplicaUtil.removeNonDefaultRegions(regions);

View File

@ -25,7 +25,6 @@ import java.util.concurrent.CancellationException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -49,7 +48,6 @@ import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
import org.apache.hadoop.hbase.util.CommonFSUtils;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@ -196,14 +194,8 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh
snapshotManifest.addTableDescriptor(this.htd);
monitor.rethrowException();
List<Pair<RegionInfo, ServerName>> regionsAndLocations;
if (TableName.META_TABLE_NAME.equals(snapshotTable)) {
regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
server.getZooKeeper());
} else {
regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
server.getConnection(), snapshotTable, false);
}
List<Pair<RegionInfo, ServerName>> regionsAndLocations =
master.getAssignmentManager().getTableRegionsAndLocations(snapshotTable, false);
// run the snapshot
snapshotRegions(regionsAndLocations);

View File

@ -27,7 +27,6 @@ import java.util.concurrent.ThreadPoolExecutor;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseInterfaceAudience;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.client.RegionInfo;
@ -44,7 +43,6 @@ import org.apache.hadoop.hbase.procedure.ZKProcedureCoordinator;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.access.AccessChecker;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
@ -130,15 +128,8 @@ public class MasterFlushTableProcedureManager extends MasterProcedureManager {
// It is possible that regions may move after we get the region server list.
// Each region server will get its own online regions for the table.
// We may still miss regions that need to be flushed.
List<Pair<RegionInfo, ServerName>> regionsAndLocations;
if (TableName.META_TABLE_NAME.equals(tableName)) {
regionsAndLocations = MetaTableLocator.getMetaRegionsAndLocations(
master.getZooKeeper());
} else {
regionsAndLocations = MetaTableAccessor.getTableRegionsAndLocations(
master.getConnection(), tableName, false);
}
List<Pair<RegionInfo, ServerName>> regionsAndLocations =
master.getAssignmentManager().getTableRegionsAndLocations(tableName, false);
Set<String> regionServers = new HashSet<>(regionsAndLocations.size());
for (Pair<RegionInfo, ServerName> region : regionsAndLocations) {

View File

@ -58,6 +58,7 @@
import="org.apache.hadoop.hbase.http.InfoServer"
import="org.apache.hadoop.hbase.master.HMaster"
import="org.apache.hadoop.hbase.master.RegionState"
import="org.apache.hadoop.hbase.master.assignment.RegionStateNode"
import="org.apache.hadoop.hbase.master.assignment.RegionStates"
import="org.apache.hadoop.hbase.master.http.MetaBrowser"
import="org.apache.hadoop.hbase.master.http.RegionReplicaInfo"
@ -315,13 +316,9 @@
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, j);
//If a metaLocation is null, All of its info would be empty here to be displayed.
ServerName metaLocation = null;
try {
metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
} catch (NotAllMetaRegionsOnlineException e) {
//Region in transition state here throw a NotAllMetaRegionsOnlineException causes
//the UI crash.
}
RegionStateNode rsn = master.getAssignmentManager().getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
ServerName metaLocation = rsn != null ? rsn.getRegionLocation() : null;
for (int i = 0; i < 1; i++) {
//If metaLocation is null, default value below would be displayed in UI.
String hostAndPort = "";
@ -388,13 +385,9 @@
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, j);
//If a metaLocation is null, All of its info would be empty here to be displayed.
ServerName metaLocation = null;
try {
metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
} catch (NotAllMetaRegionsOnlineException e) {
//Region in transition state here throw a NotAllMetaRegionsOnlineException causes
//the UI crash.
}
RegionStateNode rsn = master.getAssignmentManager().getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
ServerName metaLocation = rsn != null ? rsn.getRegionLocation() : null;
for (int i = 0; i < 1; i++) {
//If metaLocation is null, default value below would be displayed in UI.
String hostAndPort = "";
@ -444,13 +437,9 @@
RegionInfo meta = RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, j);
//If a metaLocation is null, All of its info would be empty here to be displayed.
ServerName metaLocation = null;
try {
metaLocation = MetaTableLocator.waitMetaRegionLocation(master.getZooKeeper(), j, 1);
} catch (NotAllMetaRegionsOnlineException e) {
//Region in transition state here throw a NotAllMetaRegionsOnlineException causes
//the UI crash.
}
RegionStateNode rsn = master.getAssignmentManager().getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO);
ServerName metaLocation = rsn != null ? rsn.getRegionLocation() : null;
for (int i = 0; i < 1; i++) {
//If metaLocation is null, default value below would be displayed in UI.
String hostAndPort = "";

View File

@ -54,7 +54,6 @@ import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.ClassRule;
@ -242,13 +241,6 @@ public class TestMetaTableAccessor {
abstract void metaTask() throws Throwable;
}
@Test
public void testGetRegionsFromMetaTable() throws IOException, InterruptedException {
List<RegionInfo> regions = MetaTableLocator.getMetaRegions(UTIL.getZooKeeperWatcher());
assertTrue(regions.size() >= 1);
assertTrue(MetaTableLocator.getMetaRegionsAndLocations(UTIL.getZooKeeperWatcher()).size() >= 1);
}
@Test
public void testGetRegion() throws IOException, InterruptedException {
final String name = this.name.getMethodName();

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.hbase.master.assignment.AssignmentManager;
import org.apache.hadoop.hbase.master.assignment.AssignmentTestingUtil;
import org.apache.hadoop.hbase.regionserver.StorefileRefresherChore;
import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker;
import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
import org.junit.AfterClass;
import org.junit.Rule;
import org.slf4j.Logger;
@ -65,8 +64,8 @@ public class MetaWithReplicasTestBase {
HBaseTestingUtil.setReplicas(admin, TableName.META_TABLE_NAME, 3);
AssignmentManager am = TEST_UTIL.getMiniHBaseCluster().getMaster().getAssignmentManager();
Set<ServerName> sns = new HashSet<ServerName>();
ServerName hbaseMetaServerName =
MetaTableLocator.getMetaRegionLocation(TEST_UTIL.getZooKeeperWatcher());
ServerName hbaseMetaServerName = am.getRegionStates()
.getRegionStateNode(RegionInfoBuilder.FIRST_META_REGIONINFO).getRegionLocation();
LOG.info("HBASE:META DEPLOY: on " + hbaseMetaServerName);
sns.add(hbaseMetaServerName);
for (int replicaId = 1; replicaId < 3; replicaId++) {

View File

@ -189,6 +189,8 @@ public class TestTableSnapshotScanner {
verifyScanner(scanner, bbb, yyy);
scanner.close();
} catch (Exception e) {
e.printStackTrace();
} finally {
UTIL.getAdmin().deleteSnapshot(snapshotName);
UTIL.deleteTable(tableName);

View File

@ -17,24 +17,19 @@
*/
package org.apache.hadoop.hbase.zookeeper;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import com.google.errorprone.annotations.RestrictedApi;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.NotAllMetaRegionsOnlineException;
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.exceptions.DeserializationException;
import org.apache.hadoop.hbase.master.RegionState;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.yetus.audience.InterfaceAudience;
import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.shaded.protobuf.generated.ZooKeeperProtos.MetaRegionServer;
@ -57,70 +52,13 @@ public final class MetaTableLocator {
private MetaTableLocator() {
}
/**
* @param zkw ZooKeeper watcher to be used
* @return meta table regions and their locations.
*/
public static List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw) {
return getMetaRegionsAndLocations(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
* Gets the meta regions and their locations for the given path and replica ID.
*
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica
* @return meta table regions and their locations.
*/
public static List<Pair<RegionInfo, ServerName>> getMetaRegionsAndLocations(ZKWatcher zkw,
int replicaId) {
ServerName serverName = getMetaRegionLocation(zkw, replicaId);
List<Pair<RegionInfo, ServerName>> list = new ArrayList<>(1);
list.add(new Pair<>(RegionReplicaUtil.getRegionInfoForReplica(
RegionInfoBuilder.FIRST_META_REGIONINFO, replicaId), serverName));
return list;
}
/**
* Gets the meta regions for the given path with the default replica ID.
*
* @param zkw ZooKeeper watcher to be used
* @return List of meta regions
*/
public static List<RegionInfo> getMetaRegions(ZKWatcher zkw) {
return getMetaRegions(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
/**
* Gets the meta regions for the given path and replica ID.
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param replicaId the ID of the replica
* @return List of meta regions
*/
public static List<RegionInfo> getMetaRegions(ZKWatcher zkw, int replicaId) {
List<Pair<RegionInfo, ServerName>> result;
result = getMetaRegionsAndLocations(zkw, replicaId);
return getListOfRegionInfos(result);
}
private static List<RegionInfo> getListOfRegionInfos(
final List<Pair<RegionInfo, ServerName>> pairs) {
if (pairs == null || pairs.isEmpty()) {
return Collections.emptyList();
}
List<RegionInfo> result = new ArrayList<>(pairs.size());
for (Pair<RegionInfo, ServerName> pair : pairs) {
result.add(pair.getFirst());
}
return result;
}
/**
* Gets the meta region location, if available. Does not block.
* @param zkw zookeeper connection to use
* @return server name or null if we failed to get the data.
*/
@RestrictedApi(explanation = "Should only be called in tests or ZKUtil", link = "",
allowedOnPath = ".*/src/test/.*|.*/ZKUtil\\.java")
public static ServerName getMetaRegionLocation(final ZKWatcher zkw) {
try {
RegionState state = getMetaRegionState(zkw);
@ -136,6 +74,8 @@ public final class MetaTableLocator {
* @param replicaId the ID of the replica
* @return server name
*/
@RestrictedApi(explanation = "Should only be called in self or ZKUtil", link = "",
allowedOnPath = ".*(MetaTableLocator|ZKUtil)\\.java")
public static ServerName getMetaRegionLocation(final ZKWatcher zkw, int replicaId) {
try {
RegionState state = getMetaRegionState(zkw, replicaId);
@ -156,6 +96,8 @@ public final class MetaTableLocator {
* @throws InterruptedException if interrupted while waiting
* @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
*/
@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public static ServerName waitMetaRegionLocation(ZKWatcher zkw, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
return waitMetaRegionLocation(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
@ -173,7 +115,7 @@ public final class MetaTableLocator {
* @throws InterruptedException if waiting for the socket operation fails
* @throws NotAllMetaRegionsOnlineException if a meta or root region is not online
*/
public static ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
private static ServerName waitMetaRegionLocation(ZKWatcher zkw, int replicaId, long timeout)
throws InterruptedException, NotAllMetaRegionsOnlineException {
try {
if (ZKUtil.checkExists(zkw, zkw.getZNodePaths().baseZNode) == -1) {
@ -202,6 +144,8 @@ public final class MetaTableLocator {
* @param state The region transition state
* @throws KeeperException unexpected zookeeper exception
*/
@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public static void setMetaLocation(ZKWatcher zookeeper,
ServerName serverName, RegionState.State state) throws KeeperException {
setMetaLocation(zookeeper, serverName, RegionInfo.DEFAULT_REPLICA_ID, state);
@ -249,6 +193,8 @@ public final class MetaTableLocator {
/**
* Load the meta region state from the meta server ZNode.
*/
@RestrictedApi(explanation = "Should only be called in self or tests", link = "",
allowedOnPath = ".*/src/test/.*|.*/MetaTableLocator\\.java")
public static RegionState getMetaRegionState(ZKWatcher zkw) throws KeeperException {
return getMetaRegionState(zkw, RegionInfo.DEFAULT_REPLICA_ID);
}
@ -280,6 +226,8 @@ public final class MetaTableLocator {
* @param zookeeper zookeeper reference
* @throws KeeperException unexpected zookeeper exception
*/
@RestrictedApi(explanation = "Should only be called in tests", link = "",
allowedOnPath = ".*/src/test/.*")
public static void deleteMetaLocation(ZKWatcher zookeeper)
throws KeeperException {
deleteMetaLocation(zookeeper, RegionInfo.DEFAULT_REPLICA_ID);
@ -299,55 +247,6 @@ public final class MetaTableLocator {
// Has already been deleted
}
}
/**
* Wait until the primary meta region is available. Get the secondary locations as well but don't
* block for those.
*
* @param zkw reference to the {@link ZKWatcher} which also contains configuration and operation
* @param timeout maximum time to wait in millis
* @param conf the {@link Configuration} to use
* @return ServerName or null if we timed out.
* @throws InterruptedException if waiting for the socket operation fails
*/
public static List<ServerName> blockUntilAvailable(final ZKWatcher zkw, final long timeout,
Configuration conf) throws InterruptedException {
int numReplicasConfigured = 1;
List<ServerName> servers = new ArrayList<>();
// Make the blocking call first so that we do the wait to know
// the znodes are all in place or timeout.
ServerName server = blockUntilAvailable(zkw, timeout);
if (server == null) {
return null;
}
servers.add(server);
try {
List<String> metaReplicaNodes = zkw.getMetaReplicaNodes();
numReplicasConfigured = metaReplicaNodes.size();
} catch (KeeperException e) {
LOG.warn("Got ZK exception {}", e);
}
for (int replicaId = 1; replicaId < numReplicasConfigured; replicaId++) {
// return all replica locations for the meta
servers.add(getMetaRegionLocation(zkw, replicaId));
}
return servers;
}
/**
* Wait until the meta region is available and is not in transition.
* @param zkw zookeeper connection to use
* @param timeout maximum time to wait, in millis
* @return ServerName or null if we timed out.
* @throws InterruptedException if waiting for the socket operation fails
*/
public static ServerName blockUntilAvailable(final ZKWatcher zkw, final long timeout)
throws InterruptedException {
return blockUntilAvailable(zkw, RegionInfo.DEFAULT_REPLICA_ID, timeout);
}
/**
* Wait until the meta region is available and is not in transition.
@ -357,8 +256,8 @@ public final class MetaTableLocator {
* @return ServerName or null if we timed out.
* @throws InterruptedException if waiting for the socket operation fails
*/
public static ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId,
final long timeout) throws InterruptedException {
private static ServerName blockUntilAvailable(final ZKWatcher zkw, int replicaId,
final long timeout) throws InterruptedException {
if (timeout < 0) {
throw new IllegalArgumentException();
}
@ -371,9 +270,8 @@ public final class MetaTableLocator {
ServerName sn = null;
while (true) {
sn = getMetaRegionLocation(zkw, replicaId);
if (sn != null ||
(EnvironmentEdgeManager.currentTime() - startTime) >
timeout - HConstants.SOCKET_RETRY_WAIT_MS) {
if (sn != null || (EnvironmentEdgeManager.currentTime() - startTime) > timeout -
HConstants.SOCKET_RETRY_WAIT_MS) {
break;
}
Thread.sleep(HConstants.SOCKET_RETRY_WAIT_MS);