Revert "HBASE-16488 Starting namespace and quota services in master startup asynchronizely (Stephen Yuan Jiang)"

This reverts commit af359d03b5.
This commit is contained in:
Andrew Purtell 2017-09-13 13:24:58 -07:00
parent a8ac84f1a6
commit 3f785a83b2
19 changed files with 51 additions and 233 deletions

View File

@ -156,7 +156,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.ConfigUtil;
import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.hbase.util.HFileArchiveUtil;
import org.apache.hadoop.hbase.util.HasThread;
@ -310,10 +309,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// initialization may have not completed yet.
volatile boolean serviceStarted = false;
// flag set after we complete asynchorized services and master initialization is done,
private final ProcedureEvent namespaceManagerInitialized =
new ProcedureEvent("master namespace manager initialized");
// flag set after we complete assignMeta.
private final ProcedureEvent serverCrashProcessingEnabled =
new ProcedureEvent("server crash processing");
@ -882,8 +877,8 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
periodicDoMetricsChore = new PeriodicDoMetrics(msgInterval, this);
getChoreService().scheduleChore(periodicDoMetricsChore);
status.setStatus("Starting namespace manager and quota manager");
initNamespaceAndQuotaManager();
status.setStatus("Starting namespace manager");
initNamespace();
if (this.cpHost != null) {
try {
@ -902,6 +897,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
// Set master as 'initialized'.
setInitialized(true);
assignmentManager.checkIfShouldMoveSystemRegionAsync();
status.setStatus("Starting quota manager");
initQuotaManager();
// assign the meta replicas
Set<ServerName> EMPTY_SET = new HashSet<ServerName>();
int numReplicas = conf.getInt(HConstants.META_REPLICAS_NUM,
@ -933,6 +933,12 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
zombieDetector.interrupt();
}
private void initQuotaManager() throws IOException {
quotaManager = new MasterQuotaManager(this);
this.assignmentManager.setRegionStateListener((RegionStateListener) quotaManager);
quotaManager.start();
}
/**
* Create a {@link ServerManager} instance.
* @param master
@ -1079,60 +1085,10 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
}
}
/*
* The main purpose is to start namespace manager and quota manager async to
* unblock overall master initialization
*
* @throws IOException
*/
private void initNamespaceAndQuotaManager() throws IOException {
void initNamespace() throws IOException {
//create namespace manager
tableNamespaceManager = new TableNamespaceManager(this);
//create quota manager
this.quotaManager = new MasterQuotaManager(this);
this.assignmentManager.setRegionStateListener((RegionStateListener)quotaManager);
if (conf.getBoolean("hbase.master.start.wait.for.namespacemanager", false)) {
// If being asked not to async start namespace manager, then just block
// master service starting until namespace manager is ready.
//
// Note: Quota manager depends on namespace manager. Therefore, its starting
// method has to be in-sync with namespace manager.
LOG.info("Starting namespace manager and quota manager synchronously");
tableNamespaceManager.start();
setNamespaceManagerInitializedEvent(true);
LOG.info("Namespace manager started successfully.");
quotaManager.start();
LOG.info("Quota manager started successfully.");
} else { // else asynchronously start namespace manager and quota manager
LOG.info("Starting namespace manager and quota manager asynchronously");
Threads.setDaemonThreadRunning(new Thread(new Runnable() {
@Override
public void run() {
// Start namespace manager and wait to it to be fully started.
try {
tableNamespaceManager.start();
setNamespaceManagerInitializedEvent(true);
LOG.info("Namespace manager started successfully.");
} catch (IOException e) {
LOG.error("Namespace manager failed to start. ", e);
abort("Shutdown Master due to namespace manager failed to start. ", e);
}
// Quota Manager depends on Namespace manager to be fully initialized.
try {
quotaManager.start();
LOG.info("Quota manager started successfully.");
} catch (IOException ie) {
LOG.error("Quota Manager failed to start. ", ie);
abort("Shutdown Master due to Quota Manager failure to start. ", ie);
}
}
}, "Init Namespace Manager and Quota Manager Async"));
}
assignmentManager.checkIfShouldMoveSystemRegionAsync();
tableNamespaceManager.start();
}
boolean isCatalogJanitorEnabled() {
@ -2671,26 +2627,11 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
void checkNamespaceManagerReady() throws IOException {
checkInitialized();
if (tableNamespaceManager == null) {
if (tableNamespaceManager == null ||
!tableNamespaceManager.isTableAvailableAndInitialized(true)) {
throw new IOException("Table Namespace Manager not ready yet, try again later");
} else if (!tableNamespaceManager.isTableAvailableAndInitialized(true)) {
try {
// Wait some time.
long startTime = EnvironmentEdgeManager.currentTime();
int timeout = conf.getInt("hbase.master.namespace.wait.for.ready", 30000);
while (!tableNamespaceManager.isTableNamespaceManagerStarted() &&
EnvironmentEdgeManager.currentTime() - startTime < timeout) {
Thread.sleep(100);
}
} catch (InterruptedException e) {
throw (InterruptedIOException) new InterruptedIOException().initCause(e);
}
if (!tableNamespaceManager.isTableNamespaceManagerStarted()) {
throw new IOException("Table Namespace Manager not fully initialized, try again later");
}
}
}
/**
* Report whether this master is currently the active master or not.
* If not active master, we are parked on ZK waiting to become active.
@ -2736,16 +2677,6 @@ public class HMaster extends HRegionServer implements MasterServices, Server {
return initialized;
}
public void setNamespaceManagerInitializedEvent(boolean isNamespaceManagerInitialized) {
procedureExecutor.getEnvironment().setEventReady(
namespaceManagerInitialized,
isNamespaceManagerInitialized);
}
public ProcedureEvent getNamespaceManagerInitializedEvent() {
return namespaceManagerInitialized;
}
/**
* ServerCrashProcessingEnabled is set false before completing assignMeta to prevent processing
* of crashed servers.

View File

@ -1303,7 +1303,7 @@ public class MasterRpcServices extends RSRpcServices
public RestoreSnapshotResponse restoreSnapshot(RpcController controller,
RestoreSnapshotRequest request) throws ServiceException {
try {
master.checkNamespaceManagerReady();
master.checkInitialized();
master.snapshotManager.checkSnapshotSupport();
// ensure namespace exists
@ -1606,7 +1606,7 @@ public class MasterRpcServices extends RSRpcServices
@Override
public SetQuotaResponse setQuota(RpcController c, SetQuotaRequest req) throws ServiceException {
try {
master.checkNamespaceManagerReady();
master.checkInitialized();
return master.getMasterQuotaManager().setQuota(req);
} catch (Exception e) {
throw new ServiceException(e);

View File

@ -48,7 +48,6 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.Sets;
/**
@ -73,8 +72,7 @@ public class TableNamespaceManager {
public static final String KEY_MAX_REGIONS = "hbase.namespace.quota.maxregions";
public static final String KEY_MAX_TABLES = "hbase.namespace.quota.maxtables";
static final String NS_INIT_TIMEOUT = "hbase.master.namespace.init.timeout";
static final int DEFAULT_NS_INIT_TIMEOUT = 1800000; // default is 30 minutes
private static final int WAIT_MESSAGE_TO_PRINTOUT = 300000; // print out message every 5 minutes
static final int DEFAULT_NS_INIT_TIMEOUT = 300000;
public TableNamespaceManager(MasterServices masterServices) {
this.masterServices = masterServices;
@ -93,18 +91,12 @@ public class TableNamespaceManager {
// If timed out, we will move ahead without initializing it.
// So that it should be initialized later on lazily.
long startTime = EnvironmentEdgeManager.currentTime();
long msgCount = 0;
long waitTime;
int timeout = conf.getInt(NS_INIT_TIMEOUT, DEFAULT_NS_INIT_TIMEOUT);
while (!isTableAvailableAndInitialized(false)) {
waitTime = EnvironmentEdgeManager.currentTime() - startTime;
if (waitTime > timeout) {
if (EnvironmentEdgeManager.currentTime() - startTime + 100 > timeout) {
// We can't do anything if ns is not online.
throw new IOException("Timedout " + timeout + "ms waiting for namespace table to " +
"be assigned");
} else if (waitTime > msgCount * WAIT_MESSAGE_TO_PRINTOUT) {
LOG.info("Waiting for namespace table to be online. Time waited = " + waitTime + " ms.");
msgCount++;
}
Thread.sleep(100);
}
@ -113,11 +105,6 @@ public class TableNamespaceManager {
}
}
@VisibleForTesting
public boolean isTableNamespaceManagerStarted() {
return initialized;
}
private synchronized Table getNamespaceTable() throws IOException {
if (!isTableNamespaceManagerInitialized()) {
throw new IOException(this.getClass().getName() + " isn't ready to serve");

View File

@ -193,20 +193,21 @@ public class CreateNamespaceProcedure
sb.append(")");
}
private static boolean isBootstrapNamespace(final NamespaceDescriptor nsDescriptor) {
private boolean isBootstrapNamespace() {
return nsDescriptor.equals(NamespaceDescriptor.DEFAULT_NAMESPACE) ||
nsDescriptor.equals(NamespaceDescriptor.SYSTEM_NAMESPACE);
}
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
// Namespace manager might not be ready if master is not fully initialized,
// return false to reject user namespace creation; return true for default
// and system namespace creation (this is part of master initialization).
if (!isBootstrapNamespace(nsDescriptor) && env.waitNamespaceManagerInitialized(this)) {
return false;
if (!env.getMasterServices().isInitialized()) {
// Namespace manager might not be ready if master is not fully initialized,
// return false to reject user namespace creation; return true for default
// and system namespace creation (this is part of master initialization).
if (!isBootstrapNamespace() && env.waitInitialized(this)) {
return false;
}
}
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
}
@ -330,7 +331,7 @@ public class CreateNamespaceProcedure
protected static void setNamespaceQuota(
final MasterProcedureEnv env,
final NamespaceDescriptor nsDescriptor) throws IOException {
if (env.getMasterServices().isInitialized() && !isBootstrapNamespace(nsDescriptor)) {
if (env.getMasterServices().isInitialized()) {
env.getMasterServices().getMasterQuotaManager().setNamespaceQuota(nsDescriptor);
}
}
@ -369,6 +370,6 @@ public class CreateNamespaceProcedure
protected boolean shouldWaitClientAck(MasterProcedureEnv env) {
// hbase and default namespaces are created on bootstrap internally by the system
// the client does not know about this procedures.
return !isBootstrapNamespace(nsDescriptor);
return !isBootstrapNamespace();
}
}

View File

@ -269,7 +269,7 @@ public class CreateTableProcedure
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (!getTableName().isSystemTable() && env.waitNamespaceManagerInitialized(this)) {
if (!getTableName().isSystemTable() && env.waitInitialized(this)) {
return false;
}
return env.getProcedureQueue().tryAcquireTableExclusiveLock(this, getTableName());

View File

@ -210,9 +210,7 @@ public class DeleteNamespaceProcedure
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (env.waitNamespaceManagerInitialized(this)) {
return false;
}
if (env.waitInitialized(this)) return false;
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
}

View File

@ -132,10 +132,6 @@ public class MasterProcedureEnv {
return procSched.waitEvent(((HMaster)master).getInitializedEvent(), proc);
}
public boolean waitNamespaceManagerInitialized(Procedure proc) {
return procSched.waitEvent(((HMaster)master).getNamespaceManagerInitializedEvent(), proc);
}
public boolean waitServerCrashProcessingEnabled(Procedure proc) {
return procSched.waitEvent(((HMaster)master).getServerCrashProcessingEnabledEvent(), proc);
}

View File

@ -191,9 +191,7 @@ public class ModifyNamespaceProcedure
@Override
protected boolean acquireLock(final MasterProcedureEnv env) {
if (env.waitNamespaceManagerInitialized(this)) {
return false;
}
if (env.waitInitialized(this)) return false;
return env.getProcedureQueue().tryAcquireNamespaceExclusiveLock(this, getNamespaceName());
}

View File

@ -222,7 +222,7 @@ public class TestRollbackFromClient {
HColumnDescriptor col = new HColumnDescriptor(FAMILY);
col.setMaxVersions(versions);
desc.addFamily(col);
TEST_UTIL.createTable(tableName, FAMILY, versions);
TEST_UTIL.getHBaseAdmin().createTable(desc);
int expected;
List<Cell> cells;
try (Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());

View File

@ -508,9 +508,6 @@ public class TestMasterFailover {
HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
Configuration conf = TEST_UTIL.getConfiguration();
conf.setBoolean("hbase.assignment.usezk", true);
// The test depends on namespace region is online, therefore, we have to
// wait for namespace manager starting.
conf.setBoolean("hbase.master.start.wait.for.namespacemanager", true);
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 2);

View File

@ -66,9 +66,6 @@ public class TestMasterMetrics {
public static void startCluster() throws Exception {
LOG.info("Starting cluster");
TEST_UTIL = new HBaseTestingUtility();
// The metrics depends on namespace region is online, therefore, we have to
// wait for namespace manager starting.
TEST_UTIL.getConfiguration().setBoolean("hbase.master.start.wait.for.namespacemanager", true);
TEST_UTIL.startMiniCluster(1, 1, 1, null, MyMaster.class, null);
cluster = TEST_UTIL.getHBaseCluster();
LOG.info("Waiting for active/ready master");

View File

@ -215,6 +215,10 @@ public class TestMasterNoCluster {
public ClusterConnection getConnection() {
return mockedConnection;
}
@Override
void initNamespace() {
}
};
master.start();
@ -292,6 +296,10 @@ public class TestMasterNoCluster {
return null;
}
}
@Override
void initNamespace() {
}
};
master.start();

View File

@ -54,10 +54,10 @@ public class TestCreateTableHandler {
private static final Log LOG = LogFactory.getLog(TestCreateTableHandler.class);
private static final byte[] FAMILYNAME = Bytes.toBytes("fam");
private static boolean throwException = false;
@Before
public void setUp() throws Exception {
TEST_UTIL.getConfiguration().setBoolean("hbase.master.start.wait.for.namespacemanager", true);
TEST_UTIL.startMiniCluster(1);
}

View File

@ -52,10 +52,6 @@ public class TestCreateNamespaceProcedure {
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
// We have to disable the async namespace manager startup due to the recovery and rollback
// tests would halt the execution of procedure. If at that time the namespace table is
// not created, it would be forced to stop/wait and cause the test to fail.
conf.setBoolean("hbase.master.start.wait.for.namespacemanager", true);
}
@BeforeClass

View File

@ -35,7 +35,6 @@ import org.apache.hadoop.hbase.procedure2.store.wal.WALProcedureStore;
import org.apache.hadoop.hbase.regionserver.HRegionServer;
import org.apache.hadoop.hbase.testclassification.MasterTests;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
@ -44,8 +43,6 @@ import org.junit.experimental.categories.Category;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
@Category({MasterTests.class, MediumTests.class})
public class TestMasterProcedureEvents {
private static final Log LOG = LogFactory.getLog(TestCreateTableProcedure.class);
@ -79,75 +76,13 @@ public class TestMasterProcedureEvents {
ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
MasterProcedureScheduler procSched = procExec.getEnvironment().getProcedureQueue();
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor hcd = new HColumnDescriptor("f");
htd.addFamily(hcd);
while (!master.isInitialized()) Thread.sleep(250);
UTIL.createTable(htd, null);
// Modify the table descriptor
HTableDescriptor newHTD =
new HTableDescriptor(UTIL.getHBaseAdmin().getTableDescriptor(tableName));
long newMaxFileSize = newHTD.getMaxFileSize() * 2;
newHTD.setMaxFileSize(newMaxFileSize);
newHTD.setRegionReplication(3);
master.setInitialized(false); // fake it, set back later
ModifyTableProcedure proc = new ModifyTableProcedure(procExec.getEnvironment(), newHTD);
long pollCalls = procSched.getPollCalls();
long nullPollCalls = procSched.getNullPollCalls();
long procId = procExec.submitProcedure(proc);
for (int i = 0; i < 10; ++i) {
Thread.sleep(100);
assertEquals(pollCalls + 1, procSched.getPollCalls());
assertEquals(nullPollCalls, procSched.getNullPollCalls());
}
master.setInitialized(true);
ProcedureTestingUtility.waitProcedure(procExec, procId);
assertEquals(pollCalls + 2, procSched.getPollCalls());
assertEquals(nullPollCalls, procSched.getNullPollCalls());
}
@Test
public void testNamespaceManagerInitializedEvent() throws Exception {
TableName tableName = TableName.valueOf("testNamespaceManagerInitializedEvent");
HMaster master = UTIL.getMiniHBaseCluster().getMaster();
ProcedureExecutor<MasterProcedureEnv> procExec = master.getMasterProcedureExecutor();
MasterProcedureScheduler procSched = procExec.getEnvironment().getProcedureQueue();
HRegionInfo hri = new HRegionInfo(tableName);
HTableDescriptor htd = new HTableDescriptor(tableName);
HColumnDescriptor hcd = new HColumnDescriptor("f");
htd.addFamily(hcd);
while (!master.isInitialized()) Thread.sleep(250);
final int maxwait = 60000;
final long startTime = EnvironmentEdgeManager.currentTime();
do {
if(master.getTableNamespaceManager().isTableNamespaceManagerStarted()) {
break;
}
try {
Thread.sleep(100);
} catch (InterruptedException e) {
throw new IOException("Interrupt while waiting for master namespace manager starting.");
}
} while (EnvironmentEdgeManager.currentTime() - startTime < maxwait);
if(!master.getTableNamespaceManager().isTableNamespaceManagerStarted()) {
throw new IOException(
"Cannot continue testing due to master namespace manager not started after waiting " +
(EnvironmentEdgeManager.currentTime() - startTime) + " milliseconds");
}
master.setNamespaceManagerInitializedEvent(false); // fake it, set back later
master.setInitialized(false); // fake it, set back later
CreateTableProcedure proc = new CreateTableProcedure(
procExec.getEnvironment(), htd, new HRegionInfo[] { hri });
@ -162,7 +97,7 @@ public class TestMasterProcedureEvents {
assertEquals(nullPollCalls, procSched.getNullPollCalls());
}
master.setNamespaceManagerInitializedEvent(true);
master.setInitialized(true);
ProcedureTestingUtility.waitProcedure(procExec, procId);
assertEquals(pollCalls + 2, procSched.getPollCalls());

View File

@ -48,10 +48,6 @@ public class TestModifyNamespaceProcedure {
private static void setupConf(Configuration conf) {
conf.setInt(MasterProcedureConstants.MASTER_PROCEDURE_THREADS, 1);
// We have to disable the async namespace manager startup due to the recovery and rollback
// tests would halt the execution of procedure. If at that time the namespace table is
// not created, it would be forced to stop/wait and cause the test to fail.
conf.setBoolean("hbase.master.start.wait.for.namespacemanager", true);
}
@BeforeClass

View File

@ -18,10 +18,12 @@
*/
package org.apache.hadoop.hbase.regionserver;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
@ -32,6 +34,7 @@ import org.apache.hadoop.hbase.CategoryBasedTimeout;
import org.apache.hadoop.hbase.CoordinatedStateManager;
import org.apache.hadoop.hbase.HBaseConfiguration;
import org.apache.hadoop.hbase.HBaseTestingUtility;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.LocalHBaseCluster;
import org.apache.hadoop.hbase.MiniHBaseCluster;
import org.apache.hadoop.hbase.ServerName;
@ -41,6 +44,7 @@ import org.apache.hadoop.hbase.master.ServerManager;
import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
import org.apache.hadoop.hbase.testclassification.MediumTests;
import org.apache.hadoop.hbase.testclassification.RegionServerTests;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.JVMClusterUtil.MasterThread;
import org.apache.hadoop.hbase.util.Threads;
import org.junit.Rule;
@ -80,7 +84,6 @@ public class TestRSKilledWhenInitializing {
// Create config to use for this cluster
Configuration conf = HBaseConfiguration.create();
conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
conf.setBoolean("hbase.master.start.wait.for.namespacemanager", false);
// Start the cluster
final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(conf);
TEST_UTIL.startMiniDFSCluster(3);
@ -97,46 +100,24 @@ public class TestRSKilledWhenInitializing {
}
// Now wait on master to see NUM_RS + 1 servers as being online, thats NUM_RS plus
// the Master itself (because Master hosts hbase:meta and checks in as though it a RS).
ServerManager serverManager = master.getMaster().getServerManager();
List<ServerName> onlineServersList = null;
do {
onlineServersList = master.getMaster().getServerManager().getOnlineServersList();
} while (onlineServersList.size() < NUM_RS);
LOG.info("!!!Print online Server list: " + onlineServersList);
// Wait until killedRS is set. Means RegionServer is starting to go down.
while (killedRS.get() == null) {
Threads.sleep(1);
}
LOG.info("!!!Print dead server: " + killedRS);
LOG.info("!!!Print online Server list: " + serverManager.getOnlineServersList());
// Wait on the RegionServer to fully die.
while (cluster.getLiveRegionServers().size() >= NUM_RS) {
Threads.sleep(1);
}
LOG.info("!!!Print live server size: " + cluster.getLiveRegionServers().size());
LOG.info("!!!Print online Server list after kill: " + serverManager.getOnlineServersList());
LOG.info("!!!Print is server " + killedRS.get() + " online: " + serverManager.isServerOnline(killedRS.get()));
// Make sure Master is fully up before progressing. Could take a while if regions
// being reassigned.
while (!master.getMaster().isInitialized()) {
Threads.sleep(1);
}
int i=0;
while (i < 100000) {
if (!serverManager.isServerOnline(killedRS.get())) {
break;
}
Threads.sleep(100);
i +=100;
}
LOG.info("!!!Print online Server list after kill: " + serverManager.getOnlineServersList());
LOG.info("!!!Print wait: " + i + " ms.");
LOG.info("!!!Print is server " + killedRS.get() + " online: " + serverManager.isServerOnline(killedRS.get()));
// Now in steady state. Make sure the killed RS is no longer registered.
// branch-1 works differently to master branch.
assertTrue(!master.getMaster().getServerManager().isServerOnline(killedRS.get()));

View File

@ -66,9 +66,6 @@ public class TestRegionOpen {
@BeforeClass
public static void before() throws Exception {
// This test depends on namespace region open; therefore, we have to wait for namespace
// manager start before continue.
HTU.getConfiguration().setBoolean("hbase.master.start.wait.for.namespacemanager", true);
HTU.startMiniCluster(NB_SERVERS);
}
@ -86,6 +83,7 @@ public class TestRegionOpen {
final TableName tableName = TableName.valueOf(TestRegionOpen.class.getSimpleName());
ThreadPoolExecutor exec = getRS().getExecutorService()
.getExecutorThreadPool(ExecutorType.RS_OPEN_PRIORITY_REGION);
assertEquals(1, exec.getCompletedTaskCount()); // namespace region
HTableDescriptor htd = new HTableDescriptor(tableName);

View File

@ -48,7 +48,6 @@ public class TestWALOpenAfterDNRollingStart {
false);
TEST_UTIL.getConfiguration().setLong("hbase.regionserver.hlog.check.lowreplication.interval",
checkLowReplicationInterval);
TEST_UTIL.getConfiguration().setBoolean("hbase.master.start.wait.for.namespacemanager", true);
TEST_UTIL.startMiniDFSCluster(3);
TEST_UTIL.startMiniCluster(1);