[
https://issues.apache.org/jira/browse/HDDS-3994?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
maobaolong updated HDDS-3994:
-----------------------------
Description:
After HDDS-3350 , the retry policy changed, and the client write performance
getting lower than before.
With HDDS-3350, I restore the method RatisHelper#createRetryPolicy to the
previous commit, it works well.
The previous is
{code:java}
static RetryPolicy createRetryPolicy(ConfigurationSource conf) {
int maxRetryCount =
conf.getInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY,
OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT);
long retryInterval = conf.getTimeDuration(OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT
.toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
TimeDuration sleepDuration =
TimeDuration.valueOf(retryInterval, TimeUnit.MILLISECONDS);
RetryPolicy retryPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(maxRetryCount, sleepDuration);
return retryPolicy;
}
{code}
When I switch logLevel to TRACE level, i see the following log While using
HDDS-3350
2020-07-21 12:56:18,273
[java.util.concurrent.ThreadPoolExecutor$Worker@157a0c99[State = -1, empty
queue]] DEBUG impl.OrderedAsync: schedule* attempt #35 with sleep 4606ms and
policy
RequestTypeDependentRetryPolicy{WRITE->org.apache.ratis.retry.ExceptionDependentRetry@19a0a59,
WATCH->org.apache.ratis.retry.ExceptionDependentRetry@41afac0b} for
RaftClientRequest:client-F768C93F1755->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
cid=2034, seq=1*, Watch(0), null
2020-07-21 12:56:18,820
[java.util.concurrent.ThreadPoolExecutor$Worker@64953e21[State = -1, empty
queue]] DEBUG impl.OrderedAsync: client-6F623ADF656D: send*
RaftClientRequest:client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
cid=2876, seq=1*, Watch(0), null
2020-07-21 12:56:18,821
[java.util.concurrent.ThreadPoolExecutor$Worker@64953e21[State = -1, empty
queue]] TRACE impl.OrderedAsync: client-6F623ADF656D: Failed*
RaftClientRequest:client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
cid=2876, seq=1*, Watch(0), null
java.util.concurrent.CompletionException:
org.apache.ratis.protocol.AlreadyClosedException:
client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5 is closed.
at
java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
at
java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
at
java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
at
java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614)
at
java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983)
at
org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:234)
at
org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:187)
at
org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:278)
at
org.apache.ratis.util.SlidingWindow$Client.retry(SlidingWindow.java:294)
at
org.apache.ratis.client.impl.OrderedAsync.lambda$scheduleWithTimeout$7(OrderedAsync.java:220)
at
org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:141)
at
org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:155)
at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:38)
at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:79)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
Caused by: org.apache.ratis.protocol.AlreadyClosedException:
client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5 is closed.
at
org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.onNext(GrpcClientProtocolClient.java:313)
at
org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsync(GrpcClientRpc.java:68)
at
org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:233)
... 15 more
was:
After HDDS-3350 , the retry policy changed, and the client write performance
getting lower than before.
With HDDS-3350, I restore the method RatisHelper#createRetryPolicy to the
previous commit, it works well.
The previous is
{code:java}
static RetryPolicy createRetryPolicy(ConfigurationSource conf) {
int maxRetryCount =
conf.getInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY,
OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT);
long retryInterval = conf.getTimeDuration(OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT
.toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
TimeDuration sleepDuration =
TimeDuration.valueOf(retryInterval, TimeUnit.MILLISECONDS);
RetryPolicy retryPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(maxRetryCount, sleepDuration);
return retryPolicy;
}
{code}
> Write object when met exception can be slower than before
> ---------------------------------------------------------
>
> Key: HDDS-3994
> URL: https://issues.apache.org/jira/browse/HDDS-3994
> Project: Hadoop Distributed Data Store
> Issue Type: Bug
> Components: Ozone Client
> Affects Versions: 0.6.0
> Reporter: maobaolong
> Assignee: maobaolong
> Priority: Major
>
> After HDDS-3350 , the retry policy changed, and the client write performance
> getting lower than before.
>
> With HDDS-3350, I restore the method RatisHelper#createRetryPolicy to the
> previous commit, it works well.
>
> The previous is
>
> {code:java}
> static RetryPolicy createRetryPolicy(ConfigurationSource conf) {
> int maxRetryCount =
> conf.getInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY,
> OzoneConfigKeys.
> DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT);
> long retryInterval = conf.getTimeDuration(OzoneConfigKeys.
> DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, OzoneConfigKeys.
> DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT
> .toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
> TimeDuration sleepDuration =
> TimeDuration.valueOf(retryInterval, TimeUnit.MILLISECONDS);
> RetryPolicy retryPolicy = RetryPolicies
> .retryUpToMaximumCountWithFixedSleep(maxRetryCount, sleepDuration);
> return retryPolicy;
> }
> {code}
> When I switch logLevel to TRACE level, i see the following log While using
> HDDS-3350
> 2020-07-21 12:56:18,273
> [java.util.concurrent.ThreadPoolExecutor$Worker@157a0c99[State = -1, empty
> queue]] DEBUG impl.OrderedAsync: schedule* attempt #35 with sleep 4606ms and
> policy
> RequestTypeDependentRetryPolicy{WRITE->org.apache.ratis.retry.ExceptionDependentRetry@19a0a59,
> WATCH->org.apache.ratis.retry.ExceptionDependentRetry@41afac0b} for
> RaftClientRequest:client-F768C93F1755->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
> cid=2034, seq=1*, Watch(0), null
> 2020-07-21 12:56:18,820
> [java.util.concurrent.ThreadPoolExecutor$Worker@64953e21[State = -1, empty
> queue]] DEBUG impl.OrderedAsync: client-6F623ADF656D: send*
> RaftClientRequest:client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
> cid=2876, seq=1*, Watch(0), null
> 2020-07-21 12:56:18,821
> [java.util.concurrent.ThreadPoolExecutor$Worker@64953e21[State = -1, empty
> queue]] TRACE impl.OrderedAsync: client-6F623ADF656D: Failed*
> RaftClientRequest:client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
> cid=2876, seq=1*, Watch(0), null
> java.util.concurrent.CompletionException:
> org.apache.ratis.protocol.AlreadyClosedException:
> client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5 is closed.
> at
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
> at
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
> at
> java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
> at
> java.util.concurrent.CompletableFuture.uniApplyStage(CompletableFuture.java:614)
> at
> java.util.concurrent.CompletableFuture.thenApply(CompletableFuture.java:1983)
> at
> org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:234)
> at
> org.apache.ratis.client.impl.OrderedAsync.sendRequestWithRetry(OrderedAsync.java:187)
> at
> org.apache.ratis.util.SlidingWindow$Client.sendOrDelayRequest(SlidingWindow.java:278)
> at
> org.apache.ratis.util.SlidingWindow$Client.retry(SlidingWindow.java:294)
> at
> org.apache.ratis.client.impl.OrderedAsync.lambda$scheduleWithTimeout$7(OrderedAsync.java:220)
> at
> org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$0(TimeoutScheduler.java:141)
> at
> org.apache.ratis.util.TimeoutScheduler.lambda$onTimeout$1(TimeoutScheduler.java:155)
> at org.apache.ratis.util.LogUtils.runAndLog(LogUtils.java:38)
> at org.apache.ratis.util.LogUtils$1.run(LogUtils.java:79)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Caused by: org.apache.ratis.protocol.AlreadyClosedException:
> client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5 is closed.
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.onNext(GrpcClientProtocolClient.java:313)
> at
> org.apache.ratis.grpc.client.GrpcClientRpc.sendRequestAsync(GrpcClientRpc.java:68)
> at
> org.apache.ratis.client.impl.OrderedAsync.sendRequest(OrderedAsync.java:233)
> ... 15 more
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]