HDDS-1098. Introduce Retry Policy in Ozone Client. Contributed by Shashikant Banerjee.

This commit is contained in:
Shashikant Banerjee 2019-03-15 15:40:59 +05:30
parent ba50a36a3e
commit 155ab6d5d8
13 changed files with 205 additions and 104 deletions

View File

@ -47,6 +47,7 @@ import org.apache.ratis.protocol.RaftClientReply;
import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -74,6 +75,8 @@ public final class XceiverClientRatis extends XceiverClientSpi {
final String rpcType = ozoneConf final String rpcType = ozoneConf
.get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY, .get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT); ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
final TimeDuration clientRequestTimeout =
RatisHelper.getClientRequestTimeout(ozoneConf);
final int maxOutstandingRequests = final int maxOutstandingRequests =
HddsClientUtils.getMaxOutstandingRequests(ozoneConf); HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf); final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
@ -81,7 +84,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
SecurityConfig(ozoneConf)); SecurityConfig(ozoneConf));
return new XceiverClientRatis(pipeline, return new XceiverClientRatis(pipeline,
SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests, SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests,
retryPolicy, tlsConfig); retryPolicy, tlsConfig, clientRequestTimeout);
} }
private final Pipeline pipeline; private final Pipeline pipeline;
@ -90,6 +93,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
private final int maxOutstandingRequests; private final int maxOutstandingRequests;
private final RetryPolicy retryPolicy; private final RetryPolicy retryPolicy;
private final GrpcTlsConfig tlsConfig; private final GrpcTlsConfig tlsConfig;
private final TimeDuration clientRequestTimeout;
// Map to track commit index at every server // Map to track commit index at every server
private final ConcurrentHashMap<UUID, Long> commitInfoMap; private final ConcurrentHashMap<UUID, Long> commitInfoMap;
@ -102,7 +106,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
*/ */
private XceiverClientRatis(Pipeline pipeline, RpcType rpcType, private XceiverClientRatis(Pipeline pipeline, RpcType rpcType,
int maxOutStandingChunks, RetryPolicy retryPolicy, int maxOutStandingChunks, RetryPolicy retryPolicy,
GrpcTlsConfig tlsConfig) { GrpcTlsConfig tlsConfig, TimeDuration timeout) {
super(); super();
this.pipeline = pipeline; this.pipeline = pipeline;
this.rpcType = rpcType; this.rpcType = rpcType;
@ -111,6 +115,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
commitInfoMap = new ConcurrentHashMap<>(); commitInfoMap = new ConcurrentHashMap<>();
watchClient = null; watchClient = null;
this.tlsConfig = tlsConfig; this.tlsConfig = tlsConfig;
this.clientRequestTimeout = timeout;
} }
private void updateCommitInfosMap( private void updateCommitInfosMap(
@ -160,7 +165,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
// requests to be handled by raft client // requests to be handled by raft client
if (!client.compareAndSet(null, if (!client.compareAndSet(null,
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy, RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
maxOutstandingRequests, tlsConfig))) { maxOutstandingRequests, tlsConfig, clientRequestTimeout))) {
throw new IllegalStateException("Client is already connected."); throw new IllegalStateException("Client is already connected.");
} }
} }
@ -243,7 +248,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
if (watchClient == null) { if (watchClient == null) {
watchClient = watchClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy, RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
maxOutstandingRequests, tlsConfig); maxOutstandingRequests, tlsConfig, clientRequestTimeout);
} }
CompletableFuture<RaftClientReply> replyFuture = watchClient CompletableFuture<RaftClientReply> replyFuture = watchClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED); .sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
@ -260,9 +265,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
// TODO : need to remove the code to create the new RaftClient instance // TODO : need to remove the code to create the new RaftClient instance
// here once the watch request bypassing sliding window in Raft Client // here once the watch request bypassing sliding window in Raft Client
// gets fixed. // gets fixed.
watchClient = RatisHelper watchClient =
.newRaftClient(rpcType, getPipeline(), retryPolicy, RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
maxOutstandingRequests, tlsConfig); maxOutstandingRequests, tlsConfig, clientRequestTimeout);
reply = watchClient reply = watchClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) .sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
.get(timeout, TimeUnit.MILLISECONDS); .get(timeout, TimeUnit.MILLISECONDS);

View File

