HDDS-1349. Remove watchClient from XceiverClientRatis. Contributed by Shashikant Banerjee.

This commit is contained in:
Shashikant Banerjee 2019-04-04 16:29:38 +05:30
parent 48a58bce37
commit 33e159bf8d

View File

@ -100,9 +100,6 @@ public static XceiverClientRatis newXceiverClientRatis(
// Map to track commit index at every server
private final ConcurrentHashMap<UUID, Long> commitInfoMap;
// create a separate RaftClient for watchForCommit API
private RaftClient watchClient;
private XceiverClientMetrics metrics;
/**
@ -117,7 +114,6 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType,
this.maxOutstandingRequests = maxOutStandingChunks;
this.retryPolicy = retryPolicy;
commitInfoMap = new ConcurrentHashMap<>();
watchClient = null;
this.tlsConfig = tlsConfig;
this.clientRequestTimeout = timeout;
metrics = XceiverClientManager.getXceiverClientMetrics();
@ -187,9 +183,6 @@ public void close() {
if (c != null) {
closeRaftClient(c);
}
if (watchClient != null) {
closeRaftClient(watchClient);
}
}
private void closeRaftClient(RaftClient raftClient) {
@ -255,31 +248,14 @@ public XceiverClientReply watchForCommit(long index, long timeout)
return clientReply;
}
LOG.debug("commit index : {} watch timeout : {}", index, timeout);
// create a new RaftClient instance for watch request
if (watchClient == null) {
watchClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
maxOutstandingRequests, tlsConfig, clientRequestTimeout);
}
CompletableFuture<RaftClientReply> replyFuture = watchClient
CompletableFuture<RaftClientReply> replyFuture = getClient()
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
RaftClientReply reply;
try {
replyFuture.get(timeout, TimeUnit.MILLISECONDS);
} catch (TimeoutException toe) {
LOG.warn("3 way commit failed ", toe);
closeRaftClient(watchClient);
// generate a new raft client instance again so that next watch request
// does not get blocked for the previous one
// TODO : need to remove the code to create the new RaftClient instance
// here once the watch request bypassing sliding window in Raft Client
// gets fixed.
watchClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
maxOutstandingRequests, tlsConfig, clientRequestTimeout);
reply = watchClient
reply = getClient()
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
.get(timeout, TimeUnit.MILLISECONDS);
List<RaftProtos.CommitInfoProto> commitInfoProtoList =