sofastack / sofa-jraft

A production-grade java implementation of RAFT consensus algorithm.
https://www.sofastack.tech/projects/sofa-jraft/
Apache License 2.0
3.53k stars 1.12k forks source link

How do I manually trigger a node to copy a snapshot from a leader and apply it locally? #254

Closed kdlan closed 4 years ago

kdlan commented 4 years ago

Your question

如何手动触发节点从leader 拷贝snapshot并且应用到本地?

Your scenes

因为机器环境的原因,每次重启以后本地的文件会被清空,这个时候需要从leader拉取snapshot加载,并且apply新的log。按照counter和kv里面的启动方式,重启的节点一直会报 reject term_unmatched AppendEntriesRequest 的错误

类似的场景还有某台机器磁盘坏了导致文件丢失。更换磁盘后再起来,也是类似的问题

Your advice

目前的做法是启动的时候看当前节点是否在集群的节点列表中,如果在就通过 CliService 把节点剔除掉再加入。想问下有没有其他的办法直接发起snapshot 的拷贝?

Environment

sofastack-bot[bot] commented 4 years ago

Hi @kdlan, we detect non-English characters in the issue. This comment is an auto translation by @sofastack-robot to help other users to understand this issue.

We encourage you to describe your issue in English which is more friendly to other users.

Your question How do I manually trigger a node to copy a snapshot from a leader and apply it locally? ### Your scenes Because of the machine environment, the local files will be emptied after each reboot. At this time, you need to pull the snapshot from the leader and apply the new log. According to the startup method in counter and kv, the restarted node will always report the error of reject term_unmatched AppendEntriesRequest similar to the scenario where a certain machine disk is broken and the file is lost. Replacing the disk and then getting up is also a similar problem ### Your advice The current practice is to see if the current node is in the node list of the cluster when starting, if you remove the node by CliService and then join. Would you like to ask if there is any other way to directly initiate a copy of the snapshot? ### Environment - SOFAJRaft version: 1.2.4 - JVM version (eg java -version): 1.8 - OS version (eg uname -a): centos 7.1 - Maven version: - IDE version:

killme2008 commented 4 years ago

@kdlan 确定完全清空了数据?特别是 meta 信息保存目录。如果没清除,会报你说的错误。

kdlan commented 4 years ago

@killme2008 确定,是完全把目录清空,包括log, meta, snapshot 还有rocksdb的目录

killme2008 commented 4 years ago

@kdlan 用 counter server 做了测试,并没有发现你说的存在的问题

2019-08-20 11:23:50 [Rpc-netty-server-worker-1-thread-1] WARN  RaftRpcServerFactory:315 - JRaft SET bolt.rpc.dispatch-msg-list-in-default-executor to be false for replicator pipeline optimistic.
2019-08-20 11:23:50 [test/localhost:8003-AppendEntriesThread0] INFO  LocalRaftMetaStorage:132 - Save raft meta, path=/tmp/server3/raft_meta, term=2, votedFor=0.0.0.0:0, cost time=9 ms
2019-08-20 11:23:50 [test/localhost:8003-AppendEntriesThread0] WARN  NodeImpl:1650 - Node <test/localhost:8003> reject term_unmatched AppendEntriesRequest from localhost:8002, term=2, prevLogIndex=1002, prevLogTerm=2, localPrevLogTerm=0, lastLogIndex=0, entriesSize=0.
2019-08-20 11:23:50 [JRaft-FSMCaller-Disruptor-0] INFO  StateMachineAdapter:89 - onStartFollowing: LeaderChangeContext [leaderId=localhost:8002, term=2, status=Status[ENEWLEADER<10011>: Raft node receives message from new leader with higher term.]]
2019-08-20 11:23:50 [Bolt-default-executor-6-thread-2] INFO  NodeImpl:2798 - Node <test/localhost:8003> received InstallSnapshotRequest from localhost:8002, lastIncludedLogIndex=1002, lastIncludedLogTerm=2, lastLogId=LogId [index=0, term=0].
2019-08-20 11:23:50 [Bolt-conn-event-executor-5-thread-1] INFO  ClientServiceConnectionEventProcessor:50 - Peer localhost:8002 is connected
2019-08-20 11:23:50 [JRaft-Closure-Executor-0] INFO  LocalSnapshotStorage:167 - Deleting snapshot /tmp/server3/snapshot/snapshot_1002
2019-08-20 11:23:50 [JRaft-Closure-Executor-0] INFO  LocalSnapshotStorage:231 - Renaming /tmp/server3/snapshot/temp to /tmp/server3/snapshot/snapshot_1002
2019-08-20 11:23:50 [JRaft-FSMCaller-Disruptor-0] INFO  StateMachineAdapter:79 - onConfigurationCommitted: localhost:8001,localhost:8002,localhost:8003
2019-08-20 11:23:50 [JRaft-FSMCaller-Disruptor-0] INFO  SnapshotExecutorImpl:435 - Node <test/localhost:8003> onSnapshotLoadDone, last_included_index: 1002
last_included_term: 2
peers: "localhost:8001"
peers: "localhost:8002"
peers: "localhost:8003"

