sofastack / sofa-jraft

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

jraft一个节点与集群其他节点状态不一致 #952

Closed yuyijq closed 1 year ago

yuyijq commented 1 year ago

HI,我在使用Nacos的过程中发现其中一个节点的状态与集群状态不一致,不管怎么重启这个节点一直不能恢复(我没有重启整个集群)。Nacos对metadata的更新使用了jraft,然后我就使用arthas跟了一下,发现jraft的leader不向出问题的follower发送appendEntries。跟踪到最后大概就是在LogManagerImpl的waitMap里,根本就没有出问题follower的记录,看代码好像是log append成功后,会遍历waitMap里的记录,然后向follower发送AppendEntriesRequest。因为对jraft的代码目前还不熟悉,没法进一步追踪下去了。 不知道社区里有没有其他人碰到类似情况,或者还需要我提供什么其他信息,谢谢。 使用的是jraft 1.3.8版本,也没有在jraft的release notes里看到后续版本相关的bugfix。 另外我通过arthas和tcpdump能看到出问题的follower和leader之间心跳是正常的。

killme2008 commented 1 year ago

请先向 nacos 项目报告。

yuyijq commented 1 year ago

请先向 nacos 项目报告。

已经向Nacos社区报告过了

yuyijq commented 1 year ago

@killme2008 这个问题我大概复现了,初看起来和Nacos的用法相关,不过感觉和jraft也有一定的关系,我先描述一下,大佬可以看看。 背景

  1. Nacos 2.1.2 版本
  2. JRaft 1.3.8版本

Nacos初始化JRaft的顺序:

  1. 先初始化RpcServer
  2. 初始化multi-raft RpcServer rpcServer = createRpcServer(); rpcServer.init(null); RaftGroupService raftGroupService = new RaftGroupService(groupName, localPeerId, copy, rpcServer, true); Node node = raftGroupService.start(false);

我觉得这里可能是问题所在,虽然RaftGroupService依赖RpcServer,但是可以先不调用 RpcServer的init,只创建。按照下面的顺序: RpcServer rpcServer = createRpcServer(); RaftGroupService raftGroupService = new RaftGroupService(groupName, localPeerId, copy, rpcServer, true); Node node = raftGroupService.start(false); rpcServer.init(null);

Nacos的初始化顺序可能导致JRaft可能没初始化完成,rpc就向外提供服务了。 这本来问题不大,但是如果正好当前启动的这个node的raft数据目录被清空(我们当时是这个情况),然后Leader正好有一个AppendEntriesRequest发送过来,在NodeImpl的handleAppendEntriesRequest方法的1947行开始的逻辑就会向leader回复一个success = false,lastLogIndex = 0 的响应。

            final long prevLogIndex = request.getPrevLogIndex();
            final long prevLogTerm = request.getPrevLogTerm();
            //因为磁盘数据被清空,所以localPrevLogTerm = 0
            final long localPrevLogTerm = this.logManager.getTerm(prevLogIndex);
            if (localPrevLogTerm != prevLogTerm) {
                //因为磁盘数据被清空,所以lastLogIndex = 0
                final long lastLogIndex = this.logManager.getLastLogIndex();
                return AppendEntriesResponse.newBuilder() //
                    .setSuccess(false)
                    .setTerm(this.currTerm)
                    .setLastLogIndex(lastLogIndex)
                    .build();
            }

那么Leader侧Replicator的onAppendEntriesReturned如下逻辑会执行:

