szetszwo commented on a change in pull request #593: URL: https://github.com/apache/ratis/pull/593#discussion_r792573137
########## File path: ratis-common/src/main/java/org/apache/ratis/protocol/PauseLeaderElectionRequest.java ########## @@ -0,0 +1,39 @@ +/* + * 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; + +public final class PauseLeaderElectionRequest extends RaftClientRequest{ + + private final RaftPeerId server; + private final boolean pause; + + public PauseLeaderElectionRequest( + ClientId clientId, RaftPeerId serverId, RaftGroupId groupId, long callId, long timeoutMs, boolean pause) { + super(clientId, serverId, groupId, callId, readRequestType(), timeoutMs); + this.pause = pause; + this.server = serverId; + } + + public RaftPeerId getPausedServer() { Review comment: We already have getServerId() in RaftClientRequest so that this method is not needed. ########## File path: ratis-common/src/main/java/org/apache/ratis/protocol/PauseLeaderElectionRequest.java ########## @@ -0,0 +1,39 @@ +/* + * 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; + +public final class PauseLeaderElectionRequest extends RaftClientRequest{ Review comment: Similar to SnapshotManagementRequest, let's add Op so that we can support - pause - resume We may then add more ops later. Let's rename this LeaderElectionRequest. ########## File path: ratis-server/src/main/java/org/apache/ratis/server/impl/PauseResumeLeaderElection.java ########## @@ -0,0 +1,126 @@ +/* + * 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.server.impl; + +import org.apache.ratis.protocol.PauseLeaderElectionRequest; +import org.apache.ratis.protocol.RaftClientReply; +import org.apache.ratis.protocol.exceptions.RaftException; +import org.apache.ratis.protocol.exceptions.ServerNotReadyException; +import org.apache.ratis.protocol.exceptions.TimeoutIOException; +import org.apache.ratis.util.JavaUtils; +import org.apache.ratis.util.MemoizedSupplier; +import org.apache.ratis.util.TimeDuration; +import org.apache.ratis.util.TimeoutScheduler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class PauseResumeLeaderElection { Review comment: Since the pause request only change a boolean, this class is not needed. ########## File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RoleInfo.java ########## @@ -124,6 +124,10 @@ void setLeaderElectionPause(boolean pause) { pauseLeaderElection.set(pause); } + boolean getLeaderElectionPauseState() { Review comment: This method is not needed. ########## File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java ########## @@ -622,6 +624,13 @@ RaftClientReply newSuccessReply(RaftClientRequest request, long logIndex) { .build(); } + RaftClientReply newSuccessReply(RaftClientRequest request, RaftPeerId peerId) { Review comment: This method is not needed since the peerId must equal to the current server. ########## File path: ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java ########## @@ -1009,6 +1018,20 @@ SnapshotManagementRequestHandler getSnapshotRequestHandler() { return snapshotRequestHandler; } + CompletableFuture<RaftClientReply> pauseLeaderElectionAsync(PauseLeaderElectionRequest request) throws IOException { + LOG.info("{} receive pauseLeaderElection {}", getMemberId(), request); + assertLifeCycleState(LifeCycle.States.RUNNING); + assertGroup(request.getRequestorId(), request.getRaftGroupId()); + + boolean pause = getRole().getLeaderElectionPauseState(); + if (pause == request.getPause()) { + return CompletableFuture.completedFuture(newSuccessReply(request, request.getPausedServer())); + } + synchronized (this) { + return pauseLeaderElection.pauseLeaderElectionAsync(request); + } + } Review comment: Let's rename this method to setLeaderElectionAsync. The implementation could be very simple. ``` CompletableFuture<RaftClientReply> setLeaderElectionAsync(LeaderElectionRequest request) throws IOException { LOG.info("{} receive setLeaderElection {}", getMemberId(), request); assertLifeCycleState(LifeCycle.States.RUNNING); assertGroup(request.getRequestorId(), request.getRaftGroupId()); synchronized (this) { setLeaderElectionPause(request.getPause()); } return CompletableFuture.completedFuture(newSuccessReply(request)); } ``` -- 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]
