hashicorp / raft

Golang implementation of the Raft consensus protocol
Mozilla Public License 2.0
8.29k stars 997 forks source link

Leader stopped sending appendEntries to the follower, but heartbeat was working #612

Open pranayhere opened 3 months ago

pranayhere commented 3 months ago

Hi, Thanks for the great library! I really appreciate the efforts you guys have taken to build this project.

Issue Description: We recently encountered an issue, leader stopped sending appendEntries to the follower but could send the heartbeat. The screenshot below, taken from the metrics, shows the situation. Only one follower node was affected, and all other nodes were fine.

AppendEntried_heartbeat

Let me describe the issue in more detail, and share my findings so far on the issue.

At 09:44:36.106, Follower tries to storeLogs from Index: 151153297 to 151153341.
At 09:44:36.115, Follower received the error, "failed to append to logs error=unable to store logs in logStore, 
                          err=Error 1213 (40001): Deadlock found when trying to get lock; try restarting transaction"
                          // note that our logStore is backed by MySQL. 
At 09:44:36.126, On Follower, raft: failed to get previous log previous-index=151153341 last-index=151153296 error=log not found
At 09:44:36.136, On Follower, raft: failed to get previous log previous-index=151153342 last-index=151153296 error=log not found

The above timeline represents the events that occur at the time of the issue. At the same time, I didn't get any warn/error from the leader.

Following is the Stats on the Follower node at 9.45:29.236

{
    "applied_index":"151153288",
    "commit_index":"151153288",
    "fsm_pending":"0",
    "last_contact":"40.69452ms",
    "last_log_index":"151153296",
    "last_log_term":"34105",
    "last_snapshot_index":"151134079",
    "last_snapshot_term":"34105",
    "latest_configuration":"<Sorry, I can't share this. But I've double checked the follower node is available in the list>",
    "latest_configuration_index":"0",
    "num_peers":"8",
    "protocol_version":"3",
    "protocol_version_max":"3",
    "protocol_version_min":"0",
    "server_id":"FollowerNode",
    "snapshot_version_max":"1",
    "snapshot_version_min":"0",
    "state":"Follower",
    "term":"34105"
}

I'm referring to the code below,

    // Update s based on success
    if resp.Success {
        // Update our replication state
        updateLastAppended(s, &req)

        // Clear any failures, allow pipelining
        s.failures = 0
        s.allowPipeline = true
    } else {
        atomic.StoreUint64(&s.nextIndex, max(min(s.nextIndex-1, resp.LastLog+1), 1))
        if resp.NoRetryBackoff {
            s.failures = 0
        } else {
            s.failures++
        }
        r.logger.Warn("appendEntries rejected, sending older logs", "peer", peer, "next", atomic.LoadUint64(&s.nextIndex))
    }

I've the following questions.

  1. When "failed to append to logs" error occurred on the follower node, on the leader atomic.StoreUint64(&s.nextIndex, max(min(s.nextIndex-1, resp.LastLog+1), 1)) should set s.nextIndex to 151153297, and try to replicate again from Index=151153297. But in this case "appendEntries rejected, sending older logs" error didn't occur on the leader node. Does that mean the leader received resp.Success=true when an error occurred?

  2. Does the leader consider it has replicated logs till Index 151153341 at the follower node? As from the logs on the follower, it is looking for previousLog at index=151153341

  3. I don't understand the reason why replication stopped from Leader to Follower, i don't see "removed peer, stopping replication" in the logs on the leader node.

Note: when the leadershipTransfer happened again, a new leader was able to restart the replication.

banks commented 3 months ago

Hi thanks for the detailed description. We've not seen this occur before with any of our log stores. I say that just as information that it's not a common or known way for this library to fail which could mean you hit a rare bug or that your log store behaves a little differently to others people are using.

Some more questions that might help:

  1. The code you included is from replicateTo. Which Transport are you using? If it supports pipeline replication (which the built in NetworkTransport does) then the actual code running in the replication loop is more likely to have been the equivalent block in pipelineReplicate which sends through pipelineSend and recieves response via pipelineDecode you usually see errors on the leader like aborting pipeline replication when an error occurs during pipeline replication and the leader will revert to RPC replication and should eventually hit the same error and execute the code you pasted.

I'm not sure how to answer the other questions without the full story here - are you able to share the logs from the leader and follower (filtered to just the raft libary output with timestamps) so we can help understand.

