RATIS-1510. Add pause leader election request related code (#593)

diff --git a/ratis-common/src/main/java/org/apache/ratis/protocol/LeaderElectionRequest.java b/ratis-common/src/main/java/org/apache/ratis/protocol/LeaderElectionRequest.java
new file mode 100644
index 0000000..bce0178
--- /dev/null
+++ b/ratis-common/src/main/java/org/apache/ratis/protocol/LeaderElectionRequest.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.protocol;
+
+import org.apache.ratis.util.JavaUtils;
+
+public final class LeaderElectionRequest extends RaftClientRequest{
+  public abstract static class Op {
+
+  }
+  public static class Pause extends Op {
+    @Override
+    public String toString() {
+      return JavaUtils.getClassSimpleName(getClass()) + ":" ;
+    }
+  }
+
+  public static class Resume extends Op {
+    @Override
+    public String toString() {
+      return JavaUtils.getClassSimpleName(getClass()) + ":" ;
+    }
+  }
+
+  public static LeaderElectionRequest newPause(ClientId clientId,
+      RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs) {
+    return new LeaderElectionRequest(clientId,
+        serverId, groupId, callId, timeoutMs,new LeaderElectionRequest.Pause());
+  }
+
+  public static LeaderElectionRequest newResume(ClientId clientId,
+      RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs) {
+    return new LeaderElectionRequest(clientId,
+        serverId, groupId, callId, timeoutMs,new LeaderElectionRequest.Resume());
+  }
+
+  private final Op op;
+
+  public LeaderElectionRequest(
+      ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs, Op op) {
+    super(clientId, serverId, groupId, callId, readRequestType(), timeoutMs);
+    this.op = op;
+  }
+
+  public Pause getPause() {
+    return op instanceof Pause ? (Pause) op: null;
+  }
+
+  public Resume getResume() {
+    return op instanceof Resume ? (Resume) op : null;
+  }
+
+
+  @Override
+  public String toString() {
+    return super.toString() + ", " + op;
+  }
+}
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 21ed539..a12314b 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -1009,6 +1009,25 @@
     return snapshotRequestHandler;
   }
 
+  CompletableFuture<RaftClientReply> setLeaderElectionAsync(LeaderElectionRequest request) throws IOException {
+    LOG.info("{} receive pauseLeaderElection {}", getMemberId(), request);
+    assertLifeCycleState(LifeCycle.States.RUNNING);
+    assertGroup(request.getRequestorId(), request.getRaftGroupId());
+
+    final LeaderElectionRequest.Pause pause = request.getPause();
+    if (pause != null) {
+      getRole().setLeaderElectionPause(true);
+      return CompletableFuture.completedFuture(newSuccessReply(request));
+    }
+    final LeaderElectionRequest.Resume resume = request.getResume();
+    if (resume != null) {
+      getRole().setLeaderElectionPause(false);
+      return CompletableFuture.completedFuture(newSuccessReply(request));
+    }
+    return JavaUtils.completeExceptionally(new UnsupportedOperationException(
+        getId() + ": Request not supported " + request));
+  }
+
   public RaftClientReply setConfiguration(SetConfigurationRequest request) throws IOException {
     return waitForReply(request, setConfigurationAsync(request));
   }
diff --git a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
index 7c81d58..7c64757 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerProxy.java
@@ -537,6 +537,10 @@
     return submitRequest(request.getRaftGroupId(), impl -> impl.takeSnapshotAsync(request));
   }
 
+  public CompletableFuture<RaftClientReply> setLeaderElectionAsync(LeaderElectionRequest request) {
+    return submitRequest(request.getRaftGroupId(), impl -> impl.setLeaderElectionAsync(request));
+  }
+
   @Override
   public GroupListReply getGroupList(GroupListRequest request) {
     return new GroupListReply(request, getGroupIds());
diff --git a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
index 4d95449..a5b48db 100644
--- a/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
+++ b/ratis-server/src/test/java/org/apache/ratis/server/impl/LeaderElectionTests.java
@@ -23,6 +23,7 @@
 import org.apache.ratis.client.RaftClient;
 import org.apache.ratis.conf.RaftProperties;
 import org.apache.ratis.metrics.RatisMetricRegistry;
+import org.apache.ratis.protocol.LeaderElectionRequest;
 import org.apache.ratis.protocol.RaftClientReply;
 import org.apache.ratis.protocol.RaftGroupId;
 import org.apache.ratis.protocol.RaftGroupMemberId;
@@ -30,12 +31,12 @@
 import org.apache.ratis.protocol.RaftPeerId;
 import org.apache.ratis.protocol.exceptions.LeaderSteppingDownException;
 import org.apache.ratis.protocol.exceptions.TransferLeadershipException;
+import org.apache.ratis.rpc.CallId;
 import org.apache.ratis.server.DivisionInfo;
 import org.apache.ratis.server.RaftServer;
 import org.apache.ratis.server.RaftServerConfigKeys;
 import org.apache.ratis.server.metrics.LeaderElectionMetrics;
 import org.apache.ratis.server.raftlog.segmented.SegmentedRaftLogTestUtils;
-import org.apache.ratis.thirdparty.org.checkerframework.checker.units.qual.A;
 import org.apache.ratis.util.ExitUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.LifeCycle;
@@ -415,13 +416,18 @@
   }
 
   void runTestPauseResumeLeaderElection(CLUSTER cluster) throws IOException, InterruptedException {
-    RaftServer.Division leader = waitForLeader(cluster);
-    RaftPeerId leaderId = leader.getId();
+    final RaftClientReply pauseLeaderReply;
+    final RaftServer.Division leader = RaftTestUtil.waitForLeader(cluster);
+    final RaftPeerId leaderId = leader.getId();
     final List<RaftServer.Division> followers = cluster.getFollowers();
     Assert.assertTrue(followers.size() >= 1);
     final RaftServerImpl f1 = (RaftServerImpl)followers.get(0);
-    f1.setLeaderElectionPause(true);
-    try (RaftClient client = cluster.createClient(leader.getId())) {
+
+    try (final RaftClient client = cluster.createClient()) {
+      final LeaderElectionRequest r = LeaderElectionRequest.newPause(
+          client.getId(), f1.getId(),cluster.getGroupId(), CallId.getAndIncrement(), 3000);
+      pauseLeaderReply = f1.getRaftServer().setLeaderElectionAsync(r).join();
+      Assert.assertTrue(pauseLeaderReply.isSuccess());
       client.io().send(new RaftTestUtil.SimpleMessage("message"));
       RaftServer.Division newLeader = followers.get(0);
       List<RaftPeer> peers = cluster.getPeers();
@@ -430,7 +436,10 @@
       Assert.assertTrue(reply.isSuccess());
       JavaUtils.attempt(() -> Assert.assertEquals(leaderId, leader.getId()),
           20, HUNDRED_MILLIS, "check leader id", LOG);
-      f1.setLeaderElectionPause(false);
+      final LeaderElectionRequest r1 = LeaderElectionRequest.newResume(
+          client.getId(), f1.getId(),cluster.getGroupId(), CallId.getAndIncrement(), 3000);
+      final RaftClientReply resumeLeaderReply = f1.getRaftServer().setLeaderElectionAsync(r1).join();
+      Assert.assertTrue(resumeLeaderReply.isSuccess());
       JavaUtils.attempt(() -> Assert.assertEquals(f1.getId(), cluster.getLeader().getId()),
           20, HUNDRED_MILLIS, "check new leader", LOG);
     }