lokeshj1703 commented on a change in pull request #1005:
URL: https://github.com/apache/hadoop-ozone/pull/1005#discussion_r438143919
##########
File path:
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java
##########
@@ -269,23 +282,76 @@ static GrpcTlsConfig createTlsClientConfig(SecurityConfig
conf,
return tlsConfig;
}
- static RetryPolicy createRetryPolicy(ConfigurationSource conf) {
- int maxRetryCount =
- conf.getInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY,
+ public static RetryPolicy createRetryPolicy(ConfigurationSource conf) {
+ ExponentialBackoffRetry exponentialBackoffRetry =
+ createExponentialBackoffPolicy(conf);
+ MultipleLinearRandomRetry multipleLinearRandomRetry =
+ MultipleLinearRandomRetry.parseCommaSeparated(conf.get(
+ OzoneConfigKeys.DFS_RATIS_CLIENT_MULTILINEAR_RANDOM_RETRY_POLICY,
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;
+ DFS_RATIS_CLIENT_MULTILINEAR_RANDOM_RETRY_POLICY_DEFAULT));
+
+ long writeTimeout = conf.getTimeDuration(
+ OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_WRITE_TIMEOUT,
OzoneConfigKeys.
+ DFS_RATIS_CLIENT_REQUEST_WRITE_TIMEOUT_DEFAULT
+ .toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
+ long watchTimeout = conf.getTimeDuration(
+ OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_WATCH_TIMEOUT,
OzoneConfigKeys.
+ DFS_RATIS_CLIENT_REQUEST_WATCH_TIMEOUT_DEFAULT
+ .toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
+
+ return RequestTypeDependentRetryPolicy.newBuilder()
+ .setRetryPolicy(RaftProtos.RaftClientRequestProto.TypeCase.WRITE,
+ createExceptionDependentPolicy(exponentialBackoffRetry,
+ multipleLinearRandomRetry, exponentialBackoffRetry))
+ .setRetryPolicy(RaftProtos.RaftClientRequestProto.TypeCase.WATCH,
+ createExceptionDependentPolicy(exponentialBackoffRetry,
+ multipleLinearRandomRetry, RetryPolicies.noRetry()))
+ .setTimeout(RaftProtos.RaftClientRequestProto.TypeCase.WRITE,
+ TimeDuration.valueOf(writeTimeout, TimeUnit.MILLISECONDS))
+ .setTimeout(RaftProtos.RaftClientRequestProto.TypeCase.WATCH,
+ TimeDuration.valueOf(watchTimeout, TimeUnit.MILLISECONDS))
+ .build();
+ }
+
+ private static ExponentialBackoffRetry createExponentialBackoffPolicy(
+ ConfigurationSource conf) {
+ long exponentialBaseSleep = conf.getTimeDuration(
+ OzoneConfigKeys.DFS_RATIS_CLIENT_EXPONENTIAL_BACKOFF_BASE_SLEEP,
+ OzoneConfigKeys.DFS_RATIS_CLIENT_EXPONENTIAL_BACKOFF_BASE_SLEEP_DEFAULT
+ .toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
+ long exponentialMaxSleep = conf.getTimeDuration(
+ OzoneConfigKeys.DFS_RATIS_CLIENT_EXPONENTIAL_BACKOFF_MAX_SLEEP,
+ OzoneConfigKeys.
+ DFS_RATIS_CLIENT_EXPONENTIAL_BACKOFF_MAX_SLEEP_DEFAULT
+ .toIntExact(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
+ return ExponentialBackoffRetry.newBuilder()
+ .setBaseSleepTime(
+ TimeDuration.valueOf(exponentialBaseSleep, TimeUnit.MILLISECONDS))
+ .setMaxSleepTime(
+ TimeDuration.valueOf(exponentialMaxSleep, TimeUnit.MILLISECONDS))
+ .build();
+ }
+
+ private static ExceptionDependentRetry createExceptionDependentPolicy(
+ ExponentialBackoffRetry exponentialBackoffRetry,
+ MultipleLinearRandomRetry multipleLinearRandomRetry,
Review comment:
RaftLogIOException is never received at raft client. I have added
AlreadyClosedException.
----------------------------------------------------------------
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]