This does sound like a bug potentially. The one way I can think it could possibly happen now would be if you have network connectivity issue on the replication TCP connection - the replication loop could in theory get stuck waiting for a response indefinitely if it gets just enough packets to keep the connection alive but not actually a full response. This makes sense because the standard NetworkTransport uses a separate TCP connection for heartbeats to avoid head-of-line blocking on IO, the tradeoff is it's possible for one to fail but not the other at the switch level. This is very unusual but it is possible!

pranayhere commented 3 months ago

Hi @banks, Thank you for the reply.

To answer your question, we currently use NetworkTransport provided by the library itself. I checked further for aborting pipeline replication but i couldn't find it in the logs. Let me share all the logs between the 9.40am to 9.50am. Issue happened between 9.44am to 9.45am

Leader Logs:

Info 2024-08-22 09:40:10.019Z snapshot.go:179 _podname=Leader _msg=raft: starting snapshot up to index=151113998
Info 2024-08-22 09:40:10.178Z snapshot.go:241 _podname=Leader_msg=raft: compacting logs from=151069001 to=151103759
Info 2024-08-22 09:40:10.858Z snapshot.go:209 _podname=Leader_msg=raft: snapshot complete up to index=151113998
Warn 2024-08-22 09:41:27.628Z raft.go:1064 _podname=Leader _msg=raft: failed to contact server-id=<Follower> time=511.794387ms
Info 2024-08-22 09:43:36.899Z snapshot.go:179 _podname=Leader _msg=raft: starting snapshot up to index=151144158
Info 2024-08-22 09:43:37.068Z snapshot.go:241 _podname=Leader _msg=raft: compacting logs from=151103760 to=151134065
Info 2024-08-22 09:43:37.729Z snapshot.go:209 _podname=Leader _msg=raft: snapshot complete up to index=151144158
Info 2024-08-22 09:46:28.521Z snapshot.go:179 _podname=Leader _msg=raft: starting snapshot up to index=151171829
Info 2024-08-22 09:46:28.679Z snapshot.go:241 _podname=Leader _msg=raft: compacting logs from=151134066 to=151161589
Info 2024-08-22 09:46:29.199Z snapshot.go:209 _podname=Leader _msg=raft: snapshot complete up to index=151171829
Warn 2024-08-22 09:48:18.884Z raft.go:1064 _podname=Leader _msg=raft: failed to contact server-id=<Follower> time=543.103886ms

Follower Logs:

Info 2024-08-22 09:40:24.046Z snapshot.go:179 _podname=Follower _msg=raft: starting snapshot up to index=151115969
Info 2024-08-22 09:40:24.226Z snapshot.go:241 _podname=Follower _msg=raft: compacting logs from=151071927 to=151105729
Info 2024-08-22 09:40:24.826Z snapshot.go:209 _podname=Follower _msg=raft: snapshot complete up to index=151115969
Info 2024-08-22 09:42:28.375Z snapshot.go:179 _podname=Follower _msg=raft: starting snapshot up to index=151134079
Info 2024-08-22 09:42:28.536Z snapshot.go:241 _podname=Follower _msg=raft: compacting logs from=151105730 to=151123871
Info 2024-08-22 09:42:28.866Z snapshot.go:209 _podname=Follower _msg=raft: snapshot complete up to index=151134079
Error 2024-08-22 09:44:36.115Z raft.go:1540 _podname=Follower _msg=raft: failed to append to logs error=unable to store logs within log store, err=Error 1213 (40001): Deadlock found when trying to get lock; try restarting transaction
Warn 2024-08-22 09:44:36.126Z raft.go:1485 _podname=Follower _msg=raft: failed to get previous log previous-index=151153341 last-index=151153296 error=log not found
Warn 2024-08-22 09:44:36.136Z raft.go:1485  _podname=Follower _msg=raft: failed to get previous log previous-index=151153342 last-index=151153296 error=log not found
Info 2024-08-22 09:45:29.855Z snapshot.go:179 _podname=Follower _msg=raft: starting snapshot up to index=151153288
Info 2024-08-22 09:45:29.986Z snapshot.go:241 _podname=Follower _msg=raft: compacting logs from=151123872 to=151143056
Info 2024-08-22 09:45:30.346Z snapshot.go:209 _podname=Follower _msg=raft: snapshot complete up to index=151153288

Meanwhile, I'll also check further from pipelineReplicate perspective. Thanks for the direction.