【jraft】跟蹤transferLeaderShipTo發(fā)現(xiàn)的bug

0. 背景

HugeGraph項目用到了jraft作為控制副本一致性的底層組件,昨天review代碼的時候,發(fā)現(xiàn)transferLeaderShipTo方法和addPeer、removePeer方法的簽名不太一致,后兩個方法需要傳入一個Closure對象,然后在Closure對象的run方法中根據(jù)status的值判斷操作是否成功。但是transferLeaderShip方法不用傳Closure對象,而是直接返回了一個status。于是帶著好奇心想看看既然都是操作raft-group,為什么單這個方法是同步的,它是怎么實現(xiàn)的呢?

1. 跟蹤transferLeaderShipTo方法

// 省略一堆檢查和提前返回的情況
final long lastLogIndex = this.logManager.getLastLogIndex();
if (!this.replicatorGroup.transferLeadershipTo(peerId, lastLogIndex)) {
    LOG.warn("No such peer {}.", peer);
    return new Status(RaftError.EINVAL, "No such peer %s", peer);
}
this.state = State.STATE_TRANSFERRING;
final Status status = new Status(RaftError.ETRANSFERLEADERSHIP,
    "Raft leader is transferring leadership to %s", peerId);
onLeaderStop(status);
LOG.info("Node {} starts to transfer leadership to peer {}.", getNodeId(), peer);
final StopTransferArg stopArg = new StopTransferArg(this, this.currTerm, peerId);
this.stopTransferArg = stopArg;
this.transferTimer = this.timerManager.schedule(() -> onTransferTimeout(stopArg),
    this.options.getElectionTimeoutMs(), TimeUnit.MILLISECONDS);

這里的幾個大步驟包括:

  1. 給準leader的replicator執(zhí)行transferLeadershipTo方法,傳入lastLogIndex,這里先不管到底干了啥,但是掃描了一下下面的代碼行,基本都跟準leader的PeerId沒關(guān)系了,說明這個transferLeadershipTo方法就能觸發(fā)準leader的選舉;
  2. 把自己的狀態(tài)設(shè)置為State.STATE_TRANSFERRING,表示正在轉(zhuǎn)移;
  3. 執(zhí)行onLeaderStop,這個方法會構(gòu)造一個類型為LEADER_STOP的task,然后塞入任務(wù)隊列,自己定義的狀態(tài)機也會響應onLeaderStop事件;
  4. 啟動一個定時任務(wù),該任務(wù)在一個選舉超時時間后會執(zhí)行onTransferTimeout(stopArg)方法,這個方法我們先不管。

從這里可以大致看出主邏輯是:先讓準leader觸發(fā)選舉邏輯,如果一個選舉周期內(nèi)都沒能選出leader,會執(zhí)行onTransferTimeout方法。

既然replicator.transferLeadershipTo(peerId, lastLogIndex)這么關(guān)鍵,我們進去看看:

public boolean transferLeadershipTo(final PeerId peer, final long logIndex) {
    final ThreadId rid = this.replicatorMap.get(peer);
    // 調(diào)了下面的方法
    return rid != null && Replicator.transferLeadership(rid, logIndex);
}

public static boolean transferLeadership(final ThreadId id, final long logIndex) {
    final Replicator r = (Replicator) id.lock();
    if (r == null) {
        return false;
    }
    // dummy is unlock in _transfer_leadership
    // 調(diào)了下面的方法
    return r.transferLeadership(logIndex);
}

private boolean transferLeadership(final long logIndex) {
    if (this.hasSucceeded && this.nextIndex > logIndex) {
        // _id is unlock in _send_timeout_now
        sendTimeoutNow(true, false);
        return true;
    }
    // Register log_index so that _on_rpc_return trigger
    // _send_timeout_now if _next_index reaches log_index
    this.timeoutNowIndex = logIndex;
    this.id.unlock();
    return true;
}

關(guān)鍵方法是sendTimeoutNow,再一路跟,沒什么特別之處,直接把代碼貼出來了。

private void sendTimeoutNow(final boolean unlockId, final boolean stopAfterFinish) {
    sendTimeoutNow(unlockId, stopAfterFinish, -1);
}

