Skip to content
This repository has been archived by the owner on Dec 17, 2018. It is now read-only.

Processing a large incoming AppendEntriesReply in I/O thread can trigger an election timeout on the Receiver #40

Open
ZymoticB opened this issue Mar 22, 2014 · 3 comments
Assignees

Comments

@ZymoticB
Copy link

Not sure the best way to put the trace logging into a github issue so I'll just leave that messiness until the end.

The situation I have encountered is that with a small enough election timeout (I am using 300ms) when a node tries to re-enter the cluster there can be enough entries in the first AppendEntries message it receives that the I/O thread actually blocks long enough for an election timeout. This seems to cause that node to then send AppendEntriesReplies to all of the AppendEntries that backed up (due to heartbeats) but with the new term (since the node started an election). I was able to "fix" this by adding a call to scheduleElectionTimeout() at the end of each iteration of the for(LogEntry entry : entries) loop in onAppendEntries. Not a particularly elegant solution. Changing config params will also fix it but I thought it was worth reporting.

I think a follower could also just ignore AppendEntriesReply RPCs instead of failing on the precondition of being a leader. However, I'm sure you have spent more time with the algorithm than me and may be able to think of a reason why that would be a bad idea.

Here is some evidence of the issue.

The exception on the current leader

[New I/O worker #4] TRACE io.libraft.algorithm.RaftAlgorithm - agent2: RequestVote from agent1: term:2 lastLogIndex:15 lastLogTerm:1
[New I/O worker #4] INFO io.libraft.algorithm.RaftAlgorithm - agent2: changing role LEADER->FOLLOWER in term 2
[New I/O worker #4] INFO io.libraft.algorithm.RaftAlgorithm - agent2: leader changed from agent2 to null
[New I/O worker #4] TRACE io.libraft.algorithm.RaftAlgorithm - agent2: AppendEntriesReply from agent1: term:2 prevLogIndex:6 entryCount:9 applied:false
[New I/O worker #4] ERROR org.jboss.netty.channel.SimpleChannelUpstreamHandler - agent2: uncaught exception processing rpc:AppendEntriesReply{source=agent1, destination=agent2, term=2, prevLogIndex=6, entryCount=9, applied=false} from agent1
java.lang.IllegalStateException: role:FOLLOWER

The follower timing out while processing log entries

[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: AppendEntries from agent2: term:1 commitIndex:15 prevLogIndex:6 prevLogTerm:1 entryCount:9
[New I/O worker #3] INFO io.libraft.algorithm.RaftAlgorithm - agent1: leader changed from null to agent2
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=7, term=1, command=PrintCommand{commandId=7613830809909165162, toPrint=a string 6}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=8, term=1, command=PrintCommand{commandId=5002286235719145647, toPrint=a string 7}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=9, term=1, command=PrintCommand{commandId=-1668250811023977474, toPrint=a string 8}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=10, term=1, command=PrintCommand{commandId=3591929921586279742, toPrint=a string 9}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=11, term=1, command=PrintCommand{commandId=7008605882065303194, toPrint=a string 10}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=12, term=1, command=PrintCommand{commandId=13354652848121111, toPrint=a string 11}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=13, term=1, command=PrintCommand{commandId=-6877329815755806333, toPrint=a string 12}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=14, term=1, command=PrintCommand{commandId=-5953431672926618152, toPrint=a string 13}}
[New I/O worker #3] TRACE io.libraft.algorithm.RaftAlgorithm - agent1: add entry:ClientEntry{type=CLIENT, index=15, term=1, command=PrintCommand{commandId=7768522539884141905, toPrint=a string 14}}
At index 7 type PRINT
At index 8 type PRINT
At index 9 type PRINT
At index 10 type PRINT
At index 11 type PRINT
At index 12 type PRINT
At index 13 type PRINT
At index 14 type PRINT
At index 15 type PRINT
[Timer-0] INFO io.libraft.algorithm.RaftAlgorithm - agent1: handle election timeout
[Timer-0] INFO io.libraft.algorithm.RaftAlgorithm - agent1: changing role FOLLOWER->CANDIDATE in term 2
[Timer-0] INFO io.libraft.algorithm.RaftAlgorithm - agent1: leader changed from agent2 to null
@ZymoticB
Copy link
Author

Two things to note:

  1. I'm not locking up the thread in the listener, just printing log index and message type so I'm not adding a significant amount of time there (although with an election timeout of 300ms it is enough)
  2. I'm not sure why a follower would reply here https://github.com/allengeorge/libraft/blob/master/libraft-core/src/main/java/io/libraft/algorithm/RaftAlgorithm.java#L1302. At that point we can assert that the leader is either null or not the person who sent the AppendEntries message. Shouldn't the follower just ignore this message? (Removing this also fixes the issue, although still means there is a spurious election)

@allengeorge
Copy link
Owner

First off, thanks for reporting these issues!

I noticed both these while doing some preliminary testing on AWS, but decided to punt on them until I finished the work on snapshots. Moreover, until I looked at the log above I didn't have a good mental model of how that ISE could be tripped. That said, I have created #41 and can prioritize work on that. The fix on that is simple and should be done in the onAppendEntriesReply method.

The reason a response is sent in https://github.com/allengeorge/libraft/blob/master/libraft-core/src/main/java/io/libraft/algorithm/RaftAlgorithm.java#L1302 is so that peers who are behind can notice the term change and catch up. If you simply ignore the request you lose the opportunity to force the cluster to make progress.

@allengeorge
Copy link
Owner

As to the election timeout issue: there are a lot of optimizations that can be done to speed up operation. Again, I decided to punt on that until after the snapshot work had been completed. Through pure coincidence I was thinking about this earlier and came up with this solution:

Instead of creating an AppendEntries message on every heartbeat that contains all the missing entries for a peer, one could simply modify https://github.com/allengeorge/libraft/blob/master/libraft-core/src/main/java/io/libraft/algorithm/RaftAlgorithm.java#L1061 to only send min(MAX_ENTRIES_PER_MESSAGE, (lastLog.getIndex() - prevLogIndex)). You trade off catching up immediately (since it'll take a few rounds for the follower's log to match the leader's) but you do bound the amount of work that's done for each AppendEntries (both on processing and receiving). Note that at the same time, commitIndex at https://github.com/allengeorge/libraft/blob/master/libraft-core/src/main/java/io/libraft/algorithm/RaftAlgorithm.java#L1073 will have to be min(commitIndex, entries.last().getIndex())).

@allengeorge allengeorge self-assigned this Apr 2, 2014
@allengeorge allengeorge changed the title Processing in I/O thread can cause AppendEntriesReply's to be sent to Followers Processing incoming I/O thread can cause AppendEntriesReply's to be sent to Followers Apr 8, 2014
@allengeorge allengeorge changed the title Processing incoming I/O thread can cause AppendEntriesReply's to be sent to Followers Processing a large incoming AppendEntriesReply in I/O thread can trigger an election timeout on the Receiver Apr 8, 2014
Sign up for free to subscribe to this conversation on GitHub. Already have an account? Sign in.
Labels
None yet
Projects
None yet
Development

No branches or pull requests

2 participants