fix membership change failed when cluster restart after joint stage
issue
change_peers 过程中继续重启,可能导致无法恢复。
bad case (from braft wechat group):
- 启动 5 个节点的 cluster $(A,B,C,D,E)$
leader在 $(A,B,C)$ 中, 开始节点变更,从集群中删除 $(A,B,C)$ , 仅保留 $(D,E)$, 即 $C_{old} = (A,B,C,D,E)$ , $C_{new} = (D,E)$- 当
leader完成第一阶段日志 $C_{old,new}$ 提交 后,在leader开始第二阶段的日志($C_{new}$)提交前,这时 $(D,E)$ 节点下线,导致 $C_{new}$ 无法被提交(其大多数节点失效),而 $C_{new}$ 日志可能会被复制到 $(A,B,C)$ 中。 - 所有节点重启,此时由于 $(A,B,C)$ 中的最新日志为 $C_{new}$, 都不属于该配置 ,无法完成选举( 按照 braft 现在实现, PreVote 无法发起); 而 $(D,E)$ 节点中最新日志为 $C_{old,new}$,依然无法完成选举, 因为其相较于 $(A,B,C)$ 节点日志落后,并且 $majority(C_{old,new}) = 3 > 2 $。 所以整个集群无法选举出
leader,继续后续流程。
cause
-
braft 实现中,在节点变更的第二阶段,会将 $C_{new}$ 作为日志继续给当前的所有 follower 发送,即使是当前 $C_{new}$ 中不包含的节点。如果能够在提交第二阶段日志时, 不复制 $C_{new}$ 日志到不属于其配置的节点($diff(C_{new,old}, C_{new})$), 那么在集群恢复后,这些节点由于不持有最新的 $C_new$, 故而可以重新发起选举,完成后续流程。 安全性 a. 如果集群重启前, $C_{new}$ 已经被成功提交,那么不属于 $C_{new}$ 的节点由于持有第一阶段的配置 $C_{old, new}$, 无法获得来自 $C_{new}$ 节点中大多数的投票,故而无法选举 leader (预期的行为)。 而 $C_new$ 由于已经被提交,所以 $C_{new}$ 中的节点能够正常选举 leader, 因此完成了
membership change流程, 集群恢复正常。 b. 如果集群重启前, $C_{new}$ 未被成功提交,那么属于 $C_{new,old}$ 中的节点是可以从 $C_{new}$ 中获得大多数投票,从而完成选举。 如果选举的 leader 包含 $C_{new}$ , 那么第一次日志提交后( flush configuration 等),即可完成 membership change;如果选举的 leader 只包含 $C_{new,old}$, 重新进入joint stage, 并且接着开始复制 $C_{new}$ 并完成membership change的 stable 状态。 -
braft 实现中,如果节点 id 不属于在当前 conf,不允许节点发起选举。如果放松这个限制的话,也可以避免上述问题。 安全性 a. 如果不属于 conf 的节点可以发起投票,并且当选 leader,那么说明该节点包含当前 conf 中最新的日志,并且获得 conf 中大多数人的同意(自己的投票不会被计数),满足 leader 的安全性,见 raft §5.4.3 Safety argument。此时该节点可以看作是一个代理节点,负责转发 raft 请求,但是不参与 raft 的投票和 commit 日志的决策,不影响 raft 的安全性。 对于本例中,重启后不属于 $C_{new}$ 的节点发起投票,并且能够获得来自 $C_{new}$ 的大多数投票,当该 leader 完成第一次日志复制后( flush configuration), $C_{new}$ 日志会被提交到 $C_{new}$ 节点中(想当于继续完成了重启前未完成的 membership change 的第二阶段)。 而该 leader 会在完成 stable 阶段完成 step down,不参与后续流程。而此时 $C_{new}$ 节点中也已经持有最新的日志,其他无关节点无法继续参与 $C_{new}$ 的后续事务(因为不包含最新日志)。所以也是安全的。
solutions
- 从目前 braft 的实现来看, 如果采取第一种的话,比较难以实现:
a. 如何控制在第一阶段的 $C_{old,new}$ 提交后,$C_{new}$ 不复制到其他节点。可以采取的方案可以为
stop_replicator, 将节点设置为readonly或者发送前日志/接受日志时进行判断等等,这个会增加较多的代码逻辑。 b. 如果当前 leader 不在 $C_{new}$ 中,很难做到不复制 $C_{new}$ 日志,因为AppendEntries需要从LogManager中获得新的日志进行复制, 这两者存在一定的矛盾,代码逻辑不好做修改。 - 所以本 PR 采取第二种方案,放松可以参与选举节点的限制(不在最新的 configuration 的节点依然可以发起选举)
由于当前的 braft 中没有注入错误故障的模块,在构造一些比较复杂的用例时,无法准确控制错误发生的时机,所以引入了
SyncPoint模块 (copy from rocksdb),提供回调机制来控制节点的行为。 使用方法:
// set sync or callback at specific point identified by name.
TEST_SYNC_POINT/CALL_BACK(name, args)
// register call back function, it will be triggerd when code
// reach the sync point with the same name.
SyncPoint::GetInstance()->SetCallBack(name, callback);
test plan
NodeTest.change_peers_restart_cluster_before_stable_stage
模拟了上述 bad case 的用例。
在该修改前,测试会在重启节点之后一直保持无法选主状态,直到超时。
本修改之后,符合预期行为。