HDDS-845. Create a new raftClient instance for every watch request for Ratis. Contributed by Shashikant Banerjee.

This commit is contained in:
Shashikant Banerjee 2018-11-19 14:38:51 +05:30
parent cfb915f3df
commit 10cf5773ba
3 changed files with 33 additions and 11 deletions

View File

@ -290,7 +290,8 @@ private void reconnect(DatanodeDetails dn)
@Override @Override
public void watchForCommit(long index, long timeout) public void watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException { throws InterruptedException, ExecutionException, TimeoutException,
IOException {
// there is no notion of watch for commit index in standalone pipeline // there is no notion of watch for commit index in standalone pipeline
}; };

View File

@ -122,13 +122,17 @@ public void connect() throws Exception {
public void close() { public void close() {
final RaftClient c = client.getAndSet(null); final RaftClient c = client.getAndSet(null);
if (c != null) { if (c != null) {
closeRaftClient(c);
}
}
private void closeRaftClient(RaftClient raftClient) {
try { try {
c.close(); raftClient.close();
} catch (IOException e) { } catch (IOException e) {
throw new IllegalStateException(e); throw new IllegalStateException(e);
} }
} }
}
private RaftClient getClient() { private RaftClient getClient() {
return Objects.requireNonNull(client.get(), "client is null"); return Objects.requireNonNull(client.get(), "client is null");
@ -145,19 +149,35 @@ private CompletableFuture<RaftClientReply> sendRequestAsync(
@Override @Override
public void watchForCommit(long index, long timeout) public void watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException { throws InterruptedException, ExecutionException, TimeoutException,
// TODO: Create a new Raft client instance to watch IOException {
CompletableFuture<RaftClientReply> replyFuture = getClient() LOG.debug("commit index : {} watch timeout : {}", index, timeout);
// create a new RaftClient instance for watch request
RaftClient raftClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy);
CompletableFuture<RaftClientReply> replyFuture = raftClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED); .sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
try { try {
replyFuture.get(timeout, TimeUnit.MILLISECONDS); replyFuture.get(timeout, TimeUnit.MILLISECONDS);
} catch (TimeoutException toe) { } catch (TimeoutException toe) {
LOG.warn("3 way commit failed ", toe); LOG.warn("3 way commit failed ", toe);
getClient()
closeRaftClient(raftClient);
// 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.
raftClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy);
raftClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) .sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
.get(timeout, TimeUnit.MILLISECONDS); .get(timeout, TimeUnit.MILLISECONDS);
LOG.info("Could not commit " + index + " to all the nodes." LOG.info("Could not commit " + index + " to all the nodes."
+ "Committed by majority."); + "Committed by majority.");
} finally {
closeRaftClient(raftClient);
} }
} }
/** /**

View File

@ -126,5 +126,6 @@ public ContainerCommandResponseProto sendCommand(
public abstract HddsProtos.ReplicationType getPipelineType(); public abstract HddsProtos.ReplicationType getPipelineType();
public abstract void watchForCommit(long index, long timeout) public abstract void watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException; throws InterruptedException, ExecutionException, TimeoutException,
IOException;
} }