@ -121,12 +121,12 @@ public final class ScmConfigKeys {
TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS); TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
public static final String DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY = public static final String DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY =
"dfs.ratis.client.request.max.retries"; "dfs.ratis.client.request.max.retries";
public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT = 180; public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT = 20;
public static final String DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY = public static final String DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY =
"dfs.ratis.client.request.retry.interval"; "dfs.ratis.client.request.retry.interval";
public static final TimeDuration public static final TimeDuration
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT = DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT =
TimeDuration.valueOf(100, TimeUnit.MILLISECONDS); TimeDuration.valueOf(500, TimeUnit.MILLISECONDS);
public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY = public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
"dfs.ratis.server.retry-cache.timeout.duration"; "dfs.ratis.server.retry-cache.timeout.duration";
public static final TimeDuration public static final TimeDuration

View File

@ -133,6 +133,11 @@ public final class OzoneConfigKeys {
public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT = public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT =
"30s"; "30s";
public static final String OZONE_CLIENT_MAX_RETRIES =
"ozone.client.max.retries";
public static final int OZONE_CLIENT_MAX_RETRIES_DEFAULT = 5;
// This defines the overall connection limit for the connection pool used in // This defines the overall connection limit for the connection pool used in
// RestClient. // RestClient.
public static final String OZONE_REST_CLIENT_HTTP_CONNECTION_MAX = public static final String OZONE_REST_CLIENT_HTTP_CONNECTION_MAX =

View File

@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.ratis.client.RaftClient; import org.apache.ratis.client.RaftClient;
import org.apache.ratis.client.RaftClientConfigKeys;
import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcConfigKeys;
import org.apache.ratis.grpc.GrpcFactory; import org.apache.ratis.grpc.GrpcFactory;
@ -134,33 +135,51 @@ public interface RatisHelper {
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline, static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
RetryPolicy retryPolicy, int maxOutStandingRequest, RetryPolicy retryPolicy, int maxOutStandingRequest,
GrpcTlsConfig tlsConfig) throws IOException { GrpcTlsConfig tlsConfig, TimeDuration timeout) throws IOException {
return newRaftClient(rpcType, toRaftPeerId(pipeline.getFirstNode()), return newRaftClient(rpcType, toRaftPeerId(pipeline.getFirstNode()),
newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()), newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()),
pipeline.getNodes()), retryPolicy, maxOutStandingRequest, tlsConfig); pipeline.getNodes()), retryPolicy, maxOutStandingRequest, tlsConfig,
timeout);
}
static TimeDuration getClientRequestTimeout(Configuration conf) {
// Set the client requestTimeout
final TimeUnit timeUnit =
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
final long duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration clientRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
return clientRequestTimeout;
} }
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader, static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
RetryPolicy retryPolicy, int maxOutstandingRequests, RetryPolicy retryPolicy, int maxOutstandingRequests,
GrpcTlsConfig tlsConfig) { GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
return newRaftClient(rpcType, leader.getId(), return newRaftClient(rpcType, leader.getId(),
newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy, newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
maxOutstandingRequests, tlsConfig); maxOutstandingRequests, tlsConfig, clientRequestTimeout);
} }
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader, static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
RetryPolicy retryPolicy, int maxOutstandingRequests) { RetryPolicy retryPolicy, int maxOutstandingRequests,
TimeDuration clientRequestTimeout) {
return newRaftClient(rpcType, leader.getId(), return newRaftClient(rpcType, leader.getId(),
newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy, newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
maxOutstandingRequests, null); maxOutstandingRequests, null, clientRequestTimeout);
} }
static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader, static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest, RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest,
GrpcTlsConfig tlsConfig) { GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group); LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
final RaftProperties properties = new RaftProperties(); final RaftProperties properties = new RaftProperties();
RaftConfigKeys.Rpc.setType(properties, rpcType); RaftConfigKeys.Rpc.setType(properties, rpcType);
RaftClientConfigKeys.Rpc
.setRequestTimeout(properties, clientRequestTimeout);
GrpcConfigKeys.setMessageSizeMax(properties, GrpcConfigKeys.setMessageSizeMax(properties,
SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)); SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE));

View File

