HBASE-3749 Master can't exit when open port failed
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1096630 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e01e57ff92
commit
258c11120e
|
@ -86,6 +86,7 @@ Release 0.91.0 - Unreleased
|
|||
(Prakash Khemani)
|
||||
HBASE-3819 TestSplitLogWorker has too many SLWs running -- makes for
|
||||
contention and occasional failures
|
||||
HBASE-3749 Master can't exit when open port failed (gaojinchao)
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3290 Max Compaction Size (Nicolas Spiegelberg via Stack)
|
||||
|
|
|
@ -540,50 +540,46 @@ implements HMasterInterface, HMasterRegionInterface, MasterServices, Server {
|
|||
* as OOMEs; it should be lightly loaded. See what HRegionServer does if
|
||||
* need to install an unexpected exception handler.
|
||||
*/
|
||||
private void startServiceThreads() {
|
||||
try {
|
||||
// Start the executor service pools
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
|
||||
conf.getInt("hbase.master.executor.openregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
|
||||
conf.getInt("hbase.master.executor.closeregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 3));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 5));
|
||||
// We depend on there being only one instance of this executor running
|
||||
// at a time. To do concurrency, would need fencing of enable/disable of
|
||||
// tables.
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
|
||||
private void startServiceThreads() throws IOException{
|
||||
|
||||
// Start the executor service pools
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_OPEN_REGION,
|
||||
conf.getInt("hbase.master.executor.openregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_CLOSE_REGION,
|
||||
conf.getInt("hbase.master.executor.closeregion.threads", 5));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_SERVER_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 3));
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_META_SERVER_OPERATIONS,
|
||||
conf.getInt("hbase.master.executor.serverops.threads", 5));
|
||||
|
||||
// We depend on there being only one instance of this executor running
|
||||
// at a time. To do concurrency, would need fencing of enable/disable of
|
||||
// tables.
|
||||
this.executorService.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1);
|
||||
|
||||
// Start log cleaner thread
|
||||
String n = Thread.currentThread().getName();
|
||||
this.logCleaner =
|
||||
new LogCleaner(conf.getInt("hbase.master.cleaner.interval", 60 * 1000),
|
||||
this, conf, getMasterFileSystem().getFileSystem(),
|
||||
getMasterFileSystem().getOldLogDir());
|
||||
Threads.setDaemonThreadRunning(logCleaner, n + ".oldLogCleaner");
|
||||
// Start log cleaner thread
|
||||
String n = Thread.currentThread().getName();
|
||||
this.logCleaner =
|
||||
new LogCleaner(conf.getInt("hbase.master.cleaner.interval", 60 * 1000),
|
||||
this, conf, getMasterFileSystem().getFileSystem(),
|
||||
getMasterFileSystem().getOldLogDir());
|
||||
Threads.setDaemonThreadRunning(logCleaner, n + ".oldLogCleaner");
|
||||
|
||||
// Put up info server.
|
||||
int port = this.conf.getInt("hbase.master.info.port", 60010);
|
||||
if (port >= 0) {
|
||||
String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
|
||||
this.infoServer = new InfoServer(MASTER, a, port, false);
|
||||
this.infoServer.setAttribute(MASTER, this);
|
||||
this.infoServer.start();
|
||||
}
|
||||
// Start allowing requests to happen.
|
||||
this.rpcServer.openServer();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Started service threads");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
if (e instanceof RemoteException) {
|
||||
e = ((RemoteException)e).unwrapRemoteException();
|
||||
}
|
||||
// Something happened during startup. Shut things down.
|
||||
abort("Failed startup", e);
|
||||
// Put up info server.
|
||||
int port = this.conf.getInt("hbase.master.info.port", 60010);
|
||||
if (port >= 0) {
|
||||
String a = this.conf.get("hbase.master.info.bindAddress", "0.0.0.0");
|
||||
this.infoServer = new InfoServer(MASTER, a, port, false);
|
||||
this.infoServer.setAttribute(MASTER, this);
|
||||
this.infoServer.start();
|
||||
}
|
||||
|
||||
// Start allowing requests to happen.
|
||||
this.rpcServer.openServer();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Started service threads");
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
private void stopServiceThreads() {
|
||||
|
|
Loading…
Reference in New Issue