HBASE-3687 Bulk assign on startup should handle a ServerNotRunningException
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1084314 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
bb28add83e
commit
c1fcc65b19
|
@ -158,6 +158,7 @@ Release 0.90.2 - Unreleased
|
|||
HBASE-3621 The timeout handler in AssignmentManager does an RPC while
|
||||
holding lock on RIT; a big no-no (Ted Yu via Stack)
|
||||
HBASE-3575 Update rename table script
|
||||
HBASE-3687 Bulk assign on startup should handle a ServerNotRunningException
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-3542 MultiGet methods in Thrift
|
||||
|
|
|
@ -925,9 +925,27 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// Move on to open regions.
|
||||
try {
|
||||
// Send OPEN RPC. This can fail if the server on other end is is not up.
|
||||
// If we fail, fail the startup by aborting the server. There is one
|
||||
// exception we will tolerate: ServerNotRunningException. This is thrown
|
||||
// between report of regionserver being up and
|
||||
long maxWaitTime = System.currentTimeMillis() +
|
||||
this.master.getConfiguration().getLong("hbase.regionserver.rpc.startup.waittime", 60000);
|
||||
while (!this.master.isStopped()) {
|
||||
try {
|
||||
this.serverManager.sendRegionOpen(destination, regions);
|
||||
} catch (org.apache.hadoop.hbase.ipc.ServerNotRunningException e) {
|
||||
// This is the one exception to retry. For all else we should just fail
|
||||
// the startup.
|
||||
long now = System.currentTimeMillis();
|
||||
if (now > maxWaitTime) throw e;
|
||||
LOG.debug("Server is not yet up; waiting up to " +
|
||||
(maxWaitTime - now) + "ms", e);
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
this.master.abort("Failed assignment of regions to " + destination, t);
|
||||
this.master.abort("Failed assignment of regions to " + destination +
|
||||
"; bulk assign FAILED", t);
|
||||
return;
|
||||
}
|
||||
LOG.debug("Bulk assigning done for " + destination.getServerName());
|
||||
|
|
Loading…
Reference in New Issue