@ -231,17 +231,19 @@
<name>dfs.ratis.client.request.timeout.duration</name> <name>dfs.ratis.client.request.timeout.duration</name>
<value>3s</value> <value>3s</value>
<tag>OZONE, RATIS, MANAGEMENT</tag> <tag>OZONE, RATIS, MANAGEMENT</tag>
<description>The timeout duration for ratis client request.</description> <description>The timeout duration for ratis client request.It should be
set greater than leader election timeout in Ratis.
</description>
</property> </property>
<property> <property>
<name>dfs.ratis.client.request.max.retries</name> <name>dfs.ratis.client.request.max.retries</name>
<value>180</value> <value>20</value>
<tag>OZONE, RATIS, MANAGEMENT</tag> <tag>OZONE, RATIS, MANAGEMENT</tag>
<description>Number of retries for ratis client request.</description> <description>Number of retries for ratis client request.</description>
</property> </property>
<property> <property>
<name>dfs.ratis.client.request.retry.interval</name> <name>dfs.ratis.client.request.retry.interval</name>
<value>100ms</value> <value>500ms</value>
<tag>OZONE, RATIS, MANAGEMENT</tag> <tag>OZONE, RATIS, MANAGEMENT</tag>
<description>Interval between successive retries for a ratis client request. <description>Interval between successive retries for a ratis client request.
</description> </description>
@ -417,6 +419,14 @@
a particular request getting replayed to all servers. a particular request getting replayed to all servers.
</description> </description>
</property> </property>
<property>
<name>ozone.client.max.retries</name>
<value>5</value>
<tag>OZONE, CLIENT</tag>
<description>Maximum number of retries by Ozone Client on encountering
exception while writing a key.
</description>
</property>
<property> <property>
<name>ozone.client.protocol</name> <name>ozone.client.protocol</name>
<value>org.apache.hadoop.ozone.client.rpc.RpcClient</value> <value>org.apache.hadoop.ozone.client.rpc.RpcClient</value>

View File

@ -44,7 +44,6 @@ import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
import io.opentracing.Scope; import io.opentracing.Scope;
import org.apache.ratis.RaftConfigKeys; import org.apache.ratis.RaftConfigKeys;
import org.apache.ratis.RatisHelper; import org.apache.ratis.RatisHelper;
import org.apache.ratis.client.RaftClientConfigKeys;
import org.apache.ratis.conf.RaftProperties; import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.grpc.GrpcConfigKeys; import org.apache.ratis.grpc.GrpcConfigKeys;
import org.apache.ratis.grpc.GrpcFactory; import org.apache.ratis.grpc.GrpcFactory;
@ -176,7 +175,7 @@ public final class XceiverServerRatis extends XceiverServer {
setRaftSegmentPreallocatedSize(conf, properties); setRaftSegmentPreallocatedSize(conf, properties);
// Set max write buffer size, which is the scm chunk size // Set max write buffer size, which is the scm chunk size
final int maxChunkSize = setMaxWriteBuffer(conf, properties); final int maxChunkSize = setMaxWriteBuffer(properties);
TimeUnit timeUnit; TimeUnit timeUnit;
long duration; long duration;
@ -329,23 +328,10 @@ public final class XceiverServerRatis extends XceiverServer {
.setRequestTimeout(properties, serverRequestTimeout); .setRequestTimeout(properties, serverRequestTimeout);
} }
private int setMaxWriteBuffer(Configuration conf, RaftProperties properties) { private int setMaxWriteBuffer(RaftProperties properties) {
final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE; final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
RaftServerConfigKeys.Log.setWriteBufferSize(properties, RaftServerConfigKeys.Log.setWriteBufferSize(properties,
SizeInBytes.valueOf(maxChunkSize)); SizeInBytes.valueOf(maxChunkSize));
// Set the client requestTimeout
TimeUnit timeUnit =
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getUnit();
long duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration clientRequestTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftClientConfigKeys.Rpc
.setRequestTimeout(properties, clientRequestTimeout);
return maxChunkSize; return maxChunkSize;
} }

View File

