Skip to content

Commit

Permalink
RATIS-1930. Add a conf for enable/disable majority-add.
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo committed Nov 5, 2023
1 parent d673154 commit e42bba4
Show file tree
Hide file tree
Showing 4 changed files with 50 additions and 11 deletions.
19 changes: 18 additions & 1 deletion ratis-docs/src/site/markdown/configurations.md
Original file line number Diff line number Diff line change
Expand Up @@ -646,11 +646,28 @@ First election timeout is introduced to reduce unavailable time when a RaftGroup
| **Property** | `raft.server.leaderelection.pre-vote` |
|:----------------|:--------------------------------------|
| **Description** | enable pre-vote |
| **Type** | bool |
| **Type** | boolean |
| **Default** | true |

In Pre-Vote, the candidate does not change its term and try to learn
if a majority of the cluster would be willing to grant the candidate their votes
(if the candidate’s log is sufficiently up-to-date,
and the voters have not received heartbeats from a valid leader
for at least a baseline election timeout).

| **Property** | `raft.server.leaderelection.member.majority-add` |
|:----------------|:-------------------------------------------------|
| **Description** | enable majority-add |
| **Type** | boolean |
| **Default** | false |

Does it allow *majority-add*, i.e. adding a majority of members in a single setConf?

- Note that, when a single setConf removes and adds members at the same time,
the majority is counted after the removal.
For examples,
1. setConf to a 3-member group by adding 2 new members is NOT a majority-add.
2. However, setConf to a 3-member group by removing 2 of members and adding 2 new members is a majority-add.

- Note also that adding 1 new member to an 1-member group is always allowed,
although it is a majority-add.
Original file line number Diff line number Diff line change
Expand Up @@ -881,10 +881,29 @@ static void setLeaderStepDownWaitTime(RaftProperties properties, TimeDuration le
static boolean preVote(RaftProperties properties) {
return getBoolean(properties::getBoolean, PRE_VOTE_KEY, PRE_VOTE_DEFAULT, getDefaultLog());
}

static void setPreVote(RaftProperties properties, boolean enablePreVote) {
setBoolean(properties::setBoolean, PRE_VOTE_KEY, enablePreVote);
}

/**
* Does it allow majority-add, i.e. adding a majority of members in a single setConf?
* <p>
* Note that, when a single setConf removes and adds members at the same time,
* the majority is counted after the removal.
* For examples, setConf to a 3-member group by adding 2 new members is NOT a majority-add.
* However, setConf to a 3-member group by removing 2 of members and adding 2 new members is a majority-add.
* <p>
* Note also that adding 1 new member to an 1-member group is always allowed,
* although it is a majority-add.
*/
String MEMBER_MAJORITY_ADD_KEY = PREFIX + ".member.majority-add";
boolean MEMBER_MAJORITY_ADD_DEFAULT = false;
static boolean memberMajorityAdd(RaftProperties properties) {
return getBoolean(properties::getBoolean, MEMBER_MAJORITY_ADD_KEY, MEMBER_MAJORITY_ADD_DEFAULT, getDefaultLog());
}
static void setMemberMajorityAdd(RaftProperties properties, boolean enableMemberMajorityAdd) {
setBoolean(properties::setBoolean, MEMBER_MAJORITY_ADD_KEY, enableMemberMajorityAdd);
}
}

static void main(String[] args) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -214,6 +214,7 @@ public long[] getFollowerNextIndices() {

private final DivisionProperties divisionProperties;
private final TimeDuration leaderStepDownWaitTime;
private final boolean memberMajorityAddEnabled;
private final TimeDuration sleepDeviationThreshold;

private final LifeCycle lifeCycle;
Expand Down Expand Up @@ -259,7 +260,8 @@ public long[] getFollowerNextIndices() {

final RaftProperties properties = proxy.getProperties();
this.divisionProperties = new DivisionPropertiesImpl(properties);
leaderStepDownWaitTime = RaftServerConfigKeys.LeaderElection.leaderStepDownWaitTime(properties);
this.leaderStepDownWaitTime = RaftServerConfigKeys.LeaderElection.leaderStepDownWaitTime(properties);
this.memberMajorityAddEnabled = RaftServerConfigKeys.LeaderElection.memberMajorityAdd(properties);
this.sleepDeviationThreshold = RaftServerConfigKeys.sleepDeviationThreshold(properties);
this.proxy = proxy;

Expand Down Expand Up @@ -1314,8 +1316,11 @@ public CompletableFuture<RaftClientReply> setConfigurationAsync(SetConfiguration
return pending.getFuture();
}
if (current.changeMajority(serversInNewConf)) {
throw new SetConfigurationException("Failed to set configuration: request " + request
+ " changes a majority set of the current configuration " + current);
if (!memberMajorityAddEnabled) {
throw new SetConfigurationException("Failed to set configuration: request " + request
+ " changes a majority set of the current configuration " + current);
}
LOG.warn("Try to add/replace a majority of servers in a single setConf: {}", request);
}

getRaftServer().addRaftPeers(serversInNewConf);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@
import org.apache.ratis.conf.RaftProperties;
import org.apache.ratis.protocol.RaftClientReply;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.server.RaftServer;
import org.apache.ratis.server.RaftServerConfigKeys;
Expand Down Expand Up @@ -65,11 +64,11 @@ public abstract class InstallSnapshotFromLeaderTests<CLUSTER extends MiniRaftClu

{
final RaftProperties prop = getProperties();
RaftServerConfigKeys.Log.setPurgeGap(prop, PURGE_GAP);
RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold(
prop, SNAPSHOT_TRIGGER_THRESHOLD);
RaftServerConfigKeys.Snapshot.setAutoTriggerEnabled(prop, true);
RaftServerConfigKeys.Snapshot.setAutoTriggerThreshold(prop, SNAPSHOT_TRIGGER_THRESHOLD);
RaftServerConfigKeys.Log.setPurgeGap(prop, PURGE_GAP);
RaftServerConfigKeys.Log.Appender.setSnapshotChunkSizeMax(prop, SizeInBytes.ONE_KB);
RaftServerConfigKeys.LeaderElection.setMemberMajorityAdd(prop, true);
}

private static final int SNAPSHOT_TRIGGER_THRESHOLD = 64;
Expand Down Expand Up @@ -112,8 +111,7 @@ private void testMultiFileInstallSnapshot(CLUSTER cluster) throws Exception {
final MiniRaftCluster.PeerChanges change = cluster.addNewPeers(2, true,
true);
// trigger setConfiguration
RaftServerTestUtil.runWithMinorityPeers(cluster, Arrays.asList(change.allPeersInNewConf),
peers -> cluster.setConfiguration(peers.toArray(RaftPeer.emptyArray())));
cluster.setConfiguration(change.allPeersInNewConf);

RaftServerTestUtil
.waitAndCheckNewConf(cluster, change.allPeersInNewConf, 0, null);
Expand Down

0 comments on commit e42bba4

Please sign in to comment.