private void sendTimeoutNow(final boolean unlockId, final boolean stopAfterFinish, final int timeoutMs) {
    final TimeoutNowRequest.Builder rb = TimeoutNowRequest.newBuilder();
    rb.setTerm(this.options.getTerm());
    rb.setGroupId(this.options.getGroupId());
    rb.setServerId(this.options.getServerId().toString());
    rb.setPeerId(this.options.getPeerId().toString());
    try {
        if (!stopAfterFinish) {
            // This RPC is issued by transfer_leadership, save this call_id so that
            // the RPC can be cancelled by stop.
            // 這個方法返回的Future被接住后,并沒有什么特殊的等待操作
            this.timeoutNowInFly = timeoutNow(rb, false, timeoutMs);
            this.timeoutNowIndex = 0;
        } else {
            timeoutNow(rb, true, timeoutMs);
        }
    } finally {
        if (unlockId) {
            this.id.unlock();
        }
    }

}

private Future<Message> timeoutNow(final TimeoutNowRequest.Builder rb, final boolean stopAfterFinish,
                                   final int timeoutMs) {
    final TimeoutNowRequest request = rb.build();
    return this.rpcService.timeoutNow(this.options.getPeerId().getEndpoint(), request, timeoutMs,
        new RpcResponseClosureAdapter<TimeoutNowResponse>() {

            @Override
            public void run(final Status status) {
                if (Replicator.this.id != null) {
                    onTimeoutNowReturned(Replicator.this.id, status, request, getResponse(), stopAfterFinish);
                }
            }

        });
}

這兩段的邏輯是,先找到準leader對應的replicator對象,然后通過它給準leader發(fā)一個TimeoutNow的請求,這個Timeout應該是選舉超時的意思,所以TimeoutNow就是指“立馬處理選舉超時”。但是這里讓人費解的是:this.timeoutNowInFly = timeoutNow(rb, false, timeoutMs);只是把返回的Future接住了,并沒有做什么等待操作,所以其實transferLeaderShipTo方法本質(zhì)上還是一個異步方法。

經(jīng)驗:一到這種發(fā)消息的地方,代碼肯定是不能直接一步一步跟了的。還好處理消息/請求的方法命名都很類似,一般都是handleXXXRequest這種格式的。

全局搜索handleTimeoutNowRequest方法,果然在NodeImpl中能找到它的實現(xiàn):

// 省略一堆檢查和提前返回的情況
final long savedTerm = this.currTerm;
final TimeoutNowResponse resp = TimeoutNowResponse.newBuilder() //
    .setTerm(this.currTerm + 1) //
    .setSuccess(true) //
    .build();

// Parallelize response and election
done.sendResponse(resp);
doUnlock = false;
electSelf();
LOG.info("Node {} received TimeoutNowRequest from {}, term={}.", getNodeId(), request.getServerId(),
    savedTerm);

這里先構(gòu)造了響應,設(shè)置success為true,term為原term+1,然后再執(zhí)行electSelf實現(xiàn)自選舉。難道jraft的作者這么自信,electSelf方法一定不會出錯嗎?極端情況下:發(fā)送完響應后,準leader的機器突然宕機,會導致選舉失敗,那提前返回的響應是代表什么,代表收到了這個請求嗎?我覺得這里怪怪的。

既然懷疑transferLeaderShipTo方法是一個異步(非阻塞)方法,怎么驗證呢?很簡單,在異步處理的某個步驟加上一個睡眠,如果主方法還能瞬間返回,那就是異步無疑了。于是我隨手添加了如下代碼:

final TimeoutNowResponse resp = TimeoutNowResponse.newBuilder() //
    .setTerm(this.currTerm + 1) //
    .setSuccess(true) //
    .build();

try {
    LOG.info("==> 睡眠一會,用于調(diào)試");
    Thread.sleep(3000);
} catch (InterruptedException e) {
    e.printStackTrace();
}

// Parallelize response and election
done.sendResponse(resp);
doUnlock = false;
electSelf();
LOG.info("Node {} received TimeoutNowRequest from {}, term={}.", getNodeId(), request.getServerId(),
    savedTerm);

再運行調(diào)試,果然主方法還能瞬間返回,確認是異步了,既然如此,我覺得還是應該像addPeer、removePeer一樣傳一個Closure進來,讓用戶能處理Closure。現(xiàn)在主方法返回的status是不太嚴謹?shù)摹?/p>

本來以為問題得到了驗證,就此打住的,但是突然發(fā)現(xiàn)原leader和準leader都打出了很多日志,一直在報錯,似乎是我們程序的一個BUG。

2. 調(diào)試+定位BUG

下面是最初報錯時的日志:

原leader節(jié)點

