bharathv commented on a change in pull request #1755:
URL: https://github.com/apache/hbase/pull/1755#discussion_r430767349
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
##########
@@ -1972,6 +1975,13 @@ private void unassign(final HRegionInfo region,
final RegionState state, final int versionOfClosingNode,
final ServerName dest, final boolean transitionInZK,
final ServerName src) {
+ String encodedName = region.getEncodedName();
+ AtomicInteger failedCloseCount = failedCloseTracker.get(encodedName);
+ if (failedCloseCount == null) {
+ failedCloseCount = new AtomicInteger();
+ failedCloseTracker.put(encodedName, failedCloseCount);
Review comment:
Aren't all the codepaths reaching this point, expected to take an
exclusive lock on the region.encodedName()? If so, wondering if we should worry
about the non-thread-safe access for this map. I checked all the callers, all
except one path in forceRegionStateToOffline() follow this pattern, we should
probably fix that.
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
##########
@@ -1997,80 +2007,76 @@ private void unassign(final HRegionInfo region,
}
try {
// Send CLOSE RPC
- if (serverManager.sendRegionClose(server, region,
- versionOfClosingNode, dest, transitionInZK)) {
- LOG.debug("Sent CLOSE to " + server + " for region " +
- region.getRegionNameAsString());
+ if (serverManager.sendRegionClose(server, region,
versionOfClosingNode, dest,
+ transitionInZK)) {
+ LOG.debug("Sent CLOSE to " + server + " for region " +
region.getRegionNameAsString());
if (useZKForAssignment && !transitionInZK && state != null) {
// Retry to make sure the region is
// closed so as to avoid double assignment.
- unassign(region, state, versionOfClosingNode,
- dest, transitionInZK, src);
+ unassign(region, state, versionOfClosingNode, dest,
transitionInZK, src);
}
return;
}
// This never happens. Currently regionserver close always return true.
// Todo; this can now happen (0.96) if there is an exception in a
coprocessor
- LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
- region.getRegionNameAsString());
+ LOG.warn("Server " + server + " region CLOSE RPC returned false for "
+ + region.getRegionNameAsString());
} catch (Throwable t) {
long sleepTime = 0;
Configuration conf = this.server.getConfiguration();
if (t instanceof RemoteException) {
- t = ((RemoteException)t).unwrapRemoteException();
+ t = ((RemoteException) t).unwrapRemoteException();
}
boolean logRetries = true;
- if (t instanceof RegionServerAbortedException
- || t instanceof RegionServerStoppedException
+ if (t instanceof RegionServerAbortedException || t instanceof
RegionServerStoppedException
|| t instanceof ServerNotRunningYetException) {
// RS is aborting or stopping, we cannot offline the region since
the region may need
- // to do WAL recovery. Until we see the RS expiration, we should
retry.
+ // to do WAL recovery. Until we see the RS expiration, we should
retry.
sleepTime = 1L + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
} else if (t instanceof NotServingRegionException) {
- LOG.debug("Offline " + region.getRegionNameAsString()
- + ", it's not any more on " + server, t);
+ LOG.debug(
+ "Offline " + region.getRegionNameAsString() + ", it's not any more
on " + server, t);
if (transitionInZK) {
deleteClosingOrClosedNode(region, server);
}
if (state != null) {
regionOffline(region);
}
return;
- } else if ((t instanceof FailedServerException) || (state != null &&
- t instanceof RegionAlreadyInTransitionException)) {
- if (t instanceof FailedServerException) {
- sleepTime = 1L + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
+ } else if ((t instanceof FailedServerException)
Review comment:
Is there any change in functionality of this section of diff? I think
the answer is no and its mostly indents, but I wanted to double check..can you
please confirm?
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
##########
@@ -2079,16 +2085,29 @@ private void unassign(final HRegionInfo region,
}
if (logRetries) {
- LOG.info("Server " + server + " returned " + t + " for "
- + region.getRegionNameAsString() + ", try=" + i
- + " of " + this.maximumAttempts, t);
+ LOG.info("Server " + server + " returned " + t + " for " +
region.getRegionNameAsString()
+ + ", try=" + i + " of " + this.maximumAttempts,
+ t);
// Presume retry or server will expire.
}
}
}
- // Run out of attempts
- if (state != null) {
- regionStates.updateRegionState(region, State.FAILED_CLOSE);
+
+ long sleepTime = backoffPolicy.getBackoffTime(retryConfig,
Review comment:
Pardon my ignorance but I don't fully understand the fix. I think the
ask in the jira to spread out the unassigns a bit by using a backoff based
approach. To do that why not just fix the sleepTime above in L2075 to use an
exponential backoff based approach?
##########
File path:
hbase-server/src/main/java/org/apache/hadoop/hbase/master/AssignmentManager.java
##########
@@ -2079,16 +2085,29 @@ private void unassign(final HRegionInfo region,
}
if (logRetries) {
- LOG.info("Server " + server + " returned " + t + " for "
- + region.getRegionNameAsString() + ", try=" + i
- + " of " + this.maximumAttempts, t);
+ LOG.info("Server " + server + " returned " + t + " for " +
region.getRegionNameAsString()
+ + ", try=" + i + " of " + this.maximumAttempts,
+ t);
// Presume retry or server will expire.
}
}
}
- // Run out of attempts
- if (state != null) {
- regionStates.updateRegionState(region, State.FAILED_CLOSE);
+
+ long sleepTime = backoffPolicy.getBackoffTime(retryConfig,
+ getFailedAttempts(encodedName, failedCloseTracker));
+ if (failedCloseCount.incrementAndGet() <= maximumAttempts && sleepTime >
0) {
+ if (failedCloseTracker.containsKey(encodedName)) {
+ // Sleep before trying unassign if this region has failed to close
before
+ scheduledThreadPoolExecutor.schedule(new DelayedUnAssignCallable(this,
region, state,
Review comment:
javadoc for the method says "Send CLOSE RPC if the server is online,
otherwise, offline the region.". With this delayed callables, aren't we
violating that? If we hit this block, we will still be in the CLOSING state
with the thread pool retrying in the background. Curious if that causes any
issues in the statemachine.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]