sofastack / sofa-jraft

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

data and closure of task maybe does not match,out-of-order #341

Closed hanzhihua closed 4 years ago

hanzhihua commented 4 years ago

Your question

应用重启时偶尔会出现下面这样的错误

2019-11-18 19:08:06.691 WARN leader [JRaft-NodeImpl-Disruptor-0]-c.a.s.j.storage.impl.LogManagerImpl.checkAndResolveConflict - Received entries of which the lastLog=0 is not greater than appliedIndex=315546172, return immediately with nothing changed. 2019-11-18 19:08:06.691 ERROR leader [JRaft-Closure-Executor-201]-com.alipay.sofa.jraft.core.NodeImpl.run - Node <archer_control/10.69.0.33:6024> append [0, 1] failed.

后面感觉就是closuse跟data关系有点乱了,我看了一下代码里,面写ClosureQueue跟写Log是并行处理的,另外replicator从logmanager里面拿日志跟follower去同步,然后在做onCommmited,然后再是状态机apply.

感觉这个过程中data和closure是会存在乱序的可能性,而现在出现业务错误,也感觉是乱了,请赐教

Environment

jdk 1.8 3台集群(jraft 版本1.2.6) uname :4.9.0-0.bpo.6-amd64 #1 SMP Debian 4.9.88-1+deb9u1~bpo8+1 (2018-05-13) x86_64 GNU/Linux

sofastack-bot[bot] commented 4 years ago

Hi @hanzhihua, 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 Thread restart is the following error 2019-11-18 19:08:06.691 WARN leader [JRaft-NodeImpl-Disruptor-0]-casjstorage.impl.LogManagerImpl.checkAndResolveConflict - Received entries of which the lastLog =0 is not greater than appliedIndex=315546172, return immediately with nothing changed. 2019-11-18 19:08:06.691 ERROR leader [JRaft-Closure-Executor-201]-com.alipay.sofa.jraft.core.NodeImpl. Run - Node <archer_control/10.69.0.33:6024> Append [0, 1] failed. The feeling behind is that closuse is a bit messy with data. I looked at the code and wrote ClosureQueue and write Log in parallel. In addition, the replicator takes the log from the logmanager to synchronize with the follower, and then does it. onCommmited, then state machine apply. Feel the possibility that data and closure will be out of order during this process### Environment - SOFAJRaft version: - JVM version (eg java -version): - OS version (eg Uname -a): - Maven version: - IDE version:

killme2008 commented 4 years ago

乱序是不大可能的,整个写的过程都是写锁串行控制的,这个错误也不影响,发错了会重新探测,重新发。 除了重启之外,你还有没有做其他操作?有没有其他错误信息。 比如清空数据之类。基于现有信息无法判断。具体到业务错误,也给出更多的信息为好。

hanzhihua commented 4 years ago

这种情况会乱序吗? image

killme2008 commented 4 years ago

乱序是果,不是因,这里的情况是你的状态机已经 apply 到 315546172,但是你的 log storage 里却是空的,最大日志重新从 0 开始,所以要问为什么会出现这个状态。这种状态已经完全异常了。

masaimu commented 4 years ago

看你的日志,LogEntry list 里面的内容应该是: firstLogEntry.getId().getIndex() != 0 && lastLogEntry.getId().getIndex() == 0 这种情况的 log 已经是有问题了,怀疑是不是有过删除 log 之类的操作

hanzhihua commented 4 years ago

还真的没有删除日志,我也迷惑了,我在研究一下,就是发布那段时间有问题,其他都蛮好:)

killme2008 commented 4 years ago

你给下你的 raft options 列表,另外是不是用了 snapshot,怎么使用的。我比较怀疑是 options 设置有问题,或者 snapshot 使用方式有问题。

hanzhihua commented 4 years ago

