HBASE-19641 AsyncHBaseAdmin should use exponential backoff when polling the procedure result
This commit is contained in:
parent
a47afc84cd
commit
1fa3637b4d
|
@ -89,6 +89,7 @@ import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
|
|||
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.Timeout;
|
||||
import org.apache.hbase.thirdparty.io.netty.util.TimerTask;
|
||||
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.RequestConverter;
|
||||
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
|
||||
|
@ -2553,40 +2554,36 @@ class RawAsyncHBaseAdmin implements AsyncAdmin {
|
|||
future.completeExceptionally(error);
|
||||
return;
|
||||
}
|
||||
getProcedureResult(procId, future);
|
||||
getProcedureResult(procId, future, 0);
|
||||
});
|
||||
return future;
|
||||
}
|
||||
|
||||
private void getProcedureResult(final long procId, CompletableFuture<Void> future) {
|
||||
this.<GetProcedureResultResponse> newMasterCaller()
|
||||
.action(
|
||||
(controller, stub) -> this
|
||||
.<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
|
||||
controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
|
||||
(s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
|
||||
.call()
|
||||
.whenComplete(
|
||||
(response, error) -> {
|
||||
if (error != null) {
|
||||
LOG.warn("failed to get the procedure result procId=" + procId,
|
||||
ConnectionUtils.translateException(error));
|
||||
retryTimer.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
|
||||
TimeUnit.NANOSECONDS);
|
||||
return;
|
||||
}
|
||||
if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
|
||||
retryTimer.newTimeout(t -> getProcedureResult(procId, future), pauseNs,
|
||||
TimeUnit.NANOSECONDS);
|
||||
return;
|
||||
}
|
||||
if (response.hasException()) {
|
||||
IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
|
||||
future.completeExceptionally(ioe);
|
||||
} else {
|
||||
future.complete(null);
|
||||
}
|
||||
});
|
||||
private void getProcedureResult(long procId, CompletableFuture<Void> future, int retries) {
|
||||
this.<GetProcedureResultResponse> newMasterCaller().action((controller, stub) -> this
|
||||
.<GetProcedureResultRequest, GetProcedureResultResponse, GetProcedureResultResponse> call(
|
||||
controller, stub, GetProcedureResultRequest.newBuilder().setProcId(procId).build(),
|
||||
(s, c, req, done) -> s.getProcedureResult(c, req, done), (resp) -> resp))
|
||||
.call().whenComplete((response, error) -> {
|
||||
if (error != null) {
|
||||
LOG.warn("failed to get the procedure result procId={}", procId,
|
||||
ConnectionUtils.translateException(error));
|
||||
retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
|
||||
ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
|
||||
return;
|
||||
}
|
||||
if (response.getState() == GetProcedureResultResponse.State.RUNNING) {
|
||||
retryTimer.newTimeout(t -> getProcedureResult(procId, future, retries + 1),
|
||||
ConnectionUtils.getPauseTime(pauseNs, retries), TimeUnit.NANOSECONDS);
|
||||
return;
|
||||
}
|
||||
if (response.hasException()) {
|
||||
IOException ioe = ForeignExceptionUtil.toIOException(response.getException());
|
||||
future.completeExceptionally(ioe);
|
||||
} else {
|
||||
future.complete(null);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private <T> CompletableFuture<T> failedFuture(Throwable error) {
|
||||
|
|
Loading…
Reference in New Issue