@ -39,6 +39,7 @@ import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.retry.RetryPolicy;
import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.util.TimeDuration;
import org.junit.AfterClass; import org.junit.AfterClass;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -49,6 +50,7 @@ import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.Random; import java.util.Random;
import java.util.UUID; import java.util.UUID;
import java.util.concurrent.TimeUnit;
/** /**
* Test cases to verify CloseContainerCommandHandler in datanode. * Test cases to verify CloseContainerCommandHandler in datanode.
@ -289,8 +291,10 @@ public class TestCloseContainerCommandHandler {
final RaftGroup group = RatisHelper.newRaftGroup(raftGroupId, final RaftGroup group = RatisHelper.newRaftGroup(raftGroupId,
Collections.singleton(datanodeDetails)); Collections.singleton(datanodeDetails));
final int maxOutstandingRequests = 100; final int maxOutstandingRequests = 100;
final RaftClient client = RatisHelper.newRaftClient(SupportedRpcType.GRPC, final RaftClient client = RatisHelper
peer, retryPolicy, maxOutstandingRequests, null); .newRaftClient(SupportedRpcType.GRPC, peer, retryPolicy,
maxOutstandingRequests,
TimeDuration.valueOf(3, TimeUnit.SECONDS));
Assert.assertTrue(client.groupAdd(group, peer.getId()).isSuccess()); Assert.assertTrue(client.groupAdd(group, peer.getId()).isSuccess());
Thread.sleep(2000); Thread.sleep(2000);
final ContainerID containerId = ContainerID.valueof( final ContainerID containerId = ContainerID.valueof(

View File

@ -32,6 +32,7 @@ import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.retry.RetryPolicy;
import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.util.TimeDuration;
import org.apache.ratis.util.function.CheckedBiConsumer; import org.apache.ratis.util.function.CheckedBiConsumer;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -116,9 +117,11 @@ final class RatisPipelineUtils {
HddsClientUtils.getMaxOutstandingRequests(ozoneConf); HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig( final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(
new SecurityConfig(ozoneConf)); new SecurityConfig(ozoneConf));
final TimeDuration requestTimeout =
RatisHelper.getClientRequestTimeout(ozoneConf);
RaftClient client = RatisHelper RaftClient client = RatisHelper
.newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
retryPolicy, maxOutstandingRequests, tlsConfig); retryPolicy, maxOutstandingRequests, tlsConfig, requestTimeout);
client client
.groupRemove(RaftGroupId.valueOf(pipelineID.getId()), true, p.getId()); .groupRemove(RaftGroupId.valueOf(pipelineID.getId()), true, p.getId());
} }
@ -141,12 +144,13 @@ final class RatisPipelineUtils {
HddsClientUtils.getMaxOutstandingRequests(ozoneConf); HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
SecurityConfig(ozoneConf)); SecurityConfig(ozoneConf));
final TimeDuration requestTimeout =
RatisHelper.getClientRequestTimeout(ozoneConf);
datanodes.parallelStream().forEach(d -> { datanodes.parallelStream().forEach(d -> {
final RaftPeer p = RatisHelper.toRaftPeer(d); final RaftPeer p = RatisHelper.toRaftPeer(d);
try (RaftClient client = RatisHelper try (RaftClient client = RatisHelper
.newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p, .newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
retryPolicy, maxOutstandingRequests, tlsConfig)) { retryPolicy, maxOutstandingRequests, tlsConfig, requestTimeout)) {
rpc.accept(client, p); rpc.accept(client, p);
} catch (IOException ioe) { } catch (IOException ioe) {
String errMsg = String errMsg =

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.client;
import org.apache.hadoop.hdds.client.OzoneQuota; import org.apache.hadoop.hdds.client.OzoneQuota;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils; import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException; 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.OzoneConsts;
import org.apache.hadoop.ozone.client.rest.response.*; import org.apache.hadoop.ozone.client.rest.response.*;
import org.apache.ratis.protocol.AlreadyClosedException; import org.apache.ratis.protocol.AlreadyClosedException;
@ -27,6 +29,7 @@ import org.apache.ratis.protocol.RaftRetryFailureException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
/** A utility class for OzoneClient. */ /** A utility class for OzoneClient. */
@ -122,6 +125,14 @@ public final class OzoneClientUtils {
return keyInfo; return keyInfo;
} }
public static RetryPolicy createRetryPolicy(int maxRetryCount) {
// just retry without sleep
RetryPolicy retryPolicy = RetryPolicies
.retryUpToMaximumCountWithFixedSleep(maxRetryCount, 0,
TimeUnit.MILLISECONDS);
return retryPolicy;
}
public static List<Class<? extends Exception>> getExceptionList() { public static List<Class<? extends Exception>> getExceptionList() {
return EXCEPTION_LIST; return EXCEPTION_LIST;
} }

View File