你好,options: nodeId: state: STATE_LEADER term: 10 conf: ConfigurationEntry [id=LogId [index=330456742, term=10], conf=10.69.0.33:6024,10.69.0.34:6024,10.69.1.11:6024, oldConf=] electionTimer: RepeatedTimer [timerTask=null, stopped=true, running=false, destroyed=false, invoking=false, timeoutMs=3000] voteTimer: RepeatedTimer [timerTask=null, stopped=true, running=false, destroyed=false, invoking=false, timeoutMs=3000] stepDownTimer: RepeatedTimer [timerTask=com.alipay.sofa.jraft.util.RepeatedTimer$1@5b611a2b, stopped=false, running=true, destroyed=false, invoking=false, timeoutMs=1500] snapshotTimer: RepeatedTimer [timerTask=com.alipay.sofa.jraft.util.RepeatedTimer$1@17986965, stopped=false, running=true, destroyed=false, invoking=false, timeoutMs=3600000] logManager: storage: [329708782, 330710565] diskId: LogId [index=330710565, term=10] appliedId: LogId [index=330710565, term=10] lastSnapshotId: LogId [index=330456742, term=10] fsmCaller: StateMachine [Idle] ballotBox: lastCommittedIndex: 330710565 pendingIndex: 330710566 pendingMetaQueueSize: 0 snapshotExecutor: lastSnapshotTerm: 10 lastSnapshotIndex: 330456742 term: 9 savingSnapshot: false loadingSnapshot: false stopped: false replicatorGroup: replicators: [Replicator [state=Replicate, statInfo=,peerId=10.69.0.33:6024], Replicator [state=Replicate, statInfo=,peerId=10.69.1.11:6024]] failureReplicators: []

用了snapshot了,下面是用法,跟count例子查不多,唯一改动的是,如果snapshot出错了,只是打印日志 ‘public void onSnapshotSave(final SnapshotWriter writer, final Closure done) { log.warn("onSnapshotSave..."); Utils.runInThread(() -> { final CNStateSnapshotFile snapshot = new CNStateSnapshotFile(writer.getPath() + File.separator + DATA_FILE_NAME); if (snapshot.save(cnStateInner)) { if (writer.addFile(DATA_FILE_NAME)) { done.run(Status.OK()); return ; }else{ log.error("write addFile:{} occur error,and ignore closure",snapshot.getPath()); } }else{ log.error("snapshot.save:{} occur error,and ignore closure",snapshot.getPath()); } }); }

@Override
public boolean onSnapshotLoad(final SnapshotReader reader) {
    log.warn("onSnapshotLoad...");
    if (isLeader()) {
        log.warn("Leader is not supposed to load snapshot");
        return false;
    }
    if (reader.getFileMeta(DATA_FILE_NAME) == null) {
        log.error("Fail to find data file in {}", reader.getPath());
        return false;
    }
    final CNStateSnapshotFile snapshot = new CNStateSnapshotFile(reader.getPath() + File.separator + DATA_FILE_NAME);
    try {
        CNState loadState = snapshot.load();
        if (loadState != null) {
            cnStateInner.load(loadState);
        }
        return true;
    } catch (final Exception e) {
        log.error("Fail to load snapshot from {}", snapshot.getPath());
        return false;
    }
}

killme2008 commented 4 years ago
  1. 你还是没有给出你设置的 RaftOptions 选项。
  2. onSnapshotLoad 建议加入 meta 的日志:
log.warn("onSnapshotLoad,meta ={}", reader.load());

在出现这个错误的时候,有没有 snapshot 相关错误,你的代码里有没有对 log 做一些主动操作等。

hanzhihua commented 4 years ago

@killme2008 谢谢你的建议,我已经加上了 Raftoptions全是默认值,我没有设置,另外启动时候没有snapshot错误,也没有操作log做过操作

另外 2019-11-18 19:08:06.691 WARN leader [JRaft-NodeImpl-Disruptor-0]-c.a.s.j.storage.impl.LogManagerImpl.checkAndResolveConflict - Received entries of which the lastLog=0 is not greater than appliedIndex=315546172, return immediately with nothing changed. 这样warning一般是什么时候回出现了呢?

masaimu commented 4 years ago

Follower 在收到 logEntries 之后,会校验是否能 append 到本地的日志末尾,这个是校验失败的情况。表示 logEntries 中所有的日志从 index 来说在 Follower 上都已经 apply 了,无需再做什么操作。

killme2008 commented 4 years ago

@masaimu 他这个是 leader 的错误。现象就是状态机 apply 到 315546172,但是新增的日志 index 却是从 0 重新开始,可能的原因就是日志被意外清空了

