RATIS-1912. Fix infinity election when perform membership change.#954
RATIS-1912. Fix infinity election when perform membership change.#954szetszwo merged 10 commits intoapache:masterfrom
Conversation
|
Hi @szetszwo, could you kindly help to take a review when you have time, thanks. |
szetszwo
left a comment
There was a problem hiding this comment.
@wojiaodoubao , thanks a lot for working not this! Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13064049/954_review.patch
| return logAndReturn(phase, Result.PASSED, responses, exceptions); | ||
| } else if (singleMode) { | ||
| // if candidate is in single mode, candidate pass vote. | ||
| return logAndReturn(phase, Result.PASSED, responses, exceptions); |
There was a problem hiding this comment.
Let's add a Result.SINGLE_MODE_PASSED.
| if (conf.hasMajority(votedPeers, server.getId())) { | ||
| return logAndReturn(phase, Result.PASSED, responses, exceptions); | ||
| } else if (singleMode) { | ||
| return logAndReturn(phase, Result.PASSED, responses, exceptions); |
| * changing from single mode to HA mode. | ||
| */ | ||
| boolean changeMajority(Collection<RaftPeer> newMembers) { | ||
| Preconditions.assertNull(oldConf, "Conf must be stable."); |
There was a problem hiding this comment.
It expect to pass the name, i.e. Preconditions.assertNull(oldConf, "oldConf")
| } | ||
| // If newPeersCount reaches majority number of new conf size, the cluster may end with infinity | ||
| // election. See https://issues.apache.org/jira/browse/RATIS-1912 for more details. | ||
| return newPeersCount >= newMembers.size() / 2 + newMembers.size() % 2; |
There was a problem hiding this comment.
It may be easier to understand to return newPeersCount >= oldPeersCount, i.e.
final long oldPeersCount = newMembers.size() - newPeersCount;
return newPeersCount >= oldPeersCount;| return oldConf.size() == 1 && oldConf.contains(selfId) && conf.size() == 2 && conf.contains( | ||
| selfId); |
There was a problem hiding this comment.
Make it a single line (ratis uses 120 line width).
| pending.setReply(newSuccessReply(request)); | ||
| return pending.getFuture(); | ||
| } | ||
| if (arguments.getMode() != SetConfigurationRequest.Mode.SET_UNCONDITIONALLY |
There was a problem hiding this comment.
Since it is not safe, let's don't allow it even for SET_UNCONDITIONALLY.
| } | ||
|
|
||
| private RaftServerImpl getImpl(RaftGroupId groupId) throws IOException { | ||
| public RaftServerImpl getImpl(RaftGroupId groupId) throws IOException { |
There was a problem hiding this comment.
The method is not used in this change. Let's keep it private.
| @VisibleForTesting | ||
| void setRaftConf(RaftGroupId groupId, RaftConfigurationImpl conf) throws IOException { | ||
| getImpl(groupId).getState().setRaftConf(conf); | ||
| } |
There was a problem hiding this comment.
Move it to RaftServerTestUtil, i.e.
//RaftServerTestUtil
public static void setRaftConf(RaftServer proxy, RaftGroupId groupId, RaftConfiguration conf) {
((RaftServerImpl)getDivision(proxy, groupId)).getState().setRaftConf(conf);
}| .build(); | ||
| Assert.assertTrue(oldNewConf.isSingleMode(curPeer.getId())); | ||
| leaderServer.setRaftConf(groupId, oldNewConf); | ||
| leaderServer.triggerElection(groupId); |
There was a problem hiding this comment.
Use transferLeadership with null new leader and then remove the new triggerElection method.
try(RaftClient client = cluster.createClient()) {
client.admin().transferLeadership(null, leaderServer.getId(), 10_000);
}| leaderServer.setRaftConf(groupId, oldNewConf); | ||
| leaderServer.triggerElection(groupId); | ||
|
|
||
| RaftTestUtil.waitForLeader(cluster); |
There was a problem hiding this comment.
Check new leader and new conf:
final RaftServer.Division newLeader = RaftTestUtil.waitForLeader(cluster);
Assert.assertEquals(leaderServer.getId(), newLeader.getId());
Assert.assertEquals(oldNewConf, newLeader.getRaftConf());|
Thanks @szetszwo your nice comments ! Upload v0.7 to trigger workflow. Some unit tests will fail after v0.7. Waiting the workflow to tell me which are they. |
szetszwo
left a comment
There was a problem hiding this comment.
+1 the change looks good.
|
@wojiaodoubao , there are quite a few tests need to be updated due the new restriction on setConf. Could you take a look? |
szetszwo
left a comment
There was a problem hiding this comment.
@wojiaodoubao , thanks for fixing the tests. All tests pass except for TestInstallSnapshotNotificationWithGrpc. Could you take a look?
| public interface ConsumerWithIOException { | ||
| void accept(Collection<RaftPeer> peesToSetConf) throws IOException; | ||
| } |
There was a problem hiding this comment.
Use CheckedConsumer, i.e.
public static void runWithMinorityPeers(MiniRaftCluster cluster, Collection<RaftPeer> peersInNewConf,
CheckedConsumer<Collection<RaftPeer>, IOException> consumer) throws IOException {|
The failed case is TestInstallSnapshotNotificationWithGrpc#testInstallSnapshotDuringBootstrap. It expected 2 times of 'installSnapshot' after 'setConfiguration', but got 3 times. I failed to reproduce the failure on my local environment. Upload v0.9 based on CheckedConsumer. Re-trigger workflow to see whether the failed case occurs repeatedly. |
|
In TestInstallSnapshotNotificationWithGrpc#testInstallSnapshotDuringBootstrap, the setConfiguration is separated into 2 rpc calls. Add peer s1 first, then peer s2. The s2 will be notified twice of install snapshot event. I think it doesn't break the semantic of notifyInstallSnapshot. Let me quote from GrpcLogAppender#shouldNotifyToInstallSnapshot.
The GrpcLogAppender keeps notifyInstallSnapshot to the bootstrapping follower again and again. So install snapshot twice does happen. Changing the assert condition to 2 <= numSnapshotRequests.get() should be fine. |
On a second thought, it is good to have a server conf to enable/disable the restriction since it is more efficient to change
than
For changing from non-HA to HA, it usually is an one time thing and the admin will monitor it. The split brain problem can be ignored. @wojiaodoubao , what do you think? |
|
Filed RATIS-1930. |
|
Thanks a lot for @szetszwo's kindly help.
I totally agree. |
### What changes were proposed in this pull request? Bump Ratis version from 2.5.1 to 3.0.1. Address incompatible changes: - RATIS-589. Eliminate buffer copying in SegmentedRaftLogOutputStream.(apache/ratis#964) - RATIS-1677. Do not auto format RaftStorage in RECOVER.(apache/ratis#718) - RATIS-1710. Refactor metrics api and implementation to separated modules. (apache/ratis#749) ### Why are the changes needed? Bump Ratis version from 2.5.1 to 3.0.1. Ratis has released v3.0.0, v3.0.1, which release note refers to [3.0.0](https://ratis.apache.org/post/3.0.0.html), [3.0.1](https://ratis.apache.org/post/3.0.1.html). The 3.0.x version include new features like pluggable metrics and lease read, etc, some improvements and bugfixes including: - 3.0.0: Change list of ratis 3.0.0 In total, there are roughly 100 commits diffing from 2.5.1 including: - Incompatible Changes - RaftStorage Auto-Format - RATIS-1677. Do not auto format RaftStorage in RECOVER. (apache/ratis#718) - RATIS-1694. Fix the compatibility issue of RATIS-1677. (apache/ratis#731) - RATIS-1871. Auto format RaftStorage when there is only one directory configured. (apache/ratis#903) - Pluggable Ratis-Metrics (RATIS-1688) - RATIS-1689. Remove the use of the thirdparty Gauge. (apache/ratis#728) - RATIS-1692. Remove the use of the thirdparty Counter. (apache/ratis#732) - RATIS-1693. Remove the use of the thirdparty Timer. (apache/ratis#734) - RATIS-1703. Move MetricsReporting and JvmMetrics to impl. (apache/ratis#741) - RATIS-1704. Fix SuppressWarnings(“VisibilityModifier”) in RatisMetrics. (apache/ratis#742) - RATIS-1710. Refactor metrics api and implementation to separated modules. (apache/ratis#749) - RATIS-1712. Add a dropwizard 3 implementation of ratis-metrics-api. (apache/ratis#751) - RATIS-1391. Update library dropwizard.metrics version to 4.x (apache/ratis#632) - RATIS-1601. Use the shaded dropwizard metrics and remove the dependency (apache/ratis#671) - Streaming Protocol Change - RATIS-1569. Move the asyncRpcApi.sendForward(..) call to the client side. (apache/ratis#635) - New Features - Leader Lease (RATIS-1864) - RATIS-1865. Add leader lease bound ratio configuration (apache/ratis#897) - RATIS-1866. Maintain leader lease after AppendEntries (apache/ratis#898) - RATIS-1894. Implement ReadOnly based on leader lease (apache/ratis#925) - RATIS-1882. Support read-after-write consistency (apache/ratis#913) - StateMachine API - RATIS-1874. Add notifyLeaderReady function in IStateMachine (apache/ratis#906) - RATIS-1897. Make TransactionContext available in DataApi.write(..). (apache/ratis#930) - New Configuration Properties - RATIS-1862. Add the parameter whether to take Snapshot when stopping to adapt to different services (apache/ratis#896) - RATIS-1930. Add a conf for enable/disable majority-add. (apache/ratis#961) - RATIS-1918. Introduces parameters that separately control the shutdown of RaftServerProxy by JVMPauseMonitor. (apache/ratis#950) - RATIS-1636. Support re-config ratis properties (apache/ratis#800) - RATIS-1860. Add ratis-shell cmd to generate a new raft-meta.conf. (apache/ratis#901) - Improvements & Bug Fixes - Netty - RATIS-1898. Netty should use EpollEventLoopGroup by default (apache/ratis#931) - RATIS-1899. Use EpollEventLoopGroup for Netty Proxies (apache/ratis#932) - RATIS-1921. Shared worker group in WorkerGroupGetter should be closed. (apache/ratis#955) - RATIS-1923. Netty: atomic operations require side-effect-free functions. (apache/ratis#956) - RaftServer - RATIS-1924. Increase the default of raft.server.log.segment.size.max. (apache/ratis#957) - RATIS-1892. Unify the lifetime of the RaftServerProxy thread pool (apache/ratis#923) - RATIS-1889. NoSuchMethodError: RaftServerMetricsImpl.addNumPendingRequestsGauge apache/ratis#922 (apache/ratis#922) - RATIS-761. Handle writeStateMachineData failure in leader. (apache/ratis#927) - RATIS-1902. The snapshot index is set incorrectly in InstallSnapshotReplyProto. (apache/ratis#933) - RATIS-1912. Fix infinity election when perform membership change. (apache/ratis#954) - RATIS-1858. Follower keeps logging first election timeout. (apache/ratis#894) - 3.0.1:This is a bugfix release. See the [changes between 3.0.0 and 3.0.1](apache/ratis@ratis-3.0.0...ratis-3.0.1) releases. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Cluster manual test. Closes #2480 from SteNicholas/CELEBORN-1400. Authored-by: SteNicholas <programgeek@163.com> Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>
What changes were proposed in this pull request?
This patch resolves a membership change bug. See detail in #943.
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1912
How was this patch tested?
unit tests