Closed madjam closed 8 years ago
Here is what looks like a similar exception. This one is logged on a follower node that was killed and brought back up. This particular node was killed earlier when it was the leader.
java.lang.IndexOutOfBoundsException: inconsistent index: 65532 at io.atomix.catalyst.util.Assert.index(Assert.java:45) ~[classes/:na] at io.atomix.copycat.server.storage.Segment.append(Segment.java:265) ~[classes/:na] at io.atomix.copycat.server.storage.Log.append(Log.java:289) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.doAppendEntries(PassiveState.java:151) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:83) ~[classes/:na] at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:63) ~[classes/:na] at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:278) ~[classes/:na] at io.atomix.copycat.server.state.ServerState.lambda$registerHandlers$26(ServerState.java:427) ~[classes/:na] at io.atomix.copycat.server.state.ServerState$$Lambda$40/1408265584.handle(Unknown Source) ~[na:na] at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection.lambda$0(NettyConnection.java:90) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection$$Lambda$50/524175063.run(Unknown Source) ~[na:na] at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$7(Runnables.java:20) ~[classes/:na] at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1471868639.run(Unknown Source) [classes/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_25] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_25] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_25]
This is great. I'll try to reproduce them.
Based on the stack traces, these are two similar but different exceptions. One is thrown when appending to the log, and one is thrown when reading.
The exception thrown when appending to the log indicates that index of the entry being appended does not match the next index in the log. This is an extra consistency check to catch these types of bugs, so I suppose it did its job. Basically, the Segment's nextIndex() is different than the Entry's index(). The two should be resolved during AppendRequest handling (by truncating the log or skipping entries as necessary).
The exception thrown when reading the log indicates that the offset stored in the Segment file does not match with the expected offset in the index. The index is built from disk by reading offsets for each entry, so these should match. But there are many potential factors that can mess that up like truncating the log and searching the index. I'll have to go through that again and see if I can break it. I suspect something in OffsetIndex is off. That can actually cause both of these exceptions since the segment's nextIndex() is dependent on the entries in its OffsetIndex.
On Oct 22, 2015, at 12:46 PM, madjam notifications@github.com wrote:
Here is what looks like a similar exception. This one is logged on a follower node that was killed and brought back up. This particular node was killed earlier when it was the leader.
java.lang.IndexOutOfBoundsException: inconsistent index: 65532 at io.atomix.catalyst.util.Assert.index(Assert.java:45) ~[classes/:na] at io.atomix.copycat.server.storage.Segment.append(Segment.java:265) ~[classes/:na] at io.atomix.copycat.server.storage.Log.append(Log.java:289) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.doAppendEntries(PassiveState.java:151) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:83) ~[classes/:na] at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:63) ~[classes/:na] at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:278) ~[classes/:na] at io.atomix.copycat.server.state.ServerState.lambda$registerHandlers$26(ServerState.java:427) ~[classes/:na] at io.atomix.copycat.server.state.ServerState$$Lambda$40/1408265584.handle(Unknown Source) ~[na:na] at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection.lambda$0(NettyConnection.java:90) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection$$Lambda$50/524175063.run(Unknown Source) ~[na:na] at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$7(Runnables.java:20) ~[classes/:na] at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1471868639.run(Unknown Source) [classes/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_25] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_25] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_25]
— Reply to this email directly or view it on GitHub.
I set up a test to reproduce this but thus far haven't been successful at reproducing either. @madjam do you think you can manage to get a test that reproduces this in some branch? It might be difficult if the bug relies on how the Raft logic is skipping/truncating entries. I'll keep trying to find it. Once I can get it in a debugger it should be an easy fix.
Interesting... The code path that this follows indicates that the previous index/term for the AppendRequest
were 0
. It's odd that it would ever be 0
after the cluster's been running for any length of time. @madjam the debug logging should print the full AppendRequest
with all its fields. If we can get the AppendRequest
that caused the IndexOutOfBoundsException
I think I can probably figure out what's going on.
I wonder if MemberState
s nextIndex
is never properly initialized at the beginning of the leader's term. I don't think that should be causing this issue but doesn't help.
I pushed a fix for the nextIndex
issue to the reset-next-index
branch: d5f43edac73e32a400a79c9c8bb634584e37e6b6
I'm not sure that is related to this problem, but it certainly could be. Still looking at the other logic.
Unfortunately I don't have the logs from that run saved. But I don't think it would be very hard to reproduce. Let me try and I'll post the request dumps.
Ugh I messed that commit up: 530c3fa5b7ecbc700e012845d92e0555be12822f
No success so far reproducing this particular exception. But did run into what looked like some weird asymmetric network partition due to incoming netty connections being closed by the server. I'll create a separate issue for it once I have enough info captured.
Excellent. Yeah, I'm trying to reproduce this based on your steps above and having no luck so far myself either. I want to at least try to find out if it's related to 530c3fa5b7ecbc700e012845d92e0555be12822f which needs to be merged regardless.
Something to note is that after I saw this exception a restart restored the node to proper health. That lead me to suspect that this has something to do with one of the in-memory indices getting corrupted. Whether that is nextIndex or the segment offset index is not clear.
Ahh that's interesting and relevant. You may be totally right. The IndexOutOfBoundsException
is particularly indicative of that since it's built from disk on load. But it's also possible that some non-deterministic set of operations in the cluster (like truncating/skipping entries based on the leader's log) put it in a hard-to-reproduce state.
Just have to keep watching out for it I guess.
Check this out: https://travis-ci.org/atomix/copycat#L4628
This seems to happen right when the leader transitions. The nextIndex
for some member is outside the bounds of the log. But this may actually be fixed by the commit mentioned above. It's hard to tell exactly what's going on since this exception happened without debug logging enabled :-(
I'm going to merge the nextIndex
fix.
Another assertion failure potentially pointing to one of the indices getting out of sync. Note that here I am NOT running with your nextIndex
fix.
java.lang.IndexOutOfBoundsException: cannot truncate committed entries at io.atomix.catalyst.util.Assert.index(Assert.java:45) ~[classes/:na] at io.atomix.copycat.server.storage.Log.truncate(Log.java:418) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.doAppendEntries(PassiveState.java:151) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:83) ~[classes/:na] at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:63) ~[classes/:na] at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:278) ~[classes/:na] at io.atomix.copycat.server.state.ServerState.lambda$10(ServerState.java:427) ~[classes/:na] at io.atomix.copycat.server.state.ServerState$$Lambda$43/1177990944.handle(Unknown Source) ~[na:na] at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection.lambda$0(NettyConnection.java:90) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection$$Lambda$49/1611610784.run(Unknown Source) ~[na:na] at io.atomix.catalyst.util.concurrent.Runnables.lambda$0(Runnables.java:20) ~[classes/:na] at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/985397764.run(Unknown Source) [classes/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_25] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_25] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_25]
This time I have the debug logs saved. I will look through them to see what happened before.
Again the test involved killing the current leader (in a 3 node cluster), bringing up back up and allowing some time for its log to catch up and then repeating the process. All the while a separate client is appending log entries at the rate of a few hundred per second.
These must all be related to the same issue. I'll spend the weekend hitting it hard and hopefully get a good reproducer. Hopefully the logs will help. Should be able to deduce the expected state of the log from that.
On Oct 23, 2015, at 11:37 AM, Madan Jampani notifications@github.com wrote:
Another assertion failure potentially pointing to one of the indices getting out of sync. Note that here I am NOT running with your nextIndex fix.
java.lang.IndexOutOfBoundsException: cannot truncate committed entries at io.atomix.catalyst.util.Assert.index(Assert.java:45) ~[classes/:na] at io.atomix.copycat.server.storage.Log.truncate(Log.java:418) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.doAppendEntries(PassiveState.java:151) ~[classes/:na] at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:83) ~[classes/:na] at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:63) ~[classes/:na] at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:278) ~[classes/:na] at io.atomix.copycat.server.state.ServerState.lambda$10(ServerState.java:427) ~[classes/:na] at io.atomix.copycat.server.state.ServerState$$Lambda$43/1177990944.handle(Unknown Source) ~[na:na] at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection.lambda$0(NettyConnection.java:90) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection$$Lambda$49/1611610784.run(Unknown Source) ~[na:na] at io.atomix.catalyst.util.concurrent.Runnables.lambda$0(Runnables.java:20) ~[classes/:na] at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/985397764.run(Unknown Source) [classes/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_25] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_25] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_25]
This time I have the debug logs saved. I will look through them to see what happened before.
Again the test involved killing the current leader (in a 3 node cluster), bringing up back up and allowing some time for its log to catch up and then repeating the process. All the while a separate client is appending log entries at the rate of a few hundred per second.
— Reply to this email directly or view it on GitHub.
After parsing the logs here's what I gathered:
I killed s1 (the current leader) The following log entries are from s2. It just voted for s3 as leader. The first AE s3 sends to s2 after becoming leader points to a very old logIndex. Most likely the nextIndex value is had when it was the previous leader?
I'm expecting your change to reset nextIndex should fix this. Also per the Raft paper that is the right thing to do.
Volatile state on leaders: (Reinitialized after election) nextIndex[] for each server, index of the next log entry to send to that server (initialized to leader last log index + 1)
Oh awesome! I just pushed it. We should watch out for this from here on.
There's actually another related bug too. I don't think matchIndex
is reset either. That may seem fine on the surface, but matchIndex
is really specific to a single leader's term. If matchIndex
isn't reset for each follower at the beginning of a leader's term, the following can happen:
1
1
and replicates it to one follower and increments the follower's matchIndex
to 1
1
to a majority of the cluster, detects a partition, and steps down1
is elected for term 2
(which is fine since index 1
is only stored on 2/5 servers)1
from its log making the follower's lastIbdex
0
again1
in its log since it's partitioned3
At this point, the new leader still has matchIndex
equal to 1
for the follower. But the leader's logic is such that once a follower responds with some lastIndex
or responds success=true
to an AppendRequest
, the leader will never decrease nextIndex
below matchIndex + 1
. In this scenario, that would result in preventing the follower from progressing at all. Each AppendRequest
from the leader would have a previous index equal to the follower's matchIndex
(1
) since max(response.logIndex(), member.matchIndex())
is 1
, but the follower's log is empty.
This could be resolved by always forcing the matchIndex
to be reset to response.logIndex()
, but if responses are received out of order that can result in matchIndex
being incorrectly decreased. Leaders have to reset matchIndex
For each follower to 0
At the beginning of their term. They should converge on the correct matchIndex
very quickly based on response.logIndex()
I'll push another fix for this.
That makes sense. That is what the paper specifies on how a matchIndex should be initialized. So that is a good change to make.
I ran into the inconsistent index assertion failure again. But this time I probably have a good idea what triggered it.
Scenario:
15:15:21.281 [copycat-server-Address[localhost/127.0.0.1:5003]] ERROR i.a.c.u.c.SingleThreadContext - An uncaught exception occurred java.lang.IllegalStateException: inconsistent index: 1 at io.atomix.catalyst.util.Assert.state(Assert.java:69) ~[classes/:na] at io.atomix.copycat.server.storage.Segment.get(Segment.java:319) ~[classes/:na] at io.atomix.copycat.server.storage.Log.get(Log.java:319) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.entriesCommit(LeaderState.java:1064) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.commit(LeaderState.java:1004) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.lambda$commit$127(LeaderState.java:900) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator$$Lambda$118/1481624571.apply(Unknown Source) ~[na:na] at java.util.Map.computeIfAbsent(Map.java:957) ~[na:1.8.0_25] at io.atomix.copycat.server.state.LeaderState$Replicator.commit(LeaderState.java:898) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.access$100(LeaderState.java:842) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState.command(LeaderState.java:422) ~[classes/:na]
Ahh this is great. That it was after compaction and after a configuration change is interesting. It seems to me like maybe entry 1
(a no-op entry which is always removed the first time the segment is compacted) was removed from the log, but the OffsetIndex
believed the first index in the segment was still 1
. Maybe this is specific to the first index in a segment? I will throw some tests at OffsetIndex
.
What we know if that OffsetIndex
indicated that index 1
had not been removed during compaction and that it was (probably) the first entry in the segment. The get()
method then read the offset at position 0
(64
in the file, since the SegmentDescriptor
consumes the first 64 bytes in each segment) and it was not 0
(the offset for index 1
).
This may be a huge help.
Hmm... No such luck.
@madjam I was able to reproduce this in a debugger. But the way in which I reproduced it was via a bad configuration. Basically, if I start several servers each with the same log directory, I can get the stack trace in https://github.com/atomix/copycat/issues/29#issuecomment-150335741 Basically, I found this by accident. I'm assuming this isn't what happened in your case, but worth sharing.
@madjam this is still an issue right? I still haven't been able to reproduce it (aside from by human error). I'm gonna see if Jepsen can reproduce it for me tonight.
I haven't seen it since your change to reset all volatile state at leadership beginning. It is likely that that fixed it or made it much harder to reproduce :)
Hmm... Hopefully it's the former! We'll leave this open and I'll try to see if Jepsen can break it.
@kuujo Was there anything interesting in the Jepsen run?
@madjam nothing yet. But really the most sensitive part of the algorithms (configuration changes and log compaction) are not well tested in Jepsen yet, so we're working on updating tests to randomly add and remove servers and compact the logs much more frequently. If tests continue to go well in that case that will make me feel warm and cozy.
I got a little sidetracked going through all the code and really thinking about the algorithms. I managed to find several bugs and submitted PRs for all of them. Once they're cleaned up and merged and I review the rest of the code I'll take to Jepsen again.
I did find a case where this bug was likely still possible. The original fix to reset MemberState
at the start of the leader's term only did so for ACTIVE
members. So, essentially if a leadership change took place while a joining server was still being caught up to the rest of the cluster in PASSIVE
mode, we likely would have seen this.
Ran into this issue running a 3 node set up of AtomixReplicas. Looks like this happened immediately after a Leader switch. I need to look at the code to see what could be going on.
2015-11-12 13:43:20,623 | INFO | .168.56.104:9976 | ServerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Transitioning to FOLLOWER
2015-11-12 13:43:20,643 | INFO | .168.56.104:9976 | ServerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Found leader /192.168.56.101:9976
2015-11-12 13:43:20,645 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2346, logTerm=1, entries=[1], commitIndex=2346, globalIndex=0]: Request log term does not match local log term unknown for the same entry
2015-11-12 13:43:20,653 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2345, logTerm=1, entries=[2], commitIndex=2347, globalIndex=0]: Request log term does not match local log term unknown for the same entry
2015-11-12 13:43:20,674 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2345, logTerm=1, entries=[3], commitIndex=2347, globalIndex=0]: Request log term does not match local log term unknown for the same entry
2015-11-12 13:43:20,682 | ERROR | .168.56.104:9976 | SingleThreadContext | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | An uncaught exception occurred
java.lang.IndexOutOfBoundsException: cannot truncate committed entries
at io.atomix.catalyst.util.Assert.index(Assert.java:45)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.storage.Log.truncate(Log.java:433)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.PassiveState.doAppendEntries(PassiveState.java:151)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:83)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:63)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:278)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.ServerState.lambda$registerHandlers$26(ServerState.java:438)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.copycat.server.state.ServerState$$Lambda$129/798437420.handle(Unknown Source)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.catalyst.transport.NettyConnection.lambda$handleRequest$6(NettyConnection.java:90)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.catalyst.transport.NettyConnection$$Lambda$152/1794568287.run(Unknown Source)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$7(Runnables.java:20)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$47/891987813.run(Unknown Source)[72:org.onosproject.onlab-thirdparty:1.4.0.SNAPSHOT]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)[:1.8.0_31]
at java.util.concurrent.FutureTask.run(FutureTask.java:266)[:1.8.0_31]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)[:1.8.0_31]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)[:1.8.0_31]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)[:1.8.0_31]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)[:1.8.0_31]
at java.lang.Thread.run(Thread.java:745)[:1.8.0_31]
2015-11-12 13:43:21,395 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2349, logTerm=2, entries=[3], commitIndex=2352, globalIndex=2345]: Request log term does not match local log term 1 for the same entry
2015-11-12 13:43:21,543 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2349, logTerm=2, entries=[3], commitIndex=2352, globalIndex=2345]: Request log term does not match local log term 1 for the same entry
2015-11-12 13:43:21,693 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2349, logTerm=2, entries=[3], commitIndex=2352, globalIndex=2345]: Request log term does not match local log term 1 for the same entry
2015-11-12 13:43:21,846 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2349, logTerm=2, entries=[3], commitIndex=2352, globalIndex=2345]: Request log term does not match local log term 1 for the same entry
2015-11-12 13:43:21,993 | WARN | .168.56.104:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.104:9976 - Rejected AppendRequest[term=2, leader=-1062706512, logIndex=2349, logTerm=2, entries=[3], commitIndex=2352, globalIndex=2345]: Request log term does not match local log term 1 for the same entry
A repro this time a debug level logging shows a curious sequence events:
2015-11-12 14:38:43,257 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[6], commitIndex=3518, globalIndex=0]
2015-11-12 14:38:43,258 | WARN | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Rejected AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[6], commitIndex=3518, globalIndex=0]: Request log term does not match local log term unknown for the same entry
2015-11-12 14:38:43,259 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515]
2015-11-12 14:38:43,259 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout
2015-11-12 14:38:43,302 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout
2015-11-12 14:38:43,303 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0]
2015-11-12 14:38:43,304 | WARN | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Rejected AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0]: Request log term does not match local log term unknown for the same entry
2015-11-12 14:38:43,304 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515]
2015-11-12 14:38:43,305 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout
2015-11-12 14:38:43,381 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout
2015-11-12 14:38:43,381 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=0, entries=[8], commitIndex=3523, globalIndex=3516]
2015-11-12 14:38:43,384 | ERROR | .168.56.101:9976 | SingleThreadContext | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | An uncaught exception occurred
java.lang.IndexOutOfBoundsException: cannot truncate committed entries
This particular back and forth (right before exception) is very puzzling.
Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0]
Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515]
Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=0, entries=[8], commitIndex=3523, globalIndex=3516]
Excellent!
@madjam sorry I have been seeing your comments/PRs. I've been out of town since Monday but back now and back to work tomorrow. I'd love to figure this one out. Just too tired ATM :-)
On Nov 12, 2015, at 5:46 PM, Madan Jampani notifications@github.com wrote:
A repro this time a debug level logging shows a curious sequence events:
2015-11-12 14:38:43,257 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[6], commitIndex=3518, globalIndex=0] 2015-11-12 14:38:43,258 | WARN | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Rejected AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[6], commitIndex=3518, globalIndex=0]: Request log term does not match local log term unknown for the same entry 2015-11-12 14:38:43,259 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515] 2015-11-12 14:38:43,259 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout 2015-11-12 14:38:43,302 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout 2015-11-12 14:38:43,303 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0] 2015-11-12 14:38:43,304 | WARN | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Rejected AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0]: Request log term does not match local log term unknown for the same entry 2015-11-12 14:38:43,304 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515] 2015-11-12 14:38:43,305 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout 2015-11-12 14:38:43,381 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Reset heartbeat timeout 2015-11-12 14:38:43,381 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=0, entries=[8], commitIndex=3523, globalIndex=3516] 2015-11-12 14:38:43,384 | ERROR | .168.56.101:9976 | SingleThreadContext | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | An uncaught exception occurred java.lang.IndexOutOfBoundsException: cannot truncate committed entries Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=1, entries=[7], commitIndex=3518, globalIndex=0] Sent AppendResponse[status=OK, term=2, succeeded=false, logIndex=3515] Received AppendRequest[term=2, leader=-1062706511, logIndex=3516, logTerm=0, entries=[8], commitIndex=3523, globalIndex=3516] — Reply to this email directly or view it on GitHub.
@kuujo No worries :) I will shortly add some more details on the issue from a debug trace I captured. That should help figuring this out.
Here is a summary from the debug trace I gathered.
I restarted a 3 node cluster of AtomixReplica
s. Before restart, logs on all nodes were up to date and at index 3516
After restart, S1
gets elected as leader, appends a NoOpEntry
to its log at index 3517
and successfully replicates it to S2
and S3
. S1
then applies the NoOp entry and cleans it from the log.
Clients on each node now attempt to register and those RegisterRequest
s get forwarded to S1
(leader) which appends a RegisterEntry
to its log at index 3518 and sends out AppendRequest
to S2
and S3
. For some reason S2
and S3
never get this AppendRequest
. S1
subsequently sends out 2 more AppendRequest
s (one for each remaining client registration) and those never reach S2
and S3
. Note: those messages probably did reach the nodes physically and it is possible that the they never get handled. S2
times out on leader heart beats, starts an election and becomes leader (after getting votes from S3
). S1
steps down after seeing an AppenedRequest
from S2
with a greater term. S2
's last log index is 3517
(NoOp Entry) and the AppendRequest
received by S1
has logIndex as 3517. But when S1
looks up entry 3517 from its its log it gets null
and that is the source of the following exchange:
2015-11-12 14:38:38,099 | DEBUG | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Received AppendRequest[term=2, leader=-1062706511, logIndex=3517, logTerm=1, entries=[1], commitIndex=0, globalIndex=0]
2015-11-12 14:38:38,100 | WARN | .168.56.101:9976 | FollowerState | 72 - org.onosproject.onlab-thirdparty - 1.4.0.SNAPSHOT | /192.168.56.101:9976 - Rejected AppendRequest[term=2, leader=-1062706511, logIndex=3517, logTerm=1, entries=[1], commitIndex=0, globalIndex=0]: Request log term does not match local log term unknown for the same entry
I see couple of issues to understand
S2
and S3
and that causes them to not respond to the AppendRequest
from S1
for session registration. I don't yet have clear understanding of the threading model to know if that is somehow causing this.NoOpEntry
that is cleaned from from S1
's log at index 3517
should still be accessible and not return null when looked up, right?Wow interesting. So, S2 and S3 never receive and thus never respond to the Append for the RegisterEntry
right? It could be a deadlock, but I've also seen some cases in the past (which I attempted to fix) where exceptions were thrown in some thread during the handling of a request but never got logged. IIRC you can't set an uncaught exception handler on an Executor
thread, so I think Catalyst tries to catch them itself, but if the Executor
is used directly (which it is a lot) that doesn't help. That can result in this type of behavior. Maybe need to wrap the Executor
that's exposed by ThreadContext
to catch and log exceptions.
On the NoOpEntry
point, once it's cleaned it can be safely set to null
. For the log cleaning algorithm, tombstones must still be readable after clean()
ed until globalIndex
/majorIndex
has surpassed the entry index, but normal entries can be hidden. In theory, this reduces the amount of network traffic since cleaned normal entries don't need to be sent to followers. The design of state machines should be such that an entry isn't cleaned until it no longer contributes to the state machine's state. But I think this is a bug in how no-op entries are currently cleaned. Because no-op entries reset the timeouts for all sessions, they should be retained until stored on all servers to ensure session expiration is consistent, so they should be treated as tombstones, in which case they will indeed be visible until majorIndex
equals the no-op entry index (which in this case it still would since all servers received it).
The only critical (for safety) part of cleaning entries is that tombstones are null
if they've been cleaned and their index is <= majorIndex
. Returning null
for normal entries like the current NoOpEntry
is just an optimization that assumes if an entry has been cleaned, there's some later committed entry that effectively overrides it. However, considering that we allow reads from followers, this maybe should not be the case or at least be configurable. If a follower is a heartbeat behind the leader but misses a bunch of entries that were cleaned on the leader, it won't break the sequential consistency model, but it will mean that followers are less likely to have more up-to-date state, so I wouldn't be opposed to at least making sure entries above the limit for reading from followers (a heartbeat behind the leader) aren't treated that way.
I'll open another issue to make no-op entries tombstones.
[Re-posting comment from the PR thread]
The issue I noticed occurs when a node loses leadership with globalIndex
and logIndex
both equal and the entry at that location in the log is a NoOpEntry
.
In this state AppendRequest
from the new leader will be rejected with the following error:
Rejected AppendRequest[term=2, leader=-1062706512, logIndex=5709, logTerm=1, entries=[1], commitIndex=0, globalIndex=0]: Request log term does not match local log term unknown for the same entry
This is because when this node tries to validate previous entry term, Log#get returns null since NoOpEntry
was garbage collected. I wonder if major compactor should only be allowed to garbage collect entries upto globalIndex - 1
? Or did I miss something?
Oops guess I should have posted this here:
Ahh right. No I think you're totally right. This definitely needs to be implemented. I've seen that odd rejection of an AppendRequest before and never realized the cause.
Yeah this commit definitely wasn't a fix for that, it was just a fix for another potential problem (inconsistency in session expiration after leadership changes).
Actually, a similar restriction is already in place on log compaction. A segment can only be compacted if there's a non-empty segment with a committed entry later in the log. This ensures there aren't gaps in the log after compaction since one committed, uncompacted entry will always remain later in the log. You're right it seems the same needs to be done to ensure logIndex/logTerm can be checked in AppendRequest handling. I don't see any safety issues with this. Just like in log compaction where we slow down compaction by one entry, we're just making the globalIndex increase potentially one entry slower which is fine. Awesome catch! So glad that was figured out.
I'll submit a PR for this unless you want to. Right now globalIndex is calculated as the lowest matchIndex (the lowest index stored on all servers). I think what needs to happen is globalIndex should just be limited to max(lowest matchIndex - 1, 0). That will put the logic for handling this on the leader rather than inside the log/log compaction since this is more a detail of AppendRequest/Response.
On Nov 17, 2015, at 3:39 PM, Madan Jampani notifications@github.com wrote:
[Re-posting comment from the PR hread]
The issue I noticed occurs when a node loses leadership with globalIndex and logIndex both equal and the entry at that location in the log is a NoOpEntry.
In this state AppendRequest from the new leader will be rejected with the following error:
Rejected AppendRequest[term=2, leader=-1062706512, logIndex=5709, logTerm=1, entries=[1], commitIndex=0, globalIndex=0]: Request log term does not match local log term unknown for the same entry This is because when this node tries to validate previous entry term, Log#get returns null since NoOpEntry was garbage collected. I wonder if major compactor should only be allowed to garbage collect entries upto globalIndex - 1? Or did I miss something?
— Reply to this email directly or view it on GitHub.
Sounds good. I made this same change you suggested. I can submit a PR shortly.
Hmm... @madjam actually I think this last bug may have been handled incorrectly. I don't think the fix broke anything but it did overlook something. I'm going to open another issue so we can discuss.
I just got another instance of this exception during testing:
java.lang.IllegalStateException: inconsistent index: 24060
at io.atomix.catalyst.util.Assert.state(Assert.java:69) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Segment.get(Segment.java:319) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Log.get(Log.java:321) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.PassiveState.doCheckPreviousEntry(PassiveState.java:110) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:81) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:62) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:284) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ServerState.lambda$connectServer$50(ServerState.java:472) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ServerState$$Lambda$52/210005792.handle(Unknown Source) ~[na:na]
at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.transport.NettyConnection.lambda$handleRequest$2(NettyConnection.java:90) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.transport.NettyConnection$$Lambda$57/509111192.run(Unknown Source) ~[na:na]
at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$12(Runnables.java:20) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1169146729.run(Unknown Source) [atomix-distributed-value.jar:na]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_11]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_11]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_11]
15:27:15.765 [copycat-server-localhost/127.0.0.1:5001] ERROR i.a.c.u.c.SingleThreadContext - An uncaught exception occurred
java.lang.IllegalStateException: inconsistent index: 24060
at io.atomix.catalyst.util.Assert.state(Assert.java:69) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Segment.get(Segment.java:319) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Log.get(Log.java:321) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.PassiveState.doCheckPreviousEntry(PassiveState.java:110) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.PassiveState.handleAppend(PassiveState.java:81) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ActiveState.append(ActiveState.java:62) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.FollowerState.append(FollowerState.java:284) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ServerState.lambda$connectServer$50(ServerState.java:472) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.ServerState$$Lambda$52/210005792.handle(Unknown Source) ~[na:na]
at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.transport.NettyConnection.lambda$handleRequest$2(NettyConnection.java:90) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.transport.NettyConnection$$Lambda$57/509111192.run(Unknown Source) ~[na:na]
at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$12(Runnables.java:20) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1169146729.run(Unknown Source) [atomix-distributed-value.jar:na]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_11]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_11]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_11]
15:27:16.108 [copycat-server-localhost/127.0.0.1:5001] ERROR i.a.c.u.c.SingleThreadContext - An uncaught exception occurred
java.lang.IllegalStateException: inconsistent index: 24060
at io.atomix.catalyst.util.Assert.state(Assert.java:69) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Segment.get(Segment.java:319) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.storage.Log.get(Log.java:321) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.FollowerState.sendPollRequests(FollowerState.java:217) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.FollowerState.lambda$resetHeartbeatTimeout$129(FollowerState.java:174) ~[atomix-distributed-value.jar:na]
at io.atomix.copycat.server.state.FollowerState$$Lambda$33/1337155482.run(Unknown Source) ~[na:na]
at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$12(Runnables.java:20) ~[atomix-distributed-value.jar:na]
at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1169146729.run(Unknown Source) [atomix-distributed-value.jar:na]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_11]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_11]
at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_11]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_11]
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_11]
The scenario here was that I was simply submitting 3k writes/sec to an Atomix DistributedValue
. I didn't force any leader elections or crash any nodes. This one is likely a race condition somewhere. Maybe entries got released back to the entry pool too soon.
I'll see if it continues to reproduce this.
I think #93 may have been what it took to get rid of these. I haven't been able to reproduce any more. Going to close this.
I haven't done much debugging on this yet. But thought I will log this first.
Here's what I was doing:
Here's what I did to see this error:
12:21:00.424 [copycat-server-Address[localhost/127.0.0.1:5003]] ERROR i.a.c.u.c.SingleThreadContext - An uncaught exception occurred java.lang.IllegalStateException: inconsistent index: 1605633 at io.atomix.catalyst.util.Assert.state(Assert.java:69) ~[classes/:na] at io.atomix.copycat.server.storage.Segment.get(Segment.java:319) ~[classes/:na] at io.atomix.copycat.server.storage.Log.get(Log.java:319) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.entriesCommit(LeaderState.java:1040) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.commit(LeaderState.java:980) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.lambda$commit$126(LeaderState.java:876) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator$$Lambda$111/982607974.apply(Unknown Source) ~[na:na] at java.util.Map.computeIfAbsent(Map.java:957) ~[na:1.8.0_25] at io.atomix.copycat.server.state.LeaderState$Replicator.commit(LeaderState.java:874) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState$Replicator.access$100(LeaderState.java:818) ~[classes/:na] at io.atomix.copycat.server.state.LeaderState.accept(LeaderState.java:652) ~[classes/:na] at io.atomix.copycat.server.state.ServerState.lambda$registerHandlers$20(ServerState.java:421) ~[classes/:na] at io.atomix.copycat.server.state.ServerState$$Lambda$34/1604755402.handle(Unknown Source) ~[na:na] at io.atomix.catalyst.transport.NettyConnection.handleRequest(NettyConnection.java:102) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection.lambda$0(NettyConnection.java:90) ~[classes/:na] at io.atomix.catalyst.transport.NettyConnection$$Lambda$50/1312252238.run(Unknown Source) ~[na:na] at io.atomix.catalyst.util.concurrent.Runnables.lambda$logFailure$7(Runnables.java:20) ~[classes/:na] at io.atomix.catalyst.util.concurrent.Runnables$$Lambda$4/1471868639.run(Unknown Source) [classes/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_25] at java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_25] at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_25] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_25] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_25]