if (!response.getSuccess()) {
            //response.getLastLogIndex() = 0
            if (response.getLastLogIndex() + 1 < r.nextIndex) {
                r.nextIndex = response.getLastLogIndex() + 1; // nextIndex = 1
            } else {
                  //...
            }

这样Replicator的nextIndex就会等于1,而一旦nextIndex等于1之后,Leader是无法从向Follower的心跳中恢复到发送AppendEntriesRequest的循环中来的。

更正:下面一段的描述不对,r.block会等待一段时间之后重新进入循环 ~~ 这个时候如果这个Follower crash了,则onAppendEntriesReturned的如下逻辑会执行:

 if (!status.isOk()) {
            // If the follower crashes, any RPC to the follower fails immediately,
            // so we need to block the follower for a while instead of looping until
            // it comes back or be removed
            // dummy_id is unlock in block
            if (isLogDebugEnabled) {
                sb.append(" fail, sleep, status=") //
                    .append(status);
                LOG.debug(sb.toString());
            }
            notifyReplicatorStatusListener(r, ReplicatorEvent.ERROR, status);
            if (++r.consecutiveErrorTimes % 10 == 0) {
                LOG.warn("Fail to issue RPC to {}, consecutiveErrorTimes={}, error={}", r.options.getPeerId(),
                    r.consecutiveErrorTimes, status);
            }
            r.resetInflights();
            r.setState(State.Probe);
            // unlock in in block
            r.block(startTimeMs, status.getCode());
            return false;
        }

~~

此时因为raft磁盘文件被清空,所以下面的逻辑也会退出:

        //response.getTerm() == 0
        if (response.getTerm() != r.options.getTerm()) {
            r.resetInflights();
            r.setState(State.Probe);
            LOG.error("Fail, response term {} dismatch, expect term {}", response.getTerm(), r.options.getTerm());
            id.unlock();
            return false;
        }

这个时候Leader相当于退出了向该Follower发送AppendEntriesRequest的循环(这里排除心跳的),需要等着心跳来触发,而只有onHeartbeatReturned的如下逻辑会触发:

if (!response.getSuccess() && response.hasLastLogIndex()) {
                if (isLogDebugEnabled) {
                    sb.append(" fail, response term ") //
                        .append(response.getTerm()) //
                        .append(" lastLogIndex ") //
                        .append(response.getLastLogIndex());
                    LOG.debug(sb.toString());
                }
                LOG.warn("Heartbeat to peer {} failure, try to send a probe request.", r.options.getPeerId());
                doUnlock = false;
                r.sendProbeRequest();
                r.startHeartbeatTimer(startTimeMs);
                return;
            }

但是因为nextIndex=1,发送给Follower的prevLogIndex和prevLogTerm都会等于0,Follower下面的逻辑不会再触发了:

final long prevLogIndex = request.getPrevLogIndex();
            final long prevLogTerm = request.getPrevLogTerm();
            final long localPrevLogTerm = this.logManager.getTerm(prevLogIndex);
            if (localPrevLogTerm != prevLogTerm) {
                final long lastLogIndex = this.logManager.getLastLogIndex();
                return AppendEntriesResponse.newBuilder() //
                    .setSuccess(false) //
                    .setTerm(this.currTerm) //
                    .setLastLogIndex(lastLogIndex) //
                    .build();
            }

那么Leader就不能从heartbeat的循环里进入到sendProbeRequest,也就不能再次进入到正常的AppendEntriesRequest循环。到这种情况后,不管怎么重启Follower都没有效果,只有重启Leader。

麻烦大佬看看,谢谢

killme2008 commented 1 year ago

sorry,我没有细看,但是无论如何不应该去手工清除 raft 数据目录,尤其是启动后,数据错误,神仙都救不了。

yuyijq commented 1 year ago

sorry,我没有细看,但是无论如何不应该去手工清除 raft 数据目录,尤其是启动后,数据错误,神仙都救不了。

你是对的,但是其实这不是数据错误,是整个数据被清理了,我觉得这个应该也是合理的吧,比如一个节点出现数据损坏,然后把数据目录清理掉,然后重启,这个应该是正常的操作。

killme2008 commented 1 year ago

sorry,我没有细看,但是无论如何不应该去手工清除 raft 数据目录,尤其是启动后,数据错误,神仙都救不了。

你是对的,但是其实这不是数据错误,是整个数据被清理了,我觉得这个应该也是合理的吧,比如一个节点出现数据损坏,然后把数据目录清理掉,然后重启,这个应该是正常的操作。

你不能在节点启动后再去清理数据,这样磁盘的数据和内存的数据很可能不一致,也就可能出现你所描述的现象,这已经脱离了一个类库框架应该处理的范围了,反而是使用上的问题。

killme2008 commented 1 year ago

哪怕是要清理,也应该是停机的状态去清理,否则内存状态跟持久的状态就会出现不一致,这个应该很好理解。

yuyijq commented 1 year ago

哪怕是要清理,也应该是停机的状态去清理,否则内存状态跟持久的状态就会出现不一致,这个应该很好理解。

是停止后清理的。实际场景是这样的,我们用Docker运行Nacos,但是磁盘没有挂载,导致重启后相当于啥也没有了,然后就出现上面这种现象,只重启了一个节点。

killme2008 commented 1 year ago

大概看了下,你这里的问题其实很简单,就是这个 follower 的 replicator 在 leader 的状态发生了突变,重启前有 term 和正常在发的 log index,重启后 term 和 log 都没了,和 leader 原本预期的 replicator 状态不匹配,导致 leader 无所适从。这里 leader 只能假设数据出现了损坏,保守起见,没有帮你主动纠正。这种情况:

  1. 你可以将该节点从集群里移除,然后重新加入;
  2. 或者调用 Node#transferLeader 到另一个节点,让新 leader 重新确认所有 follower 状态,重建所有 replicator
  3. 也许 jraft 可以提供一个方法,来销毁重建这个 replicator
yuyijq commented 1 year ago

@killme2008 感谢回复 你看是不是可以在onHeartbeatReturned里,只要rpc成功了,然后就调用一次 r.sendProbeRequest(); 这样也可以将这个Replicator拉回来。 现在是在onHeartbeatReturned里,只有if (!response.getSuccess() && response.hasLastLogIndex())条件满足的时候才会sendProbeRequest(),但是这个条件在我上面描述的情况中,一直得不到满足。 看起来这样调用一次sendProbeRequest好像也没啥问题..

yuyijq commented 1 year ago

@killme2008 关于这个问题又有一些新的发现了

前面分析了 Replicator里的nextIndex 为什么等于1了,这个是因为JRaft的数据目录内容被清空导致的。然后正好今天分析了一下,出问题集群的Leader的JVM Heap dump,发现以下一些信息:

  1. Leader内存中出问题Follower对应的Replicator的reader字段是有值的,这个说明installSnapshot()曾经被调用过,且至少执行到了Replicator的625行:
    this.reader = this.options.getSnapshotStorage().open();
  2. 该Replicator的installSnapshotCounter值为0,那么表示installSnapshot()方法中途退出了,并没有执行到结束。在673行有如下代码,没有执行到这里:
    // noinspection NonAtomicOperationOnVolatileField
    this.installSnapshotCounter++; //dump中该值为0
  3. 在this.installSnapshotCounter++ 这行代码之前,有两次return,但是这两次return都会导致Replicator的reader对象被设置为null,现在reader不为null,则应该不是这两个return导致提前退出的:
            final String uri = this.reader.generateURIForCopy();
            if (uri == null) {
                final NodeImpl node = this.options.getNode();
                final RaftException error = new RaftException(EnumOutter.ErrorType.ERROR_TYPE_SNAPSHOT);
                error.setStatus(new Status(RaftError.EIO, "Fail to generate uri for snapshot reader"));
                //这里会将reader设置为null
                releaseReader();
                unlockId();
                doUnlock = false;
                node.onError(error);
                return;
            }
            final RaftOutter.SnapshotMeta meta = this.reader.load();
            if (meta == null) {
                final String snapshotPath = this.reader.getPath();
                final NodeImpl node = this.options.getNode();
                final RaftException error = new RaftException(EnumOutter.ErrorType.ERROR_TYPE_SNAPSHOT);
                error.setStatus(new Status(RaftError.EIO, "Fail to load meta from %s", snapshotPath));
                //这里会将reader设置为null
                releaseReader();
                unlockId();
                doUnlock = false;
                node.onError(error);
                return;
            }
  4. 在Replicator的635行final String uri = this.reader.generateURIForCopy();这行代码里会调用FileService.getInstance().addReader(reader),这是一个单例,如果这行代码调用过,那么jvm heap dump里应该肯定有FileService的实例,不会被GC掉,但是现在一个都没有:

    @Override
    public String generateURIForCopy() {
        if (this.addr == null || this.addr.equals(new Endpoint(Utils.IP_ANY, 0))) {
            LOG.error("Address is not specified");
            return null;
        }
        if (this.readerId == 0) {
            final SnapshotFileReader reader = new SnapshotFileReader(this.path, this.snapshotThrottle);
            reader.setMetaTable(this.metaTable);
            if (!reader.open()) {
                LOG.error("Open snapshot {} failed.", this.path);
                return null;
            }
            //这里访问过FileService.getInstance(),这是一个单例,如果访问过的话jvm heap dump里应该有FileService对象
            this.readerId = FileService.getInstance().addReader(reader);
            if (this.readerId < 0) {
                LOG.error("Fail to add reader to file_service.");
                return null;
            }
        }
    
        return String.format(REMOTE_SNAPSHOT_URI_SCHEME + "%s/%d", this.addr.toString(), this.readerId);
    }
  5. 另外,Replicator的readerId = 0 综上所有的信息,应该可以确定,上面的这个代码在下面这个位置应该有异常抛出,是异常抛出,而不是返回null,因为返回null的话releaseReader()就会执行,reader就会为null。
            final SnapshotFileReader reader = new SnapshotFileReader(this.path, this.snapshotThrottle);
            reader.setMetaTable(this.metaTable);
            if (!reader.open()) {
                LOG.error("Open snapshot {} failed.", this.path);
                return null;
            }

    所以这也解释了为什么reader不等于null,但是installSnapshotCounter的值为0,因为installSnapshot()是异常退出!!! 所以下面的代码就提前退出了,导致waitMoreEntries没有执行,退出了appendEntries循环:

    if (rb.getEntriesCount() == 0) {
                if (nextSendingIndex < this.options.getLogManager().getFirstLogIndex()) {
                    installSnapshot();
                    return false;
                }
                // _id is unlock in _wait_more
                waitMoreEntries(nextSendingIndex);
                return false;
            }

另外,因为nextIndex = 1,我们无法从心跳中恢复回来(这个前面已经有分析),所以这个Replicator算废掉了。

结论就是:

  1. 一台Follower的JRaft数据目录被清空( PS: 我觉得这是合理的,比如假设磁盘损坏等问题,我们为了处理故障,可能会停止服务,清理数据,启动,以这样的流程恢复故障)
  2. installSnapshot() 调用出现异常,异常退出,没有对异常进行捕获。

可惜的是,因为服务器有日志清理,这个问题其实是3.19号就出现了,我们直到3.31号才发现,所以日志没有保留下来,现在只能根据现有的信息进行推测,无法确定当时抛出的是什么异常,但是我觉得以上推测是符合逻辑的。

killme2008 commented 1 year ago

@yuyijq 👍🏻,分析的不错,看情况应该是走了 installSnapshot,也就是这里

https://github.com/sofastack/sofa-jraft/blob/fdf737850097eb1bf20d34b8caaa0bb6f0c92ad7/jraft-core/src/main/java/com/alipay/sofa/jraft/core/Replicator.java#L1623

为啥 installSnapshot 失败,从你给的信息来看, open 不可能失败,因为就是一个 File#exists 调用。为啥失败,可能要去日志中找答案了。

sofa 框架的日志我没记错的话,都在 ~/logs 下面,重点看看 common-error.log 这个日志。

yuyijq commented 1 year ago

@killme2008 Hi, 遗憾的是我们服务器上这个日志没有被采集,然后日志又有保留天数,所以日志现在已经没法查了,现在手头只有一个出问题后的jvm heap dump。

不过我在想的是,类似Replicator这个地方其实是一个类似 eventloop一样,我们是不是要考虑这种因为异常而退出了这个loop的情况?无论是catch异常,或者在finally里兜底。

killme2008 commented 1 year ago

jraft 本身有不少容错上的设计了,从原则上讲,框架或者类库还是有什么问题就解决什么问题,而不是尝试去兜底,这样容易掩盖一些潜在的问题,这也是 jraft 基本的设计原则。

上述的一些分析仍然只是猜测,如果有一个非常简单的 example 可以复现是最好的。否则还是停留在猜测。

我觉的你们可以加一点基本的监控,比如节点状态的监控,比如 follower 和 leader 之间state machine 的 applied log index 的进度差值等,可以更早地提前发现并介入,也能有现场的日志。jraft 本身也提供了大量的 metric 可以观测。

killme2008 commented 1 year ago

jraft 本身有不少容错上的设计了,从原则上讲,框架或者类库还是有什么问题就解决什么问题,而不是尝试去兜底,这样容易掩盖一些潜在的问题,这也是 jraft 基本的设计原则。

上述的一些分析仍然只是猜测,如果有一个非常简单的 example 可以复现是最好的。否则还是停留在猜测。

我觉的你们可以加一点基本的监控,比如节点状态的监控,比如 follower 和 leader 之间state machine 的 applied log index 的进度差值等,可以更早地提前发现并介入,也能有现场的日志。jraft 本身也提供了大量的 metric 可以观测。

yuyijq commented 1 year ago

@killme2008 今天终于又复现了这个问题,然后赶紧看了一下日志,看到了报错原因 FileServie 依赖io.netty.util.internal.ThreadLocalRandom类,然后报下面的错 Caused by: java.lang.ClassNotFoundException: io.netty.util.internal.ThreadLocalRandom ...

看了一下,最新版本的JRaft已经不依赖netty的这个ThreadLocalRandom类了,改成依赖jdk的 具体原因是JRaft是通过com.alipay.sofa:bolt间接依赖netty的,然后nacos引入JRaft的时候,不知道为啥把bolt给排除了..

到此为止,这个问题终于查清楚了,感谢大佬,我们升级一下版本

killme2008 commented 1 year ago

没帮上什么忙,查清楚就好 👍🏻