Revert "HBASE-16488 Starting namespace and quota services in master startup asynchronizely (Stephen Yuan Jiang)"
This reverts commit af359d03b5
.
This commit is contained in:
parent
a8ac84f1a6
commit
3f785a83b2
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
Loading…
Reference in New Issue