@ -32,6 +32,8 @@ import org.apache.hadoop.hdds.scm.storage.BufferPool;
import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList; import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID; import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.io.retry.RetryPolicies;
import org.apache.hadoop.io.retry.RetryPolicy;
import org.apache.hadoop.ozone.client.OzoneClientUtils; import org.apache.hadoop.ozone.client.OzoneClientUtils;
import org.apache.hadoop.ozone.om.helpers.*; import org.apache.hadoop.ozone.om.helpers.*;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
@ -45,6 +47,7 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.io.InterruptedIOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
@ -87,6 +90,8 @@ public class KeyOutputStream extends OutputStream {
private OmMultipartCommitUploadPartInfo commitUploadPartInfo; private OmMultipartCommitUploadPartInfo commitUploadPartInfo;
private FileEncryptionInfo feInfo; private FileEncryptionInfo feInfo;
private ExcludeList excludeList; private ExcludeList excludeList;
private final RetryPolicy retryPolicy;
private int retryCount;
/** /**
* A constructor for testing purpose only. * A constructor for testing purpose only.
*/ */
@ -111,6 +116,8 @@ public class KeyOutputStream extends OutputStream {
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT); OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT);
this.bytesPerChecksum = OzoneConfigKeys this.bytesPerChecksum = OzoneConfigKeys
.OZONE_CLIENT_BYTES_PER_CHECKSUM_DEFAULT_BYTES; // Default is 1MB .OZONE_CLIENT_BYTES_PER_CHECKSUM_DEFAULT_BYTES; // Default is 1MB
this.retryPolicy = RetryPolicies.TRY_ONCE_THEN_FAIL;
retryCount = 0;
} }
@VisibleForTesting @VisibleForTesting
@ -147,7 +154,7 @@ public class KeyOutputStream extends OutputStream {
String requestId, ReplicationFactor factor, ReplicationType type, String requestId, ReplicationFactor factor, ReplicationType type,
long bufferFlushSize, long bufferMaxSize, long size, long watchTimeout, long bufferFlushSize, long bufferMaxSize, long size, long watchTimeout,
ChecksumType checksumType, int bytesPerChecksum, ChecksumType checksumType, int bytesPerChecksum,
String uploadID, int partNumber, boolean isMultipart) { String uploadID, int partNumber, boolean isMultipart, int maxRetryCount) {
this.streamEntries = new ArrayList<>(); this.streamEntries = new ArrayList<>();
this.currentStreamIndex = 0; this.currentStreamIndex = 0;
this.omClient = omClient; this.omClient = omClient;
@ -183,6 +190,8 @@ public class KeyOutputStream extends OutputStream {
this.bufferPool = this.bufferPool =
new BufferPool(chunkSize, (int)streamBufferMaxSize / chunkSize); new BufferPool(chunkSize, (int)streamBufferMaxSize / chunkSize);
this.excludeList = new ExcludeList(); this.excludeList = new ExcludeList();
this.retryPolicy = OzoneClientUtils.createRetryPolicy(maxRetryCount);
this.retryCount = 0;
} }
/** /**
@ -308,8 +317,6 @@ public class KeyOutputStream extends OutputStream {
current.write(b, off, writeLen); current.write(b, off, writeLen);
} }
} catch (IOException ioe) { } catch (IOException ioe) {
Throwable t = checkForException(ioe);
if (t != null) {
// for the current iteration, totalDataWritten - currentPos gives the // for the current iteration, totalDataWritten - currentPos gives the
// amount of data already written to the buffer // amount of data already written to the buffer
@ -321,10 +328,7 @@ public class KeyOutputStream extends OutputStream {
writeLen = retry ? (int) len : writeLen = retry ? (int) len :
(int) (current.getWrittenDataLength() - currentPos); (int) (current.getWrittenDataLength() - currentPos);
LOG.debug("writeLen {}, total len {}", writeLen, len); LOG.debug("writeLen {}, total len {}", writeLen, len);
handleException(current, currentStreamIndex, t); handleException(current, currentStreamIndex, ioe);
} else {
throw ioe;
}
} }
if (current.getRemaining() <= 0) { if (current.getRemaining() <= 0) {
// since the current block is already written close the stream. // since the current block is already written close the stream.
@ -390,7 +394,8 @@ public class KeyOutputStream extends OutputStream {
* @throws IOException Throws IOException if Write fails * @throws IOException Throws IOException if Write fails
*/ */
private void handleException(BlockOutputStreamEntry streamEntry, private void handleException(BlockOutputStreamEntry streamEntry,
int streamIndex, Throwable exception) throws IOException { int streamIndex, IOException exception) throws IOException {
Throwable t = checkForException(exception);
boolean retryFailure = checkForRetryFailure(exception); boolean retryFailure = checkForRetryFailure(exception);
boolean closedContainerException = false; boolean closedContainerException = false;
if (!retryFailure) { if (!retryFailure) {
@ -413,9 +418,9 @@ public class KeyOutputStream extends OutputStream {
if (!failedServers.isEmpty()) { if (!failedServers.isEmpty()) {
excludeList.addDatanodes(failedServers); excludeList.addDatanodes(failedServers);
} }
if (checkIfContainerIsClosed(exception)) { if (checkIfContainerIsClosed(t)) {
excludeList.addConatinerId(ContainerID.valueof(containerId)); excludeList.addConatinerId(ContainerID.valueof(containerId));
} else if (retryFailure || exception instanceof TimeoutException) { } else if (retryFailure || t instanceof TimeoutException) {
pipelineId = streamEntry.getPipeline().getId(); pipelineId = streamEntry.getPipeline().getId();
excludeList.addPipeline(pipelineId); excludeList.addPipeline(pipelineId);
} }
@ -425,7 +430,7 @@ public class KeyOutputStream extends OutputStream {
// If the data is still cached in the underlying stream, we need to // If the data is still cached in the underlying stream, we need to
// allocate new block and write this data in the datanode. // allocate new block and write this data in the datanode.
currentStreamIndex += 1; currentStreamIndex += 1;
handleWrite(null, 0, bufferedDataLen, true); handleRetry(exception, bufferedDataLen);
} }
if (totalSuccessfulFlushedData == 0) { if (totalSuccessfulFlushedData == 0) {
streamEntries.remove(streamIndex); streamEntries.remove(streamIndex);
@ -448,6 +453,43 @@ public class KeyOutputStream extends OutputStream {
} }
} }
private void handleRetry(IOException exception, long len) throws IOException {
RetryPolicy.RetryAction action;
try {
action = retryPolicy
.shouldRetry(exception, retryCount, 0, true);
} catch (Exception e) {
throw e instanceof IOException ? (IOException) e : new IOException(e);
}
if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
if (action.reason != null) {
LOG.error("Retry request failed. " + action.reason,
exception);
}
throw exception;
}
// Throw the exception if the thread is interrupted
if (Thread.currentThread().isInterrupted()) {
LOG.warn("Interrupted while trying for retry");
throw exception;
}
Preconditions.checkArgument(
action.action == RetryPolicy.RetryAction.RetryDecision.RETRY);
if (action.delayMillis > 0) {
try {
Thread.sleep(action.delayMillis);
} catch (InterruptedException e) {
throw (IOException) new InterruptedIOException(
"Interrupted: action=" + action + ", retry policy=" + retryPolicy)
.initCause(e);
}
}
retryCount++;
LOG.trace("Retrying Write request. Already tried "
+ retryCount + " time(s); retry policy is " + retryPolicy);
handleWrite(null, 0, len, true);
}
/** /**
* Checks if the provided exception signifies retry failure in ratis client. * Checks if the provided exception signifies retry failure in ratis client.
* In case of retry failure, ratis client throws RaftRetryFailureException * In case of retry failure, ratis client throws RaftRetryFailureException
@ -462,7 +504,7 @@ public class KeyOutputStream extends OutputStream {
return t instanceof ContainerNotOpenException; return t instanceof ContainerNotOpenException;
} }
private Throwable checkForException(IOException ioe) { private Throwable checkForException(IOException ioe) throws IOException {
Throwable t = ioe.getCause(); Throwable t = ioe.getCause();
while (t != null) { while (t != null) {
for (Class<? extends Exception> cls : OzoneClientUtils for (Class<? extends Exception> cls : OzoneClientUtils
@ -473,7 +515,7 @@ public class KeyOutputStream extends OutputStream {
} }
t = t.getCause(); t = t.getCause();
} }
return null; throw ioe;
} }
private long getKeyLength() { private long getKeyLength() {
@ -512,6 +554,7 @@ public class KeyOutputStream extends OutputStream {
if (streamEntries.size() == 0) { if (streamEntries.size() == 0) {
return; return;
} }
while (true) {
int size = streamEntries.size(); int size = streamEntries.size();
int streamIndex = int streamIndex =
currentStreamIndex >= size ? size - 1 : currentStreamIndex; currentStreamIndex >= size ? size - 1 : currentStreamIndex;
@ -519,7 +562,6 @@ public class KeyOutputStream extends OutputStream {
if (entry != null) { if (entry != null) {
try { try {
Collection<DatanodeDetails> failedServers = entry.getFailedServers(); Collection<DatanodeDetails> failedServers = entry.getFailedServers();
// failed servers can be null in case there is no data written in the // failed servers can be null in case there is no data written in the
// stream // stream
if (failedServers != null && !failedServers.isEmpty()) { if (failedServers != null && !failedServers.isEmpty()) {
@ -531,17 +573,11 @@ public class KeyOutputStream extends OutputStream {
entry.flush(); entry.flush();
} }
} catch (IOException ioe) { } catch (IOException ioe) {
Throwable t = checkForException(ioe); handleException(entry, streamIndex, ioe);
if (t != null) { continue;
// This call will allocate a new streamEntry and write the Data.
// Close needs to be retried on the newly allocated streamEntry as
// as well.
handleException(entry, streamIndex, t);
handleFlushOrClose(close);
} else {
throw ioe;
} }
} }
break;
} }
} }
@ -616,6 +652,7 @@ public class KeyOutputStream extends OutputStream {
private String multipartUploadID; private String multipartUploadID;
private int multipartNumber; private int multipartNumber;
private boolean isMultipartKey; private boolean isMultipartKey;
private int maxRetryCount;
public Builder setMultipartUploadID(String uploadID) { public Builder setMultipartUploadID(String uploadID) {
@ -704,11 +741,17 @@ public class KeyOutputStream extends OutputStream {
return this; return this;
} }
public Builder setMaxRetryCount(int maxCount) {
this.maxRetryCount = maxCount;
return this;
}
public KeyOutputStream build() throws IOException { public KeyOutputStream build() throws IOException {
return new KeyOutputStream(openHandler, xceiverManager, scmClient, return new KeyOutputStream(openHandler, xceiverManager, scmClient,
omClient, chunkSize, requestID, factor, type, streamBufferFlushSize, omClient, chunkSize, requestID, factor, type, streamBufferFlushSize,
streamBufferMaxSize, blockSize, watchTimeout, checksumType, streamBufferMaxSize, blockSize, watchTimeout, checksumType,
bytesPerChecksum, multipartUploadID, multipartNumber, isMultipartKey); bytesPerChecksum, multipartUploadID, multipartNumber, isMultipartKey,
maxRetryCount);
} }
} }

View File

@ -123,6 +123,7 @@ public class RpcClient implements ClientProtocol {
private final long blockSize; private final long blockSize;
private final long watchTimeout; private final long watchTimeout;
private final ClientId clientId = ClientId.randomId(); private final ClientId clientId = ClientId.randomId();
private final int maxRetryCount;
/** /**
* Creates RpcClient instance with the given configuration. * Creates RpcClient instance with the given configuration.
@ -205,6 +206,9 @@ public class RpcClient implements ClientProtocol {
this.verifyChecksum = this.verifyChecksum =
conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM, conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM,
OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT); OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT);
maxRetryCount =
conf.getInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, OzoneConfigKeys.
OZONE_CLIENT_MAX_RETRIES_DEFAULT);
} }
private InetSocketAddress getScmAddressForClient() throws IOException { private InetSocketAddress getScmAddressForClient() throws IOException {
@ -594,7 +598,7 @@ public class RpcClient implements ClientProtocol {
.build(); .build();
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
KeyOutputStream groupOutputStream = KeyOutputStream keyOutputStream =
new KeyOutputStream.Builder() new KeyOutputStream.Builder()
.setHandler(openKey) .setHandler(openKey)
.setXceiverClientManager(xceiverClientManager) .setXceiverClientManager(xceiverClientManager)
@ -610,20 +614,21 @@ public class RpcClient implements ClientProtocol {
.setBlockSize(blockSize) .setBlockSize(blockSize)
.setChecksumType(checksumType) .setChecksumType(checksumType)
.setBytesPerChecksum(bytesPerChecksum) .setBytesPerChecksum(bytesPerChecksum)
.setMaxRetryCount(maxRetryCount)
.build(); .build();
groupOutputStream.addPreallocateBlocks( keyOutputStream.addPreallocateBlocks(
openKey.getKeyInfo().getLatestVersionLocations(), openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion()); openKey.getOpenVersion());
final FileEncryptionInfo feInfo = groupOutputStream final FileEncryptionInfo feInfo = keyOutputStream
.getFileEncryptionInfo(); .getFileEncryptionInfo();
if (feInfo != null) { if (feInfo != null) {
KeyProvider.KeyVersion decrypted = getDEK(feInfo); KeyProvider.KeyVersion decrypted = getDEK(feInfo);
final CryptoOutputStream cryptoOut = new CryptoOutputStream( final CryptoOutputStream cryptoOut = new CryptoOutputStream(
groupOutputStream, OzoneKMSUtil.getCryptoCodec(conf, feInfo), keyOutputStream, OzoneKMSUtil.getCryptoCodec(conf, feInfo),
decrypted.getMaterial(), feInfo.getIV()); decrypted.getMaterial(), feInfo.getIV());
return new OzoneOutputStream(cryptoOut); return new OzoneOutputStream(cryptoOut);
} else { } else {
return new OzoneOutputStream(groupOutputStream); return new OzoneOutputStream(keyOutputStream);
} }
} }
@ -856,7 +861,7 @@ public class RpcClient implements ClientProtocol {
.build(); .build();
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
KeyOutputStream groupOutputStream = KeyOutputStream keyOutputStream =
new KeyOutputStream.Builder() new KeyOutputStream.Builder()
.setHandler(openKey) .setHandler(openKey)
.setXceiverClientManager(xceiverClientManager) .setXceiverClientManager(xceiverClientManager)
@ -875,11 +880,12 @@ public class RpcClient implements ClientProtocol {
.setMultipartNumber(partNumber) .setMultipartNumber(partNumber)
.setMultipartUploadID(uploadID) .setMultipartUploadID(uploadID)
.setIsMultipartKey(true) .setIsMultipartKey(true)
.setMaxRetryCount(maxRetryCount)
.build(); .build();
groupOutputStream.addPreallocateBlocks( keyOutputStream.addPreallocateBlocks(
openKey.getKeyInfo().getLatestVersionLocations(), openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion()); openKey.getOpenVersion());
return new OzoneOutputStream(groupOutputStream); return new OzoneOutputStream(keyOutputStream);
} }
@Override @Override

View File

@ -31,6 +31,7 @@ import org.apache.ratis.client.RaftClient;
import org.apache.ratis.protocol.RaftPeer; import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.rpc.RpcType; import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType; import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -126,9 +127,11 @@ public interface RatisTestHelper {
final OzoneConfiguration conf = new OzoneConfiguration(); final OzoneConfiguration conf = new OzoneConfiguration();
final int maxOutstandingRequests = final int maxOutstandingRequests =
HddsClientUtils.getMaxOutstandingRequests(conf); HddsClientUtils.getMaxOutstandingRequests(conf);
final TimeDuration requestTimeout =
RatisHelper.getClientRequestTimeout(conf);
final RaftClient client = final RaftClient client =
newRaftClient(rpc, p, RatisHelper.createRetryPolicy(conf), newRaftClient(rpc, p, RatisHelper.createRetryPolicy(conf),
maxOutstandingRequests); maxOutstandingRequests, requestTimeout);
client.groupAdd(RatisHelper.newRaftGroup(pipeline), p.getId()); client.groupAdd(RatisHelper.newRaftGroup(pipeline), p.getId());
} }
} }

View File

@ -88,6 +88,7 @@ public final class DistributedStorageHandler implements StorageHandler {
private final ChecksumType checksumType; private final ChecksumType checksumType;
private final int bytesPerChecksum; private final int bytesPerChecksum;
private final boolean verifyChecksum; private final boolean verifyChecksum;
private final int maxRetryCount;
/** /**
* Creates a new DistributedStorageHandler. * Creates a new DistributedStorageHandler.
@ -154,6 +155,9 @@ public final class DistributedStorageHandler implements StorageHandler {
this.verifyChecksum = this.verifyChecksum =
conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM, conf.getBoolean(OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM,
OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT); OzoneConfigKeys.OZONE_CLIENT_VERIFY_CHECKSUM_DEFAULT);
this.maxRetryCount =
conf.getInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, OzoneConfigKeys.
OZONE_CLIENT_MAX_RETRIES_DEFAULT);
} }
@Override @Override
@ -438,7 +442,7 @@ public final class DistributedStorageHandler implements StorageHandler {
.build(); .build();
// contact OM to allocate a block for key. // contact OM to allocate a block for key.
OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs); OpenKeySession openKey = ozoneManagerClient.openKey(keyArgs);
KeyOutputStream groupOutputStream = KeyOutputStream keyOutputStream =
new KeyOutputStream.Builder() new KeyOutputStream.Builder()
.setHandler(openKey) .setHandler(openKey)
.setXceiverClientManager(xceiverClientManager) .setXceiverClientManager(xceiverClientManager)
@ -454,11 +458,12 @@ public final class DistributedStorageHandler implements StorageHandler {
.setWatchTimeout(watchTimeout) .setWatchTimeout(watchTimeout)
.setChecksumType(checksumType) .setChecksumType(checksumType)
.setBytesPerChecksum(bytesPerChecksum) .setBytesPerChecksum(bytesPerChecksum)
.setMaxRetryCount(maxRetryCount)
.build(); .build();
groupOutputStream.addPreallocateBlocks( keyOutputStream.addPreallocateBlocks(
openKey.getKeyInfo().getLatestVersionLocations(), openKey.getKeyInfo().getLatestVersionLocations(),
openKey.getOpenVersion()); openKey.getOpenVersion());
return new OzoneOutputStream(groupOutputStream); return new OzoneOutputStream(keyOutputStream);
} }
@Override @Override