diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java index b52f39380b..83632b5c67 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClientUtils.java @@ -17,22 +17,29 @@ */ package org.apache.hadoop.ozone.client; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import org.apache.hadoop.hdds.client.OzoneQuota; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException; import org.apache.hadoop.io.retry.RetryPolicies; import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.ozone.OzoneConsts; -import org.apache.hadoop.ozone.client.rest.response.*; +import org.apache.hadoop.ozone.client.rest.response.BucketInfo; +import org.apache.hadoop.ozone.client.rest.response.KeyInfo; +import org.apache.hadoop.ozone.client.rest.response.KeyInfoDetails; +import org.apache.hadoop.ozone.client.rest.response.KeyLocation; +import org.apache.hadoop.ozone.client.rest.response.VolumeInfo; +import org.apache.hadoop.ozone.client.rest.response.VolumeOwner; import org.apache.ratis.protocol.AlreadyClosedException; import org.apache.ratis.protocol.GroupMismatchException; import org.apache.ratis.protocol.RaftRetryFailureException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - /** A utility class for OzoneClient. */ public final class OzoneClientUtils { @@ -129,14 +136,31 @@ public static KeyInfoDetails asKeyInfoDetails(OzoneKeyDetails key) { public static RetryPolicy createRetryPolicy(int maxRetryCount, long retryInterval) { - // just retry without sleep - RetryPolicy retryPolicy = RetryPolicies - .retryUpToMaximumCountWithFixedSleep(maxRetryCount, retryInterval, - TimeUnit.MILLISECONDS); - return retryPolicy; + // retry with fixed sleep between retries + return RetryPolicies.retryUpToMaximumCountWithFixedSleep( + maxRetryCount, retryInterval, TimeUnit.MILLISECONDS); } public static List> getExceptionList() { return EXCEPTION_LIST; } + + public static Map, RetryPolicy> + getRetryPolicyByException(int maxRetryCount, long retryInterval) { + Map, RetryPolicy> policyMap = new HashMap<>(); + for (Class ex : EXCEPTION_LIST) { + if (ex == TimeoutException.class || + ex == RaftRetryFailureException.class) { + // retry without sleep + policyMap.put(ex, createRetryPolicy(maxRetryCount, 0)); + } else { + // retry with fixed sleep between retries + policyMap.put(ex, createRetryPolicy(maxRetryCount, retryInterval)); + } + } + // Default retry policy + policyMap.put(Exception.class, createRetryPolicy( + maxRetryCount, retryInterval)); + return policyMap; + } } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java index 8ab773817a..c4c2524fea 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyOutputStream.java @@ -22,8 +22,7 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.hdds.protocol.DatanodeDetails; -import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos - .ChecksumType; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException; import org.apache.hadoop.hdds.scm.storage.BufferPool; @@ -52,7 +51,10 @@ import java.util.List; import java.util.Collection; import java.util.ListIterator; +import java.util.Map; import java.util.concurrent.TimeoutException; +import java.util.function.Function; +import java.util.stream.Collectors; /** * Maintaining a list of BlockInputStream. Write based on offset. @@ -95,7 +97,7 @@ enum StreamAction { private OmMultipartCommitUploadPartInfo commitUploadPartInfo; private FileEncryptionInfo feInfo; private ExcludeList excludeList; - private final RetryPolicy retryPolicy; + private final Map, RetryPolicy> retryPolicyMap; private int retryCount; private long offset; /** @@ -121,7 +123,10 @@ public KeyOutputStream() { OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT); this.bytesPerChecksum = OzoneConfigKeys .OZONE_CLIENT_BYTES_PER_CHECKSUM_DEFAULT_BYTES; // Default is 1MB - this.retryPolicy = RetryPolicies.TRY_ONCE_THEN_FAIL; + this.retryPolicyMap = OzoneClientUtils.getExceptionList() + .stream() + .collect(Collectors.toMap(Function.identity(), + e -> RetryPolicies.TRY_ONCE_THEN_FAIL)); retryCount = 0; offset = 0; } @@ -200,8 +205,8 @@ public KeyOutputStream(OpenKeySession handler, this.bufferPool = new BufferPool(chunkSize, (int)streamBufferMaxSize / chunkSize); this.excludeList = new ExcludeList(); - this.retryPolicy = OzoneClientUtils.createRetryPolicy(maxRetryCount, - retryInterval); + this.retryPolicyMap = OzoneClientUtils.getRetryPolicyByException( + maxRetryCount, retryInterval); this.retryCount = 0; } @@ -502,10 +507,14 @@ private void markStreamClosed() { } private void handleRetry(IOException exception, long len) throws IOException { + RetryPolicy retryPolicy = + retryPolicyMap.get(checkForException(exception).getClass()); + if (retryPolicy == null) { + retryPolicy = retryPolicyMap.get(Exception.class); + } RetryPolicy.RetryAction action; try { - action = retryPolicy - .shouldRetry(exception, retryCount, 0, true); + action = retryPolicy.shouldRetry(exception, retryCount, 0, true); } catch (Exception e) { throw e instanceof IOException ? (IOException) e : new IOException(e); }