在清除全部数据后,正确安装了 snapshot 并启动。

kdlan commented 4 years ago

我拿 CounterServer 试了下也重现了,操作如下

# terminal 1
> mvn exec:java -Dexec.mainClass=com.alipay.sofa.jraft.example.counter.CounterServer -Dexec.args="/tmp/server1 counter 127.0.0.1:8081 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"

# terminal 2
> mvn exec:java -Dexec.mainClass=com.alipay.sofa.jraft.example.counter.CounterServer -Dexec.args="/tmp/server2 counter 127.0.0.1:8082 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"

# terminal 3
> mvn exec:java -Dexec.mainClass=com.alipay.sofa.jraft.example.counter.CounterServer -Dexec.args="/tmp/server3 counter 127.0.0.1:8083 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"

等集群达成一致以后再在 terminal 3 执行

# terminal 3
# ctrl+c
> rm -rf /tmp/server3
> mvn exec:java -Dexec.mainClass=com.alipay.sofa.jraft.example.counter.CounterServer -Dexec.args="/tmp/server3 counter 127.0.0.1:8083 127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083"

然后日志就会一直报错 reject term_unmatched

Sofa-Middleware-Log SLF4J : Actual binding is of type [ com.alipay.remoting Log4j2 ]
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] INFO  log:30 - Sofa-Middleware-Log SLF4J : Actual binding is of type [ com.alipay.remoting Log4j2 ]
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] INFO  NodeImpl:455 - The number of active nodes increment to 1.
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] INFO  FSMCallerImpl:200 - Starts FSMCaller successfully.
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] WARN  LocalSnapshotStorage:306 - No data for snapshot reader /tmp/server3/snapshot
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] INFO  NodeImpl:835 - Node <counter/127.0.0.1:8083> init, term=0, lastLogId=LogId [index=0, term=0], conf=127.0.0.1:8081,127.0.0.1:8082,127.0.0.1:8083, oldConf=.
2019-08-20 17:17:26 [com.alipay.sofa.jraft.example.counter.CounterServer.main()] INFO  RaftGroupService:136 - Start the RaftGroupService successfully.
Started counter server at port:8083
2019-08-20 17:17:26 [Rpc-netty-server-worker-1-thread-1] WARN  RaftRpcServerFactory:315 - JRaft SET bolt.rpc.dispatch-msg-list-in-default-executor to be false for replicator pipeline optimistic.
2019-08-20 17:17:26 [counter/127.0.0.1:8083-AppendEntriesThread0] INFO  LocalRaftMetaStorage:132 - Save raft meta, path=/tmp/server3/raft_meta, term=1, votedFor=0.0.0.0:0, cost time=15 ms
2019-08-20 17:17:26 [counter/127.0.0.1:8083-AppendEntriesThread0] WARN  NodeImpl:1650 - Node <counter/127.0.0.1:8083> reject term_unmatched AppendEntriesRequest from 127.0.0.1:8081, term=1, prevLogIndex=1, prevLogTerm=1, localPrevLogTerm=0, lastLogIndex=0, entriesSize=0.
2019-08-20 17:17:26 [JRaft-FSMCaller-Disruptor-0] INFO  StateMachineAdapter:89 - onStartFollowing: LeaderChangeContext [leaderId=127.0.0.1:8081, term=1, status=Status[ENEWLEADER<10011>: Raft node receives message from new leader with higher term.]]
2019-08-20 17:17:26 [counter/127.0.0.1:8083-AppendEntriesThread0] WARN  NodeImpl:1650 - Node <counter/127.0.0.1:8083> reject term_unmatched AppendEntriesRequest from 127.0.0.1:8081, term=1, prevLogIndex=1, prevLogTerm=1, localPrevLogTerm=0, lastLogIndex=0, entriesSize=0.
2019-08-20 17:17:27 [counter/127.0.0.1:8083-AppendEntriesThread0] WARN  NodeImpl:1650 - Node <counter/127.0.0.1:8083> reject term_unmatched AppendEntriesRequest from 127.0.0.1:8081, term=1, prevLogIndex=1, prevLogTerm=1, localPrevLogTerm=0, lastLogIndex=0, entriesSize=0.

