HBASE-21227 Implement exponential retrying backoff for Assign/UnassignRegionHandler introduced in HBASE-21217

This commit is contained in:
zhangduo 2018-09-26 15:31:45 +08:00
parent 7b2f5950ed
commit d7e08317d2
4 changed files with 80 additions and 10 deletions

View File

@ -18,12 +18,15 @@
*/
package org.apache.hadoop.hbase.util;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
/**
* Operation retry accounting.
* Use to calculate wait period, {@link #getBackoffTimeAndIncrementAttempts()}}, or for performing
@ -44,6 +47,7 @@ public class RetryCounter {
private long maxSleepTime;
private TimeUnit timeUnit;
private BackoffPolicy backoffPolicy;
private float jitter;
private static final BackoffPolicy DEFAULT_BACKOFF_POLICY = new ExponentialBackoffPolicy();
@ -53,6 +57,7 @@ public class RetryCounter {
maxSleepTime = -1;
timeUnit = TimeUnit.MILLISECONDS;
backoffPolicy = DEFAULT_BACKOFF_POLICY;
jitter = 0.0f;
}
public RetryConfig(int maxAttempts, long sleepInterval, long maxSleepTime,
@ -89,6 +94,13 @@ public class RetryCounter {
return this;
}
public RetryConfig setJitter(float jitter) {
Preconditions.checkArgument(jitter >= 0.0f && jitter < 1.0f,
"Invalid jitter: %s, should be in range [0.0, 1.0)", jitter);
this.jitter = jitter;
return this;
}
public int getMaxAttempts() {
return maxAttempts;
}
@ -105,17 +117,26 @@ public class RetryCounter {
return timeUnit;
}
public float getJitter() {
return jitter;
}
public BackoffPolicy getBackoffPolicy() {
return backoffPolicy;
}
}
private static long addJitter(long interval, float jitter) {
long jitterInterval = (long) (interval * ThreadLocalRandom.current().nextFloat() * jitter);
return interval + jitterInterval;
}
/**
* Policy for calculating sleeping intervals between retry attempts
*/
public static class BackoffPolicy {
public long getBackoffTime(RetryConfig config, int attempts) {
return config.getSleepInterval();
return addJitter(config.getSleepInterval(), config.getJitter());
}
}
@ -123,7 +144,7 @@ public class RetryCounter {
@Override
public long getBackoffTime(RetryConfig config, int attempts) {
long backoffTime = (long) (config.getSleepInterval() * Math.pow(2, attempts));
return backoffTime;
return addJitter(backoffTime, config.getJitter());
}
}
@ -155,7 +176,6 @@ public class RetryCounter {
/**
* Sleep for a back off time as supplied by the backoff policy, and increases the attempts
* @throws InterruptedException
*/
public void sleepUntilNextRetry() throws InterruptedException {
int attempts = getAttemptTimes();

View File

@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerServices.PostOpenDeployContext;
import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -54,12 +55,15 @@ public class AssignRegionHandler extends EventHandler {
private final long masterSystemTime;
private final RetryCounter retryCounter;
public AssignRegionHandler(RegionServerServices server, RegionInfo regionInfo,
@Nullable TableDescriptor tableDesc, long masterSystemTime, EventType eventType) {
super(server, eventType);
this.regionInfo = regionInfo;
this.tableDesc = tableDesc;
this.masterSystemTime = masterSystemTime;
this.retryCounter = HandlerUtil.getRetryCounter();
}
private RegionServerServices getServer() {
@ -106,10 +110,11 @@ public class AssignRegionHandler extends EventHandler {
// calling reportRegionStateTransition, so the HMaster will think the region is offline,
// before we actually close the region, as reportRegionStateTransition is part of the
// closing process.
LOG.info("Receiving OPEN for the region:{}, which we are trying to close, try again later",
regionName);
// TODO: backoff
rs.getExecutorService().delayedSubmit(this, 1, TimeUnit.SECONDS);
long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
LOG.info(
"Receiving OPEN for the region:{}, which we are trying to close, try again after {}ms",
regionName, backoff);
rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
}
return;
}

View File

@ -0,0 +1,41 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hbase.regionserver.handler;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.hadoop.hbase.util.RetryCounterFactory;
import org.apache.yetus.audience.InterfaceAudience;
@InterfaceAudience.Private
final class HandlerUtil {
private HandlerUtil() {
}
/**
* Get an exponential backoff retry counter. The base unit is 100 milliseconds, and the max
* backoff time is 30 seconds.
*/
public static RetryCounter getRetryCounter() {
return new RetryCounterFactory(
new RetryCounter.RetryConfig().setBackoffPolicy(new RetryCounter.ExponentialBackoffPolicy())
.setSleepInterval(100).setMaxSleepTime(30000).setMaxAttempts(Integer.MAX_VALUE)
.setTimeUnit(TimeUnit.MILLISECONDS).setJitter(0.01f)).create();
}
}

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hbase.regionserver.Region;
import org.apache.hadoop.hbase.regionserver.RegionServerServices;
import org.apache.hadoop.hbase.regionserver.RegionServerServices.RegionStateTransitionContext;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RetryCounter;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -54,12 +55,15 @@ public class UnassignRegionHandler extends EventHandler {
private final ServerName destination;
private final RetryCounter retryCounter;
public UnassignRegionHandler(RegionServerServices server, String encodedName, boolean abort,
@Nullable ServerName destination, EventType eventType) {
super(server, eventType);
this.encodedName = encodedName;
this.abort = abort;
this.destination = destination;
this.retryCounter = HandlerUtil.getRetryCounter();
}
private RegionServerServices getServer() {
@ -76,10 +80,10 @@ public class UnassignRegionHandler extends EventHandler {
// This could happen as we will update the region state to OPEN when calling
// reportRegionStateTransition, so the HMaster will think the region is online, before we
// actually open the region, as reportRegionStateTransition is part of the opening process.
long backoff = retryCounter.getBackoffTimeAndIncrementAttempts();
LOG.warn("Received CLOSE for the region: {}, which we are already " +
"trying to OPEN. try again later.", encodedName);
// TODO: backoff
rs.getExecutorService().delayedSubmit(this, 1, TimeUnit.SECONDS);
"trying to OPEN. try again after {}ms", encodedName, backoff);
rs.getExecutorService().delayedSubmit(this, backoff, TimeUnit.MILLISECONDS);
} else {
LOG.info("Received CLOSE for the region: {}, which we are already trying to CLOSE," +
" but not completed yet", encodedName);