[
https://issues.apache.org/jira/browse/HDDS-3994?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17163765#comment-17163765
]
maobaolong commented on HDDS-3994:
----------------------------------
[~ljain] Thanks for your clarification, after a quick overlook at the the new
retry policy, i believe it could be a better retry policy, as you said, we need
some more time to tune the default values, but now, it bring us performance
impact after we upgrade our s3g.
We spent a whole day to find out the original retry policy has been replaced by
the newer one, so that our configure key related the RetryLimitedPolicy cannot
affected. I mean, although the newer retry policy could be better, but we
should construct a retry policy configurable for user, and default to the
original one, meanwhile, we can claim that we invent a new better retry policy,
and the user like us can do some testing and tuning, after that, we can be
brave to change the retry policy to the newer one by modify the ozone-site.xml,
even though, we have to change the retry policy for some node and monitor the
performance on the production env for some day, then, we can change all other
nodes.
So, i think a configurable framework for the retry policy is necessary, and
default to the original RetryLimitedPolicy is also important, new policy can be
acceptable should after a studying, testing and tuning to the new policy, it
need a process, we will continue to tuning the new policy later. Now we need to
restore the retry logic and solve the low performance caused by the lack of
familiarity with new retry policy.
Please take a look at my PR, thanks, related,
https://github.com/apache/hadoop-ozone/pull/1231
> 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
> Labels: pull-request-available
>
> 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:11,822 [grpc-default-executor-5] 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.LeaderNotReadyException:
> 207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F is in LEADER state
> but not ready yet.
> 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$UniApply.tryFire(CompletableFuture.java:577)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.completeReplyExceptionally(GrpcClientProtocolClient.java:358)
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers.access$000(GrpcClientProtocolClient.java:264)
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$1.onNext(GrpcClientProtocolClient.java:283)
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$1.onNext(GrpcClientProtocolClient.java:269)
> at
> org.apache.ratis.thirdparty.io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onMessage(ClientCalls.java:436)
> at
> org.apache.ratis.thirdparty.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInternal(ClientCallImpl.java:658)
> at
> org.apache.ratis.thirdparty.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1MessagesAvailable.runInContext(ClientCallImpl.java:643)
> at
> org.apache.ratis.thirdparty.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
> at
> org.apache.ratis.thirdparty.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
> 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.LeaderNotReadyException:
> 207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F is in LEADER state
> but not ready yet.
> at
> org.apache.ratis.client.impl.ClientProtoUtils.toRaftClientReply(ClientProtoUtils.java:281)
> at
> org.apache.ratis.grpc.client.GrpcClientProtocolClient$AsyncStreamObservers$1.onNext(GrpcClientProtocolClient.java:274)
> ... 9 more
> 2020-07-21 12:56:11,822 [grpc-default-executor-5] DEBUG impl.OrderedAsync:
> schedule* attempt #1 with sleep 608ms and policy
> RequestTypeDependentRetryPolicy{WRITE->org.apache.ratis.retry.ExceptionDependentRetry@5685c509,
> WATCH->org.apache.ratis.retry.ExceptionDependentRetry@5572f5cd} for
> RaftClientRequest:client-6F623ADF656D->207b98d9-ad64-45a8-940f-504b514feff5@group-83A28012848F,
> cid=2876, seq=1*, Watch(0), null
> 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]