这样看不仅仅是snapshot拿不到,删除文件重启后连正常log append也有问题

killme2008 commented 4 years ago

完全按照你的步骤,仍然无法重现,只有一条 warn ,onStartFollowing 后就没有了:

2019-08-20 17:56:29 [JRaft-FSMCaller-Disruptor-0] INFO  StateMachineAdapter:89 - onStartFollowing: LeaderChangeContext [leaderId=127.0.0.1:8082, term=5, status=Status[ENEWLEADER<10011>: Raft node receives message from new leader with higher term.]]
2019-08-20 17:56:58 [JRaft-Closure-Executor-2] INFO  LocalSnapshotStorage:167 - Deleting snapshot /tmp/server3/snapshot/snapshot_2
2019-08-20 17:56:58 [JRaft-Closure-Executor-2] INFO  LocalSnapshotStorage:231 - Renaming /tmp/server3/snapshot/temp to /tmp/server3/snapshot/snapshot_2
killme2008 commented 4 years ago

刚启动的时候,有这个 reject 日志很正常,因为新节点还没有加入选举,没有成为 follower

2019-08-20 17:56:29 [counter/127.0.0.1:8083-AppendEntriesThread0] WARN  NodeImpl:1650 - Node <counter/127.0.0.1:8083> reject term_unmatched AppendEntriesRequest from 127.0.0.1:8082, term=5, prevLogIndex=2, prevLogTerm=5, localPrevLogTerm=0, lastLogIndex=0, entriesSize=0.
2019-08-20 17:56:29 [JRaft-FSMCaller-Disruptor-0] INFO  StateMachineAdapter:89 - onStartFollowing: LeaderChangeContext [leaderId=127.0.0.1:8082, term=5, status=Status[ENEWLEADER<10011>: Raft node receives message from new leader with higher term.]]
2019-08-20 17:56:58 [JRaft-Closure-Executor-2] INFO  LocalSnapshotStorage:167 - Deleting snapshot /tmp/server3/snapshot/snapshot_2
2019-08-20 17:56:58 [JRaft-Closure-Executor-2] INFO  LocalSnapshotStorage:231 - Renaming /tmp/server3/snapshot/temp to /tmp/server3/snapshot/snapshot_2

成为 follower 之后就没有了。

killme2008 commented 4 years ago

@kdlan 重现了,反复重复清除和重启 server3 的步骤就可以重现。初步定为应该是内存中的 replicator 状态紊乱引(仍然保存着清除前的节点的状态)起的,这种情况临时解决办法可以通过重启 leader 解决,也可以按照你说的重新加入来解决。

更进一步的修复我们先分析下,看看怎么解决比较好。理论上重建下 replicator 即可。

kdlan commented 4 years ago

@killme2008 我也发现了,好像是有多几次snapshot后,再关闭,删除,重启后就有时候正常,有时候有问题

fengjiachun commented 4 years ago

@kdlan 应该是以下原因:

  1. 集群启动成功以后(3节点),leader 已经成功将 log 复制到 2 个 follower,假设此时 prevLogIndex == 1
  2. 此时你重启并删除 node3 (follower)的数据
  3. leader 中 replicator 的内存状态仍认为 prevLogIndex == 1 并发心跳给重启的 node3
  4. 因为 node3 发现自己本地的日志与 leader 不符,所以node3 对这个心跳拒绝并返回失败,如此循环

恰好的你的试验里整个 raft group 从未有新的写入即 AppendEntries Request,如果有新的写入就会有新的 raft log 需要复制即AppendEntries Request,这个重启的 node3 会立刻恢复正常

以上,所以我在 onHeartbeatReturned 方法内判断如下条件成立的话 !response.getSuccess() && response.getLastLogIndex() == 0L ,会触发一次 r.sendEmptyEntries(false) 促使集群恢复正常

我不确定思路是没有漏洞的 @killme2008 可以 review 以下,这个 PR 我也会跑一下 jepsen 验证