You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2011/04/26 01:13:41 UTC
svn commit: r1096630 - in /hbase/trunk: CHANGES.txt
src/main/java/org/apache/hadoop/hbase/master/HMaster.java
Author: stack
Date: Mon Apr 25 23:13:41 2011
New Revision: 1096630
URL: http://svn.apache.org/viewvc?rev=1096630&view=rev
Log:
HBASE-3749 Master can't exit when open port failed
Modified:
hbase/trunk/CHANGES.txt
hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
Modified: hbase/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hbase/trunk/CHANGES.txt?rev=1096630&r1=1096629&r2=1096630&view=diff
==============================================================================
--- hbase/trunk/CHANGES.txt (original)
+++ hbase/trunk/CHANGES.txt Mon Apr 25 23:13:41 2011
@@ -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)
Modified: hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java
URL: http://svn.apache.org/viewvc/hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java?rev=1096630&r1=1096629&r2=1096630&view=diff
==============================================================================
--- hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (original)
+++ hbase/trunk/src/main/java/org/apache/hadoop/hbase/master/HMaster.java Mon Apr 25 23:13:41 2011
@@ -540,50 +540,46 @@ implements HMasterInterface, HMasterRegi
* 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);
-
- // 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);
+ 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");
+
+ // 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() {