szetszwo commented on code in PR #898:
URL: https://github.com/apache/ratis/pull/898#discussion_r1323521727


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.leader.FollowerInfo;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.Timestamp;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+public class LeaderLease {

Review Comment:
   Remove `public`.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderStateImpl.java:
##########
@@ -247,6 +247,11 @@ List<FollowerInfo> getCurrent() {
     List<FollowerInfo> getOld() {
       return old;
     }
+
+    Stream<FollowerInfo> getCurrentAndOld() {
+      return Stream.concat(current.stream(),
+          Optional.ofNullable(old).map(List::stream).orElse(Stream.empty()));

Review Comment:
   We should deduplicate the peers.



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.leader.FollowerInfo;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.Timestamp;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+public class LeaderLease {
+
+  private final long leaseTimeoutMs;
+  // TODO invalidate leader lease when stepDown / transferLeader
+  private final AtomicReference<Timestamp> lease = new 
AtomicReference<>(Timestamp.currentTime());
+
+  public LeaderLease(RaftProperties properties) {
+    final double leaseRatio = 
RaftServerConfigKeys.Read.leaderLeaseTimeoutRatio(properties);
+    Preconditions.assertTrue(leaseRatio > 0.0 && leaseRatio <= 1.0,
+        "leader ratio should sit in (0,1], now is " + leaseRatio);
+    this.leaseTimeoutMs = RaftServerConfigKeys.Rpc.timeoutMin(properties)
+        .multiply(leaseRatio)
+        .toIntExact(TimeUnit.MILLISECONDS);
+  }
+
+  boolean isValid() {
+    return lease.get().elapsedTimeMs() < leaseTimeoutMs;
+  }
+
+  void extendLeaderLease(Stream<FollowerInfo> allFollowers, 
Predicate<List<RaftPeerId>> hasMajority) {
+    // check the latest heartbeats of all peers (including those in 
transitional)
+    final List<RaftPeerId> activePeers = new ArrayList<>();
+    final List<Timestamp> lastRespondedAppendEntriesSendTimes = new 
ArrayList<>();
+
+    allFollowers.forEach(follower -> {
+      final Timestamp lastRespondedAppendEntriesSendTime = 
follower.getLastRespondedAppendEntriesSendTime();
+      
lastRespondedAppendEntriesSendTimes.add(lastRespondedAppendEntriesSendTime);
+      if (lastRespondedAppendEntriesSendTime.elapsedTimeMs() < leaseTimeoutMs) 
{
+        activePeers.add(follower.getId());
+      }
+    });
+
+    if (hasMajority.test(activePeers)) {
+      // can extend leader lease
+      if (lastRespondedAppendEntriesSendTimes.isEmpty()) {
+        lease.set(Timestamp.currentTime());
+      } else {
+        Collections.sort(lastRespondedAppendEntriesSendTimes);
+        final Timestamp newLease =
+            
lastRespondedAppendEntriesSendTimes.get(lastRespondedAppendEntriesSendTimes.size()
 / 2);

Review Comment:
   `lastRespondedAppendEntriesSendTimes` may contain the timestamps from both 
current and old.  Should we
   1.  separate the timestamps into two lists,
   2. get two mid values, and
   3. use the min of the two mid values?



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerInfoImpl.java:
##########
@@ -202,4 +204,15 @@ public Timestamp getLastRpcTime() {
   public Timestamp getLastHeartbeatSendTime() {
     return lastHeartbeatSendTime.get();
   }
+
+  @Override
+  public Timestamp getLastRespondedAppendEntriesSendTime() {
+    return lastRespondedAppendEntriesSendTime.get();
+  }
+
+  @Override
+  public void updateLastAppendEntriesResponseTime(Timestamp sendTime) {
+    updateLastRpcResponseTime();
+    lastRespondedAppendEntriesSendTime.set(sendTime);
+  }

Review Comment:
   Let's call it` updateLastRespondedAppendEntriesSendTime` and update only the 
sendTime. It will be easier to understand. i.e.
   ```java
     @Override
     public void updateLastRespondedAppendEntriesSendTime(Timestamp sendTime) {
       lastRespondedAppendEntriesSendTime.set(sendTime);
     }
   ```
   



##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.conf.RaftProperties;
+import org.apache.ratis.protocol.RaftPeerId;
+import org.apache.ratis.server.RaftServerConfigKeys;
+import org.apache.ratis.server.leader.FollowerInfo;
+import org.apache.ratis.util.Preconditions;
+import org.apache.ratis.util.Timestamp;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+public class LeaderLease {
+
+  private final long leaseTimeoutMs;
+  // TODO invalidate leader lease when stepDown / transferLeader
+  private final AtomicReference<Timestamp> lease = new 
AtomicReference<>(Timestamp.currentTime());
+
+  public LeaderLease(RaftProperties properties) {

Review Comment:
   Remove `public`.



##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java:
##########
@@ -390,6 +390,9 @@ public void onNext(AppendEntriesReplyProto reply) {
       AppendEntriesRequest request = pendingRequests.remove(reply);
       if (request != null) {
         request.stopRequestTimer(); // Update completion time
+        
getFollower().updateLastAppendEntriesResponseTime(request.getSendTime()); // 
Update the last rpc time
+      } else {
+        getFollower().updateLastRpcResponseTime();
       }

Review Comment:
   How about moving the `updateLastRpcResponseTime()` out?
   ```java
           
getFollower().updateLastRespondedAppendEntriesSendTime(request.getSendTime());
         }
         getFollower().updateLastRpcResponseTime();
   ```



-- 
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]

Reply via email to