HBASE-3263 Stack overflow in AssignmentManager
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1040342 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
450f6d28ab
commit
c849b4c26d
|
@ -720,6 +720,7 @@ Release 0.90.0 - Unreleased
|
|||
HBASE-3277 HBase Shell zk_dump command broken
|
||||
HBASE-3267 close_region shell command breaks region
|
||||
HBASE-3265 Regionservers waiting for ROOT while Master waiting for RegionServers
|
||||
HBASE-3263 Stack overflow in AssignmentManager
|
||||
|
||||
|
||||
IMPROVEMENTS
|
||||
|
|
|
@ -93,6 +93,11 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
|
||||
private TimeoutMonitor timeoutMonitor;
|
||||
|
||||
/*
|
||||
* Maximum times we recurse an assignment. See below in {@link #assign()}.
|
||||
*/
|
||||
private final int maximumAssignmentRecursions;
|
||||
|
||||
/**
|
||||
* Regions currently in transition. Map of encoded region names to the master
|
||||
* in-memory state for that region.
|
||||
|
@ -158,6 +163,8 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
Threads.setDaemonThreadRunning(timeoutMonitor,
|
||||
master.getServerName() + ".timeoutMonitor");
|
||||
this.zkTable = new ZKTable(this.master.getZooKeeper());
|
||||
this.maximumAssignmentRecursions =
|
||||
this.master.getConfiguration().getInt("hbase.assignment.maximum.recursions", 10);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -811,9 +818,23 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
/**
|
||||
* Caller must hold lock on the passed <code>state</code> object.
|
||||
* @param state
|
||||
* @param setOfflineInZK
|
||||
* @param forceNewPlan
|
||||
*/
|
||||
private void assign(final RegionState state, final boolean setOfflineInZK,
|
||||
final boolean forceNewPlan) {
|
||||
assign(state, setOfflineInZK, forceNewPlan, new AtomicInteger(0));
|
||||
}
|
||||
|
||||
/**
|
||||
* Caller must hold lock on the passed <code>state</code> object.
|
||||
* @param state
|
||||
* @param setOfflineInZK
|
||||
* @param forceNewPlan
|
||||
* @param recursions Keep a count so can have upper bound on recursions.
|
||||
*/
|
||||
private void assign(final RegionState state, final boolean setOfflineInZK,
|
||||
final boolean forceNewPlan, final int recursions) {
|
||||
if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
|
||||
if (this.master.isStopped()) {
|
||||
LOG.debug("Server stopped; skipping assign of " + state);
|
||||
|
@ -831,18 +852,22 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
} catch (Throwable t) {
|
||||
LOG.warn("Failed assignment of " +
|
||||
state.getRegion().getRegionNameAsString() + " to " +
|
||||
plan.getDestination() + ", trying to assign elsewhere instead", t);
|
||||
plan.getDestination() + ", trying to assign elsewhere instead; retry=" +
|
||||
recursions, t);
|
||||
// Clean out plan we failed execute and one that doesn't look like it'll
|
||||
// succeed anyways; we need a new plan!
|
||||
// Transition back to OFFLINE
|
||||
state.update(RegionState.State.OFFLINE);
|
||||
// Force a new plan and reassign.
|
||||
// Force a new plan and reassign. Will return null if no servers.
|
||||
if (getRegionPlan(state, plan.getDestination(), true) == null) {
|
||||
LOG.warn("Unable to find a viable location to assign region " +
|
||||
state.getRegion().getRegionNameAsString());
|
||||
return;
|
||||
}
|
||||
assign(state, false, false);
|
||||
if (recursions < this.maximumAssignmentRecursions) {
|
||||
assign(state, false, false, recursions + 1);
|
||||
}
|
||||
// Else Just leave the region in RIT. On timeout, we'll retry later.
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -926,7 +951,7 @@ public class AssignmentManager extends ZooKeeperListener {
|
|||
// The remove below hinges on the fact that the call to
|
||||
// serverManager.getOnlineServersList() returns a copy
|
||||
if (serverToExclude != null) servers.remove(serverToExclude);
|
||||
if (servers.size() <= 0) return null;
|
||||
if (servers.isEmpty()) return null;
|
||||
RegionPlan randomPlan = new RegionPlan(state.getRegion(), null,
|
||||
LoadBalancer.randomAssignment(servers));
|
||||
synchronized (this.regionPlans) {
|
||||
|
|
Loading…
Reference in New Issue