ソースを参照

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

Shashikant Banerjee 6 年 前
コミット
33e159bf8d

+ 2 - 26
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java

@@ -100,9 +100,6 @@ public final class XceiverClientRatis extends XceiverClientSpi {
   // 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 @@ public final class XceiverClientRatis extends XceiverClientSpi {
     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 final class XceiverClientRatis extends XceiverClientSpi {
     if (c != null) {
       closeRaftClient(c);
     }
-    if (watchClient != null) {
-      closeRaftClient(watchClient);
-    }
   }
 
   private void closeRaftClient(RaftClient raftClient) {
@@ -255,31 +248,14 @@ public final class XceiverClientRatis extends XceiverClientSpi {
       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 =