ChenSammi commented on code in PR #4128:
URL: https://github.com/apache/ozone/pull/4128#discussion_r1064379819


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/ratis/RatisHelper.java:
##########
@@ -483,4 +488,81 @@ public static void debug(ByteBuf buf, String name, Logger 
log) {
     }
     log.debug("{}: {}\n  {}", name, buf, builder);
   }
+
+
+  /**
+   * Use raft client to send admin request, transfer the leadership.
+   * If not isRandom, then use host to match the target leader.
+   * 1. Set priority and send setConfiguration request
+   * 2. Trigger transferLeadership API.
+   *
+   * @param server        the Raft server
+   * @param groupId       the Raft group Id
+   * @param targetPeerId  the target expected leader
+
+   * @throws IOException
+   */
+  public static void transferRatisLeadership(RaftServer server,
+      RaftGroupId groupId, RaftPeerId targetPeerId)
+      throws IOException {
+    RaftGroup raftGroup = server.getDivision(groupId).getGroup();
+    RaftClient raftClient = createBackoffRaftClient(raftGroup);
+    if (!raftGroup.getPeers().stream().map(RaftPeer::getId)
+        .collect(Collectors.toSet()).contains(targetPeerId)) {
+      throw new IOException("Cannot choose the target leader. The expected " +
+          "leader RaftPeerId is " + targetPeerId + " and the peers are " +
+          raftGroup.getPeers().stream().map(RaftPeer::getId)
+              .collect(Collectors.toList()) + ".");
+    }
+    LOG.info("Chosen the targetLeaderId {} to transfer leadership",
+        targetPeerId);
+
+    // Set priority
+    List<RaftPeer> peersWithNewPriorities = new ArrayList<>();
+    for (RaftPeer peer : raftGroup.getPeers()) {
+      peersWithNewPriorities.add(
+          RaftPeer.newBuilder(peer)
+              .setPriority(peer.getId().equals(targetPeerId) ? 2 : 1)
+              .build()
+      );
+    }
+    RaftClientReply reply;
+    // Set new configuration
+    reply = raftClient.admin().setConfiguration(peersWithNewPriorities);
+    if (reply.isSuccess()) {
+      LOG.info("Successfully set new priority for division: {}",
+          peersWithNewPriorities);
+    } else {
+      LOG.warn("Failed to set new priority for division: {}." +
+          " Ratis reply: {}", peersWithNewPriorities, reply);
+      throw new IOException(reply.getException());
+    }
+
+    // Trigger the transferLeadership
+    reply = raftClient.admin().transferLeadership(targetPeerId, 60000);
+    if (reply.isSuccess()) {
+      LOG.info("Successfully transferred leadership to {}.", targetPeerId);
+    } else {
+      LOG.warn("Failed to transfer leadership to {}. Ratis reply: {}",
+          targetPeerId, reply);
+      throw new IOException(reply.getException());
+    }
+  }
+
+  public static RaftClient createBackoffRaftClient(RaftGroup raftGroup) {
+    RaftProperties properties = new RaftProperties();
+    RaftClientConfigKeys.Rpc.setRequestTimeout(properties,
+        TimeDuration.valueOf(15, TimeUnit.SECONDS));
+    ExponentialBackoffRetry retryPolicy = ExponentialBackoffRetry.newBuilder()

Review Comment:
   Please use ratisClientConfig for the config.  Can refer to 
RequestTypeDependentRetryPolicyCreator#createExponentialBackoffPolicy. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to