2020-10-26 16:23:50 105014 [grizzly-http-server-1] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> starts to transfer leadership to peer 127.0.0.1:8282.
2020-10-26 16:23:50 105017 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreStateMachine [] - The node 127.0.0.1:8281 abdicated from leader
2020-10-26 16:23:50 105017 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.alipay.sofa.jraft.core.StateMachineAdapter [] - onLeaderStop: status=Status[ETRANSFERLEADERSHIP<10013>: Raft leader is transferring leadership to 127.0.0.1:8282].
2020-10-26 16:23:52 106747 [Bolt-conn-event-executor-9-thread-1] [INFO ] com.alipay.sofa.jraft.rpc.impl.core.ClientServiceConnectionEventProcessor [] - Peer 127.0.0.1:8281 is connected
2020-10-26 16:24:00 115020 [JRaft-Node-ScheduleThreadPool5] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> failed to transfer leadership to peer 127.0.0.1:8282, reached timeout.
2020-10-26 16:24:00 115021 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreStateMachine [] - The node 127.0.0.1:8281 become to leader
2020-10-26 16:24:00 115021 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.alipay.sofa.jraft.core.StateMachineAdapter [] - onLeaderStart: term=1.
2020-10-26 16:24:01 116087 [JRaft-StepDownTimer-<default-group/127.0.0.1:8281>0] [WARN ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> steps down when alive nodes don't satisfy quorum, term=1, deadNodes=127.0.0.1:8282, conf=127.0.0.1:8281,127.0.0.1:8282.
2020-10-26 16:24:01 116088 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreStateMachine [] - The node 127.0.0.1:8281 abdicated from leader
2020-10-26 16:24:01 116088 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.alipay.sofa.jraft.core.StateMachineAdapter [] - onLeaderStop: status=Status[ERAFTTIMEDOUT<10001>: Majority of the group dies: 1/2].
2020-10-26 16:24:01 116088 [JRaft-StepDownTimer-<default-group/127.0.0.1:8281>0] [INFO ] com.alipay.sofa.jraft.core.Replicator [] - Replicator Replicator [state=Replicate, statInfo=<running=IDLE, firstLogIndex=46, lastLogIncluded=0, lastLogIndex=46, lastTermIncluded=0>, peerId=127.0.0.1:8282, type=Follower] is going to quit
2020-10-26 16:24:01 116089 [JRaft-Rpc-Closure-Executor-3] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Replicator 127.0.0.1:8282 prepare to offline
2020-10-26 16:24:12 126476 [JRaft-ElectionTimer-<default-group/127.0.0.1:8281>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> term 1 start preVote.
2020-10-26 16:24:22 137260 [JRaft-ElectionTimer-<default-group/127.0.0.1:8281>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> term 1 start preVote.
2020-10-26 16:24:32 147278 [JRaft-ElectionTimer-<default-group/127.0.0.1:8281>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> term 1 start preVote.
2020-10-26 16:24:43 158161 [JRaft-ElectionTimer-<default-group/127.0.0.1:8281>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8281> term 1 start preVote.
2020-10-26 16:24:49 163820 [Bolt-conn-event-executor-4-thread-1] [INFO ] com.alipay.sofa.jraft.rpc.RpcRequestProcessor [] - Connection disconnected: 127.0.0.1:57648

原follower節(jié)點

2020-10-26 16:23:50 92315 [Bolt-default-executor-5-thread-8] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - ==> 睡眠一會,用于調(diào)試
2020-10-26 16:23:53 95318 [Bolt-default-executor-5-thread-8] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> start vote and grant vote self, term=1.
2020-10-26 16:23:53 95320 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreStateMachine [] - The node 127.0.0.1:8282 abdicated from follower
2020-10-26 16:23:53 95320 [JRaft-FSMCaller-Disruptor-0] [INFO ] com.alipay.sofa.jraft.core.StateMachineAdapter [] - onStopFollowing: LeaderChangeContext [leaderId=127.0.0.1:8281, term=1, status=Status[ERAFTTIMEDOUT<10001>: A follower's leader_id is reset to NULL as it begins to request_vote.]].
2020-10-26 16:23:53 95321 [default-group/127.0.0.1:8282-AppendEntriesThread0] [WARN ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> ignore stale AppendEntriesRequest from 127.0.0.1:8281, term=1, currTerm=2.
2020-10-26 16:23:53 95328 [Bolt-default-executor-5-thread-8] [INFO ] com.alipay.sofa.jraft.storage.impl.LocalRaftMetaStorage [] - Save raft meta, path=/Users/liningrui/IdeaProjects/baidu/xbu-data/hugegraph/node2/raft-log/meta, term=2, votedFor=127.0.0.1:8282, cost time=1 ms
2020-10-26 16:23:53 95329 [Bolt-default-executor-5-thread-8] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> received TimeoutNowRequest from 127.0.0.1:8281, term=1.
2020-10-26 16:23:56 98323 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 3.002s
2020-10-26 16:23:59 101324 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 6.003s
2020-10-26 16:24:02 104330 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 9.009s
2020-10-26 16:24:03 105339 [JRaft-RPC-Processor-7] [WARN ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> RequestVote to 127.0.0.1:8281 error: Status[EINTERNAL<1004>: RPC exception:Invoke timeout when invoke with callback.The address is 127.0.0.1:8281].
2020-10-26 16:24:04 105802 [JRaft-VoteTimer-<default-group/127.0.0.1:8282>0] [WARN ] com.alipay.sofa.jraft.core.NodeImpl [] - Candidate node <default-group/127.0.0.1:8282> term 2 steps down when election reaching vote timeout: fail to get quorum vote-granted.
2020-10-26 16:24:04 105803 [JRaft-VoteTimer-<default-group/127.0.0.1:8282>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> term 2 start preVote.
2020-10-26 16:24:05 107332 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 12.011s
2020-10-26 16:24:08 110337 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 15.015s
2020-10-26 16:24:11 113342 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 18.021s
2020-10-26 16:24:12 113778 [Bolt-default-executor-5-thread-12] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> received PreVoteRequest from 127.0.0.1:8281, term=2, currTerm=2, granted=true, requestLastLogId=LogId [index=46, term=1], lastLogId=LogId [index=46, term=1].
2020-10-26 16:24:14 115816 [JRaft-RPC-Processor-8] [WARN ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> PreVote to 127.0.0.1:8281 error: Status[EINTERNAL<1004>: RPC exception:Invoke timeout when invoke with callback.The address is 127.0.0.1:8281].
2020-10-26 16:24:14 116226 [JRaft-ElectionTimer-<default-group/127.0.0.1:8282>0] [INFO ] com.alipay.sofa.jraft.core.NodeImpl [] - Node <default-group/127.0.0.1:8282> term 2 start preVote.
2020-10-26 16:24:14 116345 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 21.024s
2020-10-26 16:24:17 119349 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 24.028s
2020-10-26 16:24:20 122350 [server-info-db-worker-1] [WARN ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - Waiting for raft group 'default-group' election cost 27.029s

從日志可以看出兩個節(jié)點的表現(xiàn):

  • 原leader節(jié)點先從leader退位,然后10秒后(設(shè)置的選舉超時時間),觸發(fā)了onTransferTimeout方法(failed to transfer leadership),再變回leader,但是立馬又從leader退位,原因是存活節(jié)點不過半(steps down when alive nodes don't satisfy quorum);
  • 準leader節(jié)點睡眠了3秒后,開始自選舉,但是在超時時間內(nèi)收不到投票請求,然后開啟新一輪的預投票,各種失敗警告,另外一直有等待選舉耗時多少秒的警告(這個是HugeGraph種添加的)

先不管這些錯誤信息,我們想想正確的流程應該是怎樣的。準leader睡眠3秒后,執(zhí)行自選舉,這時應該能收到原leader(已退位)的投票,然后順利成為新的leader。那為什么它沒有成為新的leader呢?一定是投票環(huán)節(jié)出了問題。

我在electSelf方法中添加日志打印

// 省略
LOG.info("HG ==> 向 {} 發(fā)起投票請求", peer.getEndpoint());
this.rpcService.requestVote(peer.getEndpoint(), done.request, done);
// 省略

這里能打印出來,說明確實發(fā)送了請求投票的請求。

繼續(xù)在接收請求處也添加日志打印,具體是在RequestVoteRequestProcessor.processRequest0方法中

if (request.getPreVote()) {
    LOG.info("HG ==> 準備處理 PreVoteRequest");
    return service.handlePreVoteRequest(request);
} else {
    LOG.info("HG ==> 準備處理 RequestVoteRequest");
    return service.handleRequestVoteRequest(request);
}

這里就不正常了,if和else分支都不打印日志,說明原leader根本就沒有收到準leader的投票請求。

那為什么收不到呢?進程都是在本地,不可能是網(wǎng)絡(luò)的問題,況且先前都已經(jīng)選出過leader了,難道是所有的rpc線程都在使用中,導致請求無法被rpc線程處理?但是又為啥rpc線程被用完呢?雖然說是在本地,好歹筆記本配置還可以,也沒開什么大的任務(wù),就這就讓線程這么長時間都無法響應了?

突然我瞟到了代碼中的forwardToLeader方法,這個方法的作用是把follower節(jié)點上的寫請求轉(zhuǎn)發(fā)到leader節(jié)點上,這里是會顯式地占用rpc線程的。這里很可以,添加了日志打印,另外在接收請求的地方(StoreCommandRequestProcessor.processRequest方法)也添加日志。

一運行,居然發(fā)現(xiàn)了消息的遞歸調(diào)用

2020-10-26 22:01:57 42474 [Bolt-default-executor-5-thread-18] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreCommandRequestProcessor [] - 線程 Bolt-default-executor-5-thread-18 提交并等待命令執(zhí)行
2020-10-26 22:01:57 42475 [Bolt-default-executor-5-thread-18] [INFO ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - The node <default-group/127.0.0.1:8281> forward request to leader 127.0.0.1:8281
2020-10-26 22:01:57 42477 [Bolt-default-executor-5-thread-20] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreCommandRequestProcessor [] - 線程 Bolt-default-executor-5-thread-20 提交并等待命令執(zhí)行
2020-10-26 22:01:57 42478 [Bolt-default-executor-5-thread-20] [INFO ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - The node <default-group/127.0.0.1:8281> forward request to leader 127.0.0.1:8281
2020-10-26 22:01:57 42480 [Bolt-default-executor-5-thread-1] [INFO ] com.baidu.hugegraph.backend.store.raft.StoreCommandRequestProcessor [] - 線程 Bolt-default-executor-5-thread-1 提交并等待命令執(zhí)行
2020-10-26 22:01:57 42480 [Bolt-default-executor-5-thread-1] [INFO ] com.baidu.hugegraph.backend.store.raft.RaftNode [] - The node <default-group/127.0.0.1:8281> forward request to leader 127.0.0.1:8281
...

這就奇怪了,走到forwardToLeader方法說明當前節(jié)點不是leader,需要轉(zhuǎn)發(fā),但是能收到請求又說明自己是leader。這種矛盾的狀態(tài)應該跟判斷l(xiāng)eader角色的代碼有關(guān)。經(jīng)過定位,找到了這個代碼片段:

this.waitLeaderElected(RaftSharedContext.NO_TIMEOUT);
if (!this.isRaftLeader()) {
    this.forwardToLeader(command, closure);
    return;
}

其中waitLeaderElected()會用Node.getLeaderId()判斷l(xiāng)eader存不存在,isRaftLeader()會用Node.isLeader()判斷自己是不是leader,現(xiàn)在是情況是:

  • Node.getLeaderId()判斷l(xiāng)eader存在,而且leaderId指向的就是自己;
  • Node.isLeader()判斷自己不是leader。

所以上面那個消息遞歸的原因就是:

準leader給原leader轉(zhuǎn)發(fā)了一個寫請求,原leader的rpc線程接收到了這個請求,然后調(diào)用submitAndWait執(zhí)行該請求,然后waitLeaderElected方法能走過去,但this.isRaftLeader()返回了false,于是進入到forwardToLeader方法,該方法拿到當前的leaderId,其實就是自己,然后又給自己發(fā)送請求,于是這個過程不斷重復,直到把rpc線程用完。

所以當準leader向原leader發(fā)送了投票請求時,原leader已經(jīng)沒有rpc線程能接受請求了,所以導致投票超時。于是出現(xiàn):原leader因為term低不能被選為新leader,而準leader收不到原leader的投票也不能成為新leader,導致raft group再也選不出來leader的情形。

而jraft的在onLeaderStop后出現(xiàn)兩個方法判斷l(xiāng)eader狀態(tài)不一致的問題,已經(jīng)向社區(qū)提問了:https://github.com/sofastack/sofa-jraft/issues/531

最后編輯于
?著作權(quán)歸作者所有,轉(zhuǎn)載或內(nèi)容合作請聯(lián)系作者
【社區(qū)內(nèi)容提示】社區(qū)部分內(nèi)容疑似由AI輔助生成,瀏覽時請結(jié)合常識與多方信息審慎甄別。
平臺聲明:文章內(nèi)容(如有圖片或視頻亦包括在內(nèi))由作者上傳并發(fā)布,文章內(nèi)容僅代表作者本人觀點,簡書系信息發(fā)布平臺,僅提供信息存儲服務(wù)。

友情鏈接更多精彩內(nèi)容