建议将这个错误前后的日志都发出来看下,特别是 error/warn 级别的。

hanzhihua commented 4 years ago

是的,这台机器是leader,下面是那个时候日志(Warn和error) 2019-11-18 19:08:06.219 WARN leader [JRaft-Closure-Executor-32]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=150, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:06.417 WARN leader [JRaft-Closure-Executor-41]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=160, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:06.691 ERROR leader [JRaft-NodeImpl-Disruptor-0]-c.a.s.jraft.util.LogExceptionHandler.handleEventException - Handle NodeImpl disruptor event error, event is com.alipay.sofa.jraft.core.NodeImpl$LogEntryAndClosure@6e84f714 2019-11-18 19:08:06.691 WARN leader [JRaft-NodeImpl-Disruptor-0]-c.a.s.j.storage.impl.LogManagerImpl.checkAndResolveConflict - Received entries of which the lastLog=0 is not greater than appliedIndex=315546172, return immediately with nothing changed. 2019-11-18 19:08:06.691 ERROR leader [JRaft-Closure-Executor-201]-com.alipay.sofa.jraft.core.NodeImpl.run - Node <archer_control/10.69.0.33:6024> append [0, 1] failed. 2019-11-18 19:08:06.830 WARN leader [JRaft-Closure-Executor-110]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=170, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:06.983 WARN leader [JRaft-Closure-Executor-169]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=180, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:07.132 WARN leader [JRaft-Closure-Executor-169]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=190, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:07.197 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=200, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:07.230 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=210, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:07.529 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=220, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:07.851 WARN leader [JRaft-Closure-Executor-100]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=230, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.007 WARN leader [JRaft-Closure-Executor-100]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=240, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.174 WARN leader [JRaft-Closure-Executor-100]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=250, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.205 WARN leader [JRaft-Closure-Executor-169]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=260, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.395 WARN leader [JRaft-Closure-Executor-169]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=270, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.715 WARN leader [waitCheckScheduler 11-0]-c.b.a.controlnode.ha.CNStateProxy.doTask - raft operation timeout, raftRequest:CNStatRequest{action='DEPENDENCY_REMOVE_TO_LAST', key=8495565, value=null}, cost:1000ms 2019-11-18 19:08:08.721 WARN leader [JRaft-Closure-Executor-169]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=280, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:08.864 WARN leader [JRaft-Closure-Executor-149]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=290, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:09.012 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=300, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:09.178 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=310, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:09.208 WARN leader [JRaft-Closure-Executor-186]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=320, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 2019-11-18 19:08:09.399 WARN leader [JRaft-Closure-Executor-123]-c.alipay.sofa.jraft.core.Replicator.onAppendEntriesReturned - Fail to issue RPC to 10.69.1.11:6024, consecutiveErrorTimes=330, error=Status[EINTERNAL<1004>: Fail to send a RPC request:Create connection failed. The address is 10.69.1.11:6024] 帮忙分析一下,当时3个节点,有一个节点正在重启

killme2008 commented 4 years ago

这个日志看不出来什么,你将你的详细的操作描述下吧,特别关注启动的时候 jraft 的日志。

killme2008 commented 4 years ago

找找有没有这个日志 is initialized with inconsistent log

hanzhihua commented 4 years ago

会不会跟发布顺序有关系呢?rocksjni是随机生成一个so文件吧,我发布的顺序是覆盖了原来的文件,会不会造成rocksdb工作不正常呢?

hanzhihua commented 4 years ago

@killme2008 没有出现你说的那个日志,发布的时候,也是偶尔出现这样的错误,不是必现

killme2008 commented 4 years ago

rocksdb 的替换不影响,除非你去修改了 raft snapshot/log/meta 的目录里的数据。

就你目前给出的信息,暂时无法分析出原因了。

killme2008 commented 4 years ago

另外确认的 RaftOptions 的 sync = true,否则是可能出现这个问题的。

hanzhihua commented 4 years ago

下面是nodeoptions,请参考 NodeOptions [electionTimeoutMs=3000, leaderLeaseTimeRatio=90, snapshotIntervalSecs=3600, catchupMargin=1000, initialConf=10.69.0.33:6024,10.69.0.34:6024,10.69.1.11:6024, fsm=controlnode.ha.CNStateMachine@41f57420, logUri=/mnt/storage00/archer-controlnode/raft/log, raftMetaUri=/mnt/storage00/archer-controlnode/raft/raft_meta, snapshotUri=/mnt/storage00/archer-controlnode/raft/snapshot, filterBeforeCopyRemote=false, disableCli=false, timerPoolSize=20, cliRpcThreadPoolSize=40, raftRpcThreadPoolSize=240, enableMetrics=false, snapshotThrottle=null, serviceFactory=com.alipay.sofa.jraft.core.DefaultJRaftServiceFactory@62e8ef9f, raftOptions=RaftOptions{maxByteCountPerRpc=131072, fileCheckHole=false, maxEntriesSize=1024, maxBodySize=524288, maxAppendBufferSize=262144, maxElectionDelayMs=1000, electionHeartbeatFactor=10, applyBatch=32, sync=true, syncMeta=false, replicatorPipeline=true, maxReplicatorInflightMsgs=256, disruptorBufferSize=16384, readOnlyOptions=ReadOnlySafe}]

killme2008 commented 4 years ago

你是一个节点上有多个 raft group ? multi raft group 的模式?

 Received entries of which the lastLog =0 is not greater than appliedIndex=315546172, return immediately with nothing changed

这个错误是 follower 才有的,可以忽略,会自动探测恢复。

Node <archer_control/10.69.0.33:6024> append [0, 1] failed.

这个错误却是 leader 特有的。

理论上说这两个是对不上的。

killme2008 commented 4 years ago

我理下:

  1. 你应该是重启,一台一台重启
  2. 然后 10.69.0.33 这台先是 follower,后来变成了 leader(由于原来的 leader 重启),这个错误就是切换过程中发生的,对不对?

如果是这样,两个错误信息都可以忽略,是正常的。

hanzhihua commented 4 years ago

写操作是系统调度自动驱动的,不是客户端发起的,如果是leader,就会发起写操作,判断是不是leader,就不会进入到NodeImpl.apply 判断是不是leader跟CounterStateMachine一样

killme2008 commented 4 years ago

@hanzhihua 我大概明白你这个场景了。错误的时候 10.69.0.33 肯定是发生了角色的切换,猜测是 leader -> follower。

写操作是系统调度自动驱动的,不是客户端发起的,如果是leader,就会发起写操作,判断是不是leader,就不会进入到NodeImpl.apply
判断是不是leader跟CounterStateMachine一样

判断通过,但是在通过后,可能发生了角色切换,导致写入会失败,类似 test-and-set 存在的竞态条件。

这里的情况就是这个阶段发生了角色切换,同时在:

这两个可能并行,不过没有关系,前者写入失败了,就是那条 append failed 日志,后者会重新探测,重新发。

你的 state machine 有没有加入 onStartFollowing onStopFollowing 以及 onLeaderStartonLeaderStart 的日志,找出来,看看是否跟我猜测的类似。

killme2008 commented 4 years ago

leader -> follower 角色切换的时候, ballotBox 的 task 都会清空,所有 closure 也不会串,具体见

NodeImpl#stepDown

  if (this.state == State.STATE_CANDIDATE) {
            stopVoteTimer();
        } else if (this.state.compareTo(State.STATE_TRANSFERRING) <= 0) {
            stopStepDownTimer();
            this.ballotBox.clearPendingTasks();
            // signal fsm leader stop immediately
            if (this.state == State.STATE_LEADER) {
                onLeaderStop(status);
            }
        }
hanzhihua commented 4 years ago

好像不是的,leader一直是10.69.0.33,但出现上述这些错误的时候,它一直是leader,没有看到切换,这个期间其他两台是在重启(一台一台重启)

killme2008 commented 4 years ago

@hanzhihua 建议你 grep 下吧, onLeaderStart 和 onLeaderStop,如果你是继承 StateMachineAdapter 的话。

fengjiachun commented 4 years ago

好像不是的,leader一直是10.69.0.33,但出现上述这些错误的时候,它一直是leader,没有看到切换,这个期间其他两台是在重启(一台一台重启)

你发的下面这段信息表示,当前 leader 是 10.69.0.34 这台机器,不是 10.69.0.33,所以 10.69.0.33 一直是 leader 不存在的

nodeId: state: STATE_LEADER term: 10 conf: ConfigurationEntry [id=LogId [index=330456742, term=10], conf=10.69.0.33:6024,10.69.0.34:6024,10.69.1.11:6024, oldConf=] electionTimer: RepeatedTimer [timerTask=null, stopped=true, running=false, destroyed=false, invoking=false, timeoutMs=3000] voteTimer: RepeatedTimer [timerTask=null, stopped=true, running=false, destroyed=false, invoking=false, timeoutMs=3000] stepDownTimer: RepeatedTimer [timerTask=com.alipay.sofa.jraft.util.RepeatedTimer$1@5b611a2b, stopped=false, running=true, destroyed=false, invoking=false, timeoutMs=1500] snapshotTimer: RepeatedTimer [timerTask=com.alipay.sofa.jraft.util.RepeatedTimer$1@17986965, stopped=false, running=true, destroyed=false, invoking=false, timeoutMs=3600000] logManager: storage: [329708782, 330710565] diskId: LogId [index=330710565, term=10] appliedId: LogId [index=330710565, term=10] lastSnapshotId: LogId [index=330456742, term=10] fsmCaller: StateMachine [Idle] ballotBox: lastCommittedIndex: 330710565 pendingIndex: 330710566 pendingMetaQueueSize: 0 snapshotExecutor: lastSnapshotTerm: 10 lastSnapshotIndex: 330456742 term: 9 savingSnapshot: false loadingSnapshot: false stopped: false replicatorGroup: replicators: [Replicator [state=Replicate, statInfo=,peerId=10.69.0.33:6024], Replicator [state=Replicate, statInfo=,peerId=10.69.1.11:6024]] failureReplicators: []

hanzhihua commented 4 years ago

这个是10.69.0.33 重启的时间点,是19:10,上面的错误发送在19:08 2019-11-18 19:10:23.167 INFO notaleader [JRaft-FSMCaller-Disruptor-0]-c.b.a.c.ha.AbstractCNStateMachine.onLeaderStop - leader to stop,status:Status[ESHUTDOWN<1007>: Raft node is going to quit.] 2019-11-18 19:10:23.178 INFO notaleader [JRaft-FSMCaller-Disruptor-0]-c.b.a.c.ha.AbstractCNStateMachine.onLeaderStop - leader stop ok

killme2008 commented 4 years ago

你要看的是错误之前的日志有没有 onLeaderStart 或者 onLeaderStop

之后要重启这台,他当然退出失去 leader 了。

hanzhihua commented 4 years ago

:),真的没有找到了,像这样的错误 "Received entries of which the lastLog=0 is not greater than "肯定不会出现在leader中吧

fengjiachun commented 4 years ago

这个是10.69.0.33 重启的时间点,是19:10,上面的错误发送在19:08 2019-11-18 19:10:23.167 INFO notaleader [JRaft-FSMCaller-Disruptor-0]-c.b.a.c.ha.AbstractCNStateMachine.onLeaderStop - leader to stop,status:Status[ESHUTDOWN<1007>: Raft node is going to quit.] 2019-11-18 19:10:23.178 INFO notaleader [JRaft-FSMCaller-Disruptor-0]-c.b.a.c.ha.AbstractCNStateMachine.onLeaderStop - leader stop ok

从这条日志看,发生的现象和你描述的不太相符,Status[ESHUTDOWN<1007>: Raft node is going to quit. 这个信息不是错误,这会发生在你主动 shutdown

fengjiachun commented 4 years ago

@hanzhihua 我大概明白你这个场景了。错误的时候 10.69.0.33 肯定是发生了角色的切换,猜测是 leader -> follower。

写操作是系统调度自动驱动的,不是客户端发起的,如果是leader,就会发起写操作,判断是不是leader,就不会进入到NodeImpl.apply
判断是不是leader跟CounterStateMachine一样

判断通过,但是在通过后,可能发生了角色切换,导致写入会失败,类似 test-and-set 存在的竞态条件。

这里的情况就是这个阶段发生了角色切换,同时在:

  • apply 任务的日志添加
  • 接收来自 leader 的请求

这两个可能并行,不过没有关系,前者写入失败了,就是那条 append failed 日志,后者会重新探测,重新发。

你的 state machine 有没有加入 onStartFollowing onStopFollowing 以及 onLeaderStartonLeaderStart 的日志,找出来,看看是否跟我猜测的类似。

我也认同这个分析,从上面的 node describe 信息看,也确实发生了切主,下面引用我之前的回复

你发的下面这段信息表示,当前 leader 是 10.69.0.34 这台机器,不是 10.69.0.33,所以 10.69.0.33 一直是 leader 不存在的

hanzhihua commented 4 years ago

发生错误的时候,确实leader没有切换,你看看线程

image

如果有的方法确定只能由leader做的话,有的方法确定由follower做的话,是不是可以通过判断角色来确定,再用业务参数做double check,这样代码可读性要强一点 ,不一定正确:)

fengjiachun commented 4 years ago

Reset log manager failed, nextLogIndex

能不能找到这行日志?

hanzhihua commented 4 years ago

没有,真的很奇怪,我不知道怎么就到这个环节里面去了,我的用法级别上跟counter是一样的,但如果出现了这个错误,是不是可以做一下补救(也不一定合适)

image

fengjiachun commented 4 years ago

没有,真的很奇怪,我不知道怎么就到这个环节里面去了,我的用法级别上跟counter是一样的,但如果出现了这个错误,是不是可以做一下补救(也不一定合适)

image

到这个环节里没有问题啊,不是这里的原因,或者你把上面那段日志前后完整的发给我,就那个 ip为 ..0.33 的日志

fengjiachun commented 4 years ago

你有开 debug 日志吗?开了的话搜一下 “set snapshot:” 关键字

hanzhihua commented 4 years ago

现在没有开,我准备开一下试试。 另外我上面给的日志,就是0.33那台机器在那段出错时间所有跟raft相关的日志了

fengjiachun commented 4 years ago

现在没有开,我准备开一下试试。 另外我上面给的日志,就是0.33那台机器在那段出错时间所有跟raft相关的日志了

我要看过程,上面只截取了部分没什么用

fengjiachun commented 4 years ago

CNStateSnapshotFile 方便的话这个类能发出来?

fengjiachun commented 4 years ago

还想问的一点是,在你的业务里,具体出现了什么问题?数据丢失了还是集群不可恢复或是怎样?

hanzhihua commented 4 years ago

文件在这里,谢谢支持:) jraft相关.log

hanzhihua commented 4 years ago

就是在发布期间偶尔和短暂出现了那个错误,启动完成后就自愈了,业务上面感觉没有什么问题,没有听到使用方在叫

fengjiachun commented 4 years ago

就是在发布期间偶尔和短暂出现了那个错误,启动完成后就自愈了,业务上面感觉没有什么问题,没有听到使用方在叫

恩,就你发的那一小段日志来说,本质上是没什么影响的,我有一点点怀疑 snapshot 出了问题,我先看看你新发的日志

fengjiachun commented 4 years ago

貌似没发日志?

fengjiachun commented 4 years ago

snapshot save 不能异步,我记得和你说过?加入你的存储模块支持快照读,最起码的,你需要在锁中读取一个你的 store 的快照才行,本质上要求在你读取 store 数据并进行 snapshot save 的时候不能有新的写入

hanzhihua commented 4 years ago

原始日志太大了,snaphost过程没有错误, 执行过程中没有jraft相关的日志 哦,这样,我根据counter例子来做的

image

fengjiachun commented 4 years ago

原始日志太大了,snaphost过程没有错误, 执行过程中没有jraft相关的日志 哦,这样,我根据counter例子来做的

image

counter 的例子你没看懂,你没注意到第一行就是同步读取 store 的快照 final long currVal = this.value.get(); 你的代码是没有这个逻辑的

hanzhihua commented 4 years ago

如果snapshot save异步,我理解的会出现snapshot读取的applyIndex和实际的applyIndex可能不一致,我会把它改成同步的,但目前出现这个错误,好像跟这个关系不大,很奇怪