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


##########
ratis-server/src/main/java/org/apache/ratis/server/impl/LeaderLease.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.List;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+class LeaderLease {
+
+  private final long leaseTimeoutMs;
+  // TODO invalidate leader lease when stepDown / transferLeader
+  private final AtomicReference<Timestamp> lease = new 
AtomicReference<>(Timestamp.currentTime());
+
+  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;
+  }
+
+  /**
+   * try extending the lease based on group heartbeats
+   * @param old nullable
+   */
+  void extend(List<FollowerInfo> current, List<FollowerInfo> old, 
Predicate<List<RaftPeerId>> hasMajority) {
+    final List<RaftPeerId> activePeers =
+        // check the latest heartbeats of all peers (including those in 
transitional)
+        Stream.concat(current.stream(), 
Optional.ofNullable(old).map(List::stream).orElse(Stream.empty()))
+            .filter(f -> 
f.getLastRespondedAppendEntriesSendTime().elapsedTimeMs() < leaseTimeoutMs)
+            .map(FollowerInfo::getId)
+            .collect(Collectors.toList());
+
+    if (!hasMajority.test(activePeers)) {
+      return;
+    }
+
+    // update the new lease
+    final Timestamp newLease =
+        Timestamp.earliest(getMaxTimestampWithMajorityAck(current), 
getMaxTimestampWithMajorityAck(old));
+    lease.set(newLease);
+  }
+
+  /**
+   * return maximum timestamp at when the majority of followers are known to 
be active
+   * return {@link Timestamp#currentTime()} if peers are empty
+   */
+  private Timestamp getMaxTimestampWithMajorityAck(List<FollowerInfo> peers) {
+    if (peers == null || peers.isEmpty()) {
+      return Timestamp.currentTime();
+    }
+
+    final List<Timestamp> lastRespondedAppendEntriesSendTimes = peers.stream()
+        .map(FollowerInfo::getLastRespondedAppendEntriesSendTime)
+        .sorted()
+        .collect(Collectors.toList());
+
+    return 
lastRespondedAppendEntriesSendTimes.get(lastRespondedAppendEntriesSendTimes.size()
 / 2);
+  }

Review Comment:
   Hi, @szetszwo 
   
   I believe your suggestion of using `(followers.size() - 1) / 2` is more 
accurate.
   
   Even though the timestamps are sorted in ascending order, I think it's still 
appropriate to use index 0 for 2 followers to satisfy the majority requirement.
   
   For example, if the electionTimeout is set to 3s, and the leaseTimeout is 
also set to 3s:
   
   In a Raft group with three replicas A, B, and C, where A is the Leader, 
let's assume that at time 7, A's lease expires and needs to be extended. A 
observes that B's LastRespondedAppendEntriesSendTime is at time 5, and C's 
LastRespondedAppendEntriesSendTime is at time 6. If A sets the time for 
newLease based on index 1 to be 6, suppose at time 8, a network partition 
occurs and B might trigger an election, possibly gaining support from C to 
become the new Leader. This would result in two Leaders in the cluster during 
the time range of 8-9, which violates correctness.
   
   Even though, as pointed out by @SzyWilliam, C in the example above may not 
necessarily vote during the [PreVote 
phase](https://github.com/apache/ratis/blob/master/ratis-server/src/main/java/org/apache/ratis/server/impl/VoteContext.java#L105),
 its 
[decisions](https://github.com/apache/ratis/blob/master/ratis-server/src/main/java/org/apache/ratis/server/impl/FollowerState.java#L92)
 in FollowerState are unrelated to parameters related to Lease Read, which may 
still pose a risk of split-brain under certain parameter combinations.
   
   If we set newLease to 5 at index 0, this problem would not occur.
   
   Therefore, I still agree with your previous description:
   - 1 or 2 followers: use index 0
   - 3 or 4 followers: use index 1
   
   What's your opinion?



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