sofastack / sofa-jraft

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

Handle NodeImpl disruptor event error #338

Closed hanzhihua closed 4 years ago

hanzhihua commented 4 years ago

Your question

出现这样的错误,对状态机有影响吗?需要做什么什么调整吗?

2019-11-12 18:19:02.372 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@48dc3a1c java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@15791a23 rejected from com.alipay.sofa.jraft.util.MetricThreadPoolExecutor@6ce48c52[Running, pool size = 200, active threads = 200, queued tasks = 0, completed tasks = 156014188] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063) at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830) at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379) at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112) at com.alipay.sofa.jraft.util.Utils.runInThread(Utils.java:153) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.wakeupAllWaiter(LogManagerImpl.java:410) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.appendEntries(LogManagerImpl.java:353) at com.alipay.sofa.jraft.core.NodeImpl.executeApplyingTasks(NodeImpl.java:1119) at com.alipay.sofa.jraft.core.NodeImpl.access$200(NodeImpl.java:129) at com.alipay.sofa.jraft.core.NodeImpl$LogEntryAndClosureHandler.onEvent(NodeImpl.java:259) at com.alipay.sofa.jraft.core.NodeImpl$LogEntryAndClosureHandler.onEvent(NodeImpl.java:240) at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:137) at java.lang.Thread.run(Thread.java:748)

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

hanzhihua commented 4 years ago

我看一下代码,下面这样写会不会有问题,请赐教 image

killme2008 commented 4 years ago

@hanzhihua 你贴的那个错误不影响,只是通知没有到位,还是会继续复制。

closure 的线程池是可以调整的,默认是 5*cpus 或者 100(取最大值),可以通过 -Djraft.closure.threadpool.size.max=500 启动参数来配置修改。

不过你可能要看下是不是你在 task 的 closure 回调里做了很多耗时的事情,正常来说默认值是足够的

hanzhihua commented 4 years ago

好的,我electiontimeout设置了10s,是不是有点长了 我在发布新版本时候,出现了 com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. (老leader节点,在stepdown后面出现的)

另外在一个follower重启出现下面的这个诡异的错误 [JRaft-LogManager-Disruptor-0]-c.a.s.jraft.util.LogExceptionHandler.handleEventException - Handle LogManagerImpl disruptor event error, event is com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEvent@49445bea
java.lang.NullPointerException: null at org.rocksdb.AbstractWriteBatch.put(AbstractWriteBatch.java:28) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.addDataBatch(RocksDBLogStorage.java:408) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.lambda$appendEntries$1(RocksDBLogStorage.java:443) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.executeBatch(RocksDBLogStorage.java:274) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.appendEntries(RocksDBLogStorage.java:437) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.appendToStorage(LogManagerImpl.java:428) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.access$300(LogManagerImpl.java:76) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$AppendBatcher.flush(LogManagerImpl.java:464) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$AppendBatcher.append(LogManagerImpl.java:483) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent(LogManagerImpl.java:512) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent(LogManagerImpl.java:494) at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:137) at java.lang.Thread.run(Thread.java:748)

fengjiachun commented 4 years ago

只有在 shutdown 后继续调用才会出现这个错误,如果你在重启,不会有什么影响,不过稍后我还是要分析下关闭顺序是否会导致这个问题

fengjiachun commented 4 years ago

et 根据自身需求和场景来定,不过通常确实不建议设置10秒这么长的时间,默认是1秒

hanzhihua commented 4 years ago

出现下面的异常也是正常的吧 [JRaft-LogManager-Disruptor-0]-c.a.s.jraft.util.LogExceptionHandler.handleEventException - Handle LogManagerImpl disruptor event error, event is com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEvent@49445bea java.lang.NullPointerException: null at org.rocksdb.AbstractWriteBatch.put(AbstractWriteBatch.java:28) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.addDataBatch(RocksDBLogStorage.java:408) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.lambda$appendEntries$1(RocksDBLogStorage.java:443) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.executeBatch(RocksDBLogStorage.java:274) at com.alipay.sofa.jraft.storage.impl.RocksDBLogStorage.appendEntries(RocksDBLogStorage.java:437) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.appendToStorage(LogManagerImpl.java:428) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl.access$300(LogManagerImpl.java:76) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$AppendBatcher.flush(LogManagerImpl.java:464) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$AppendBatcher.append(LogManagerImpl.java:483) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent(LogManagerImpl.java:512) at com.alipay.sofa.jraft.storage.impl.LogManagerImpl$StableClosureEventHandler.onEvent(LogManagerImpl.java:494) at com.lmax.disruptor.BatchEventProcessor.run(BatchEventProcessor.java:137) at java.lang.Thread.run(Thread.java:748)

fengjiachun commented 4 years ago

是的,我回复你的就是指这个异常没问题,这发生在logstorage被关闭但是disruptor队列中仍有未提交到storage的log,这个异常不影响正确性,但稍后我需要确认下一系列shutdown顺序,这个问题是需要改进的

hanzhihua commented 4 years ago

哦,明白了,那像这个异常呢 com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 也发生在重启的时候

fengjiachun commented 4 years ago

哦,明白了,那像这个异常呢 com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 也发生在重启的时候

手机看漏看了这个,这个是什么版本出现的?

killme2008 commented 4 years ago
com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0.

这个请给下完整的堆栈,并说明下当时的场景

hanzhihua commented 4 years ago

1.2.6版本,我看代码的异常是,先是leader stepdown,然后在出现线程不够,然后就出现上面的错误

异常栈: 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-198]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4303ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-95]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4292ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-199]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4292ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.155 ERROR leader [JRaft-StepDownTimer]-c.a.sofa.jraft.util.RepeatedTimer.run - run timer failed java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@398629f9 rejected from com.alipay.sofa.jraft.util.MetricThreadPoolExecutor@41bac47c[Running, pool size = 200, active threads = 198, queued tasks = 0, completed tasks = 45913468] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063) at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830) at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379) at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112) at com.alipay.sofa.jraft.util.Utils.runInThread(Utils.java:153) at com.alipay.sofa.jraft.util.Utils.runClosureInThread(Utils.java:163) at com.alipay.sofa.jraft.closure.ClosureQueueImpl.clear(ClosureQueueImpl.java:81) at com.alipay.sofa.jraft.core.BallotBox.clearPendingTasks(BallotBox.java:152) at com.alipay.sofa.jraft.core.NodeImpl.stepDown(NodeImpl.java:1006) at com.alipay.sofa.jraft.core.NodeImpl.checkDeadNodes(NodeImpl.java:1845) at com.alipay.sofa.jraft.core.NodeImpl.handleStepDownTimeout(NodeImpl.java:1904) at com.alipay.sofa.jraft.core.NodeImpl.access$1400(NodeImpl.java:129) at com.alipay.sofa.jraft.core.NodeImpl$3.onTrigger(NodeImpl.java:728) at com.alipay.sofa.jraft.util.RepeatedTimer.run(RepeatedTimer.java:83) at com.alipay.sofa.jraft.util.RepeatedTimer$1.run(RepeatedTimer.java:160) at java.util.TimerThread.mainLoop(Timer.java:555) at java.util.TimerThread.run(Timer.java:505) 2019-11-13 09:10:25.312 WARN leader [waitCheckScheduler 11-0]-c.b.a.controlnode.ha.CNStateProxy.doTask - raft operation timeout, raftRequest:CNStatRequest{action='DEPENDENCY_REMOVE_TO_LAST', key=8397140, value=null}, cost:1000ms 2019-11-13 09:10:25.354 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 2019-11-13 09:10:25.400 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 2019-11-13 09:10:25.418 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0.

fengjiachun commented 4 years ago

我看一下代码,下面这样写会不会有问题,请赐教 image

这个感觉是可以考虑优化一下,粒度太小了,很占线程

fengjiachun commented 4 years ago

1.2.6版本,我看代码的异常是,先是leader stepdown,然后在出现线程不够,然后就出现上面的错误

异常栈: 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-198]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4303ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-95]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4292ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.154 WARN leader [JRaft-Closure-Executor-199]-c.b.a.c.ha.ReturnValueClosure.run - raft operation fail,it cost:4292ms, error:Status[EPERM<1008>: Leader stepped down] 2019-11-13 09:10:25.155 ERROR leader [JRaft-StepDownTimer]-c.a.sofa.jraft.util.RepeatedTimer.run - run timer failed java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.FutureTask@398629f9 rejected from com.alipay.sofa.jraft.util.MetricThreadPoolExecutor@41bac47c[Running, pool size = 200, active threads = 198, queued tasks = 0, completed tasks = 45913468] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2063) at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:830) at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1379) at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112) at com.alipay.sofa.jraft.util.Utils.runInThread(Utils.java:153) at com.alipay.sofa.jraft.util.Utils.runClosureInThread(Utils.java:163) at com.alipay.sofa.jraft.closure.ClosureQueueImpl.clear(ClosureQueueImpl.java:81) at com.alipay.sofa.jraft.core.BallotBox.clearPendingTasks(BallotBox.java:152) at com.alipay.sofa.jraft.core.NodeImpl.stepDown(NodeImpl.java:1006) at com.alipay.sofa.jraft.core.NodeImpl.checkDeadNodes(NodeImpl.java:1845) at com.alipay.sofa.jraft.core.NodeImpl.handleStepDownTimeout(NodeImpl.java:1904) at com.alipay.sofa.jraft.core.NodeImpl.access$1400(NodeImpl.java:129) at com.alipay.sofa.jraft.core.NodeImpl$3.onTrigger(NodeImpl.java:728) at com.alipay.sofa.jraft.util.RepeatedTimer.run(RepeatedTimer.java:83) at com.alipay.sofa.jraft.util.RepeatedTimer$1.run(RepeatedTimer.java:160) at java.util.TimerThread.mainLoop(Timer.java:555) at java.util.TimerThread.run(Timer.java:505) 2019-11-13 09:10:25.312 WARN leader [waitCheckScheduler 11-0]-c.b.a.controlnode.ha.CNStateProxy.doTask - raft operation timeout, raftRequest:CNStatRequest{action='DEPENDENCY_REMOVE_TO_LAST', key=8397140, value=null}, cost:1000ms 2019-11-13 09:10:25.354 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 2019-11-13 09:10:25.400 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0. 2019-11-13 09:10:25.418 ERROR leader [JRaft-NodeImpl-Disruptor-0]-com.alipay.sofa.jraft.core.BallotBox.appendPendingTask - Fail to appendingTask, pendingIndex=0.

这个理论上没问题,stepdown 后作为非 leader 不能 appendPendingTask

killme2008 commented 4 years ago

嗯,这一块可以做下优化,那个错误也不影响。 @hanzhihua 有影响到你的业务吗?

hanzhihua commented 4 years ago

@killme2008 现在还不确定,就是发布时候出现了线程池(Utils.runInThread)满了Reject的错误,然后后面出现了一系列的异常,但发布完成后就自愈了 我担心会不会由于线程池满了,造成了主要的操作没有正确的处理

fengjiachun commented 4 years ago

@killme2008 现在还不确定,就是发布时候出现了线程池(Utils.runInThread)满了Reject的错误,然后后面出现了一系列的异常,但发布完成后就自愈了 我担心会不会由于线程池满了,造成了主要的操作没有正确的处理

数据正确性上不会有问题,不会因为一个异常或是错误导致产生错误的数据

hanzhihua commented 4 years ago

线上运行过程中发现了task中的data和clourse不匹配的情况,主要发生在重启的过程中

下面是伪代码: final Task task = new Task(); ReturnValueClosure closure = new ReturnValueClosure(countDownLatch, start); task.setDone(closure); task.setData(ByteBuffer .wrap(SerializerManager.getSerializer(SerializerManager.Hessian2).serialize(raftRequest))); raftServer.getNode().apply(task); 。。。 return closure.getReturnValue();

我从closure.getReturnValue拿到的value,出现了java.lang.ClassCastException

不知道,在raft集群会不会出现task中的data和closure不匹配的情况呢?谢谢