HDDS-464. Fix TestCloseContainerHandlingByClient.

Contributed by Lokesh Jain.
This commit is contained in:
Anu Engineer 2018-09-18 10:44:56 -07:00
parent 6ff509c32a
commit f938925bde
7 changed files with 100 additions and 8 deletions

View File

@ -208,6 +208,10 @@ private CompletableFuture<RaftClientReply> sendRequestAsync(
public ContainerCommandResponseProto sendCommand(
ContainerCommandRequestProto request) throws IOException {
final RaftClientReply reply = sendRequest(request);
if (reply == null) {
throw new IOException(
String.format("Could not execute the request %s", request));
}
Preconditions.checkState(reply.isSuccess());
return ContainerCommandResponseProto.parseFrom(
reply.getMessage().getContent());

View File

@ -75,6 +75,19 @@ public final class ScmConfigKeys {
public static final TimeDuration
DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT =
TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
public static final String DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY =
"dfs.ratis.client.request.max.retries";
public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT = 180;
public static final String DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY =
"dfs.ratis.client.request.retry.interval";
public static final TimeDuration
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT =
TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
"dfs.ratis.server.retry-cache.timeout.duration";
public static final TimeDuration
DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT =
TimeDuration.valueOf(600000, TimeUnit.MILLISECONDS);
public static final String DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY =
"dfs.ratis.server.request.timeout.duration";
public static final TimeDuration

View File

@ -237,6 +237,20 @@ public final class OzoneConfigKeys {
public static final TimeDuration
DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT =
ScmConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT;
public static final String DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY =
ScmConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY;
public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT =
ScmConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT;
public static final String DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY =
ScmConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY;
public static final TimeDuration
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT =
ScmConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT;
public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
ScmConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY;
public static final TimeDuration
DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT =
ScmConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT;
public static final String DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY =
ScmConfigKeys.DFS_RATIS_SERVER_REQUEST_TIMEOUT_DURATION_KEY;
public static final TimeDuration

View File

@ -34,6 +34,7 @@
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.shaded.com.google.protobuf.ByteString;
import org.apache.ratis.shaded.proto.RaftProtos;
import org.apache.ratis.util.Preconditions;
import org.apache.ratis.util.SizeInBytes;
import org.apache.ratis.util.TimeDuration;
import org.slf4j.Logger;
@ -48,6 +49,9 @@
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT;
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY;
/**
* Ratis helper methods.
*/
@ -162,12 +166,38 @@ static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
static RetryPolicy createRetryPolicy(Configuration conf) {
int maxRetryCount =
conf.getInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, OzoneConfigKeys.
OZONE_CLIENT_MAX_RETRIES_DEFAULT);
conf.getInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY,
OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT);
long retryInterval = conf.getTimeDuration(OzoneConfigKeys.
OZONE_CLIENT_RETRY_INTERVAL, OzoneConfigKeys.
OZONE_CLIENT_RETRY_INTERVAL_DEFAULT,
TimeUnit.MILLISECONDS.MILLISECONDS);
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY, OzoneConfigKeys.
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT
.toInt(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
long leaderElectionTimeout = conf.getTimeDuration(
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY,
DFS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT
.toInt(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
long clientRequestTimeout = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
.toInt(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
long retryCacheTimeout = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.toInt(TimeUnit.MILLISECONDS), TimeUnit.MILLISECONDS);
Preconditions
.assertTrue(maxRetryCount * retryInterval > 5 * leaderElectionTimeout,
"Please make sure dfs.ratis.client.request.max.retries * "
+ "dfs.ratis.client.request.retry.interval > "
+ "5 * dfs.ratis.leader.election.minimum.timeout.duration");
Preconditions.assertTrue(
maxRetryCount * (retryInterval + clientRequestTimeout)
< retryCacheTimeout,
"Please make sure "
+ "(dfs.ratis.client.request.max.retries * "
+ "(dfs.ratis.client.request.retry.interval + "
+ "dfs.ratis.client.request.timeout.duration)) "
+ "< dfs.ratis.server.retry-cache.timeout.duration");
TimeDuration sleepDuration =
TimeDuration.valueOf(retryInterval, TimeUnit.MILLISECONDS);
RetryPolicy retryPolicy = RetryPolicies

View File

@ -157,6 +157,25 @@
<tag>OZONE, RATIS, MANAGEMENT</tag>
<description>The timeout duration for ratis client request.</description>
</property>
<property>
<name>dfs.ratis.client.request.max.retries</name>
<value>180</value>
<tag>OZONE, RATIS, MANAGEMENT</tag>
<description>Number of retries for ratis client request.</description>
</property>
<property>
<name>dfs.ratis.client.request.retry.interval</name>
<value>100ms</value>
<tag>OZONE, RATIS, MANAGEMENT</tag>
<description>Interval between successive retries for a ratis client request.
</description>
</property>
<property>
<name>dfs.ratis.server.retry-cache.timeout.duration</name>
<value>600000ms</value>
<tag>OZONE, RATIS, MANAGEMENT</tag>
<description>Retry Cache entry timeout for ratis server.</description>
</property>
<property>
<name>dfs.ratis.server.request.timeout.duration</name>
<value>3s</value>

View File

@ -182,6 +182,19 @@ private RaftProperties newRaftProperties(Configuration conf,
RaftServerConfigKeys.Rpc
.setRequestTimeout(properties, serverRequestTimeout);
// set timeout for a retry cache entry
timeUnit =
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getUnit();
duration = conf.getTimeDuration(
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY,
OzoneConfigKeys.DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT
.getDuration(), timeUnit);
final TimeDuration retryCacheTimeout =
TimeDuration.valueOf(duration, timeUnit);
RaftServerConfigKeys.RetryCache
.setExpiryTime(properties, retryCacheTimeout);
// Set the ratis leader election timeout
TimeUnit leaderElectionMinTimeoutUnit =
OzoneConfigKeys.

View File

@ -55,7 +55,6 @@
import java.util.Arrays;
import java.util.List;
import java.util.UUID;
import java.util.Random;
/**
* Tests Close Container Exception handling by Ozone Client.
@ -83,9 +82,9 @@ public class TestCloseContainerHandlingByClient {
@BeforeClass
public static void init() throws Exception {
conf = new OzoneConfiguration();
// generate a no between 1 to 10
maxRetries = new Random().nextInt(10);
maxRetries = 100;
conf.setInt(OzoneConfigKeys.OZONE_CLIENT_MAX_RETRIES, maxRetries);
conf.set(OzoneConfigKeys.OZONE_CLIENT_RETRY_INTERVAL, "200ms");
chunkSize = (int) OzoneConsts.MB;
blockSize = 4 * chunkSize;
conf.setInt(ScmConfigKeys.OZONE_SCM_CHUNK_SIZE_KEY, chunkSize);