Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Raft leader sometimes does not update commitIndex even if the records are replicated to a quorum #8324

Closed
deepthidevaki opened this issue Dec 7, 2021 · 1 comment · Fixed by #8357
Assignees
Labels
kind/bug Categorizes an issue or PR as a bug severity/mid Marks a bug as having a noticeable impact but with a known workaround support Marks an issue as related to a customer support request version:1.3.0 Marks an issue as being completely or in parts released in 1.3.0

Comments

@deepthidevaki
Copy link
Contributor

Describe the bug

Found this strange behavior when investigating a support ticket. https://jira.camunda.com/browse/SUPPORT-12321
The main issue is that that after a failover, there is no leader for a partition when queried via zbctl status

From the logs:
After a failover, new leader is elected - broker 1. Hence the transition to leader is not completed and no Zeebe services are installed.

Analyzed heap dump I observed the following state in the leader:

commitIndex=37
term=39

lastAppendedEntry is the InitialEntry at term 39 and index 38. Indicating that it is not marked as committed.
image

Looking at the followers state, all of them have received the InitialEntry record at index 38.

Leader keeps a map of follower -> RaftMemberContext to keep track of follower's state.
image

RaftMemberContext for the followers shows there are no pending inflight requests (ruling out previous issues we have seen where we observed there were inflight requests that never gets completed or time out).

RaftMemberContext.matchIndex = 38 for all followers.
This means that the leader can commit the record at index 38. But we see that the commitIndex is 37.

In the logs we see following error:

2021-12-07 08:50:54.392 [] [raft-server-1-raft-partition-partition-4] WARN 
      io.atomix.raft.roles.LeaderAppender - RaftServer{raft-partition-partition-4} - AppendRequest{term=39, leader=1, prevLogIndex=33, prevLogTerm=34, entries=5, commitIndex=37} to 0 failed: java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException: Request ProtocolRequest{id=695, subject=raft-partition-partition-4-append, sender=10.184.0.147:26602, payload=byte[]{length=326, hash=-846329344}} to 10.184.0.147:26605 timed out in PT5S
2021-12-07 08:50:54.392 [] [raft-server-1-raft-partition-partition-4] WARN 
      io.atomix.raft.roles.LeaderAppender - RaftServer{raft-partition-partition-4} - AppendRequest{term=39, leader=1, prevLogIndex=33, prevLogTerm=34, entries=5, commitIndex=37} to 2 failed: java.util.concurrent.CompletionException: java.util.concurrent.TimeoutException: Request ProtocolRequest{id=696, subject=raft-partition-partition-4-append, sender=10.184.0.147:26602, payload=byte[]{length=326, hash=-846329344}} to 10.184.0.147:26604 timed out in PT5S

After that there are no errors, and there are no new leader elections happening. This means that both leaders and followers are able to communicate with each other.

After investigating further, I believe the problem is here:
https://github.com/camunda-cloud/zeebe/blob/6ffc0cb40ad355daf2b97882aa4db35765f41382/atomix/cluster/src/main/java/io/atomix/raft/roles/LeaderAppender.java#L208-L210

Consider the following scenario:

  1. Leader sends AppendRequest{index = 38} to follower1 and follower2
  2. follower 1 and follower 2 receives request and writed record at index 38 to its log
  3. follower 1 and follower 2 send ack back to leader
  4. But leader does not receives ack => network problems. (Indicating the above two error logs)
  5. Leader sends heartbeat in the next heartbeat interval (AppendRequest {lastLogIndex = 38, commitIndex = 37})
  6. Followers ack success AppendResponse{success=true, lastLogIndex=38} because they have already received record at index 38
  7. Leader on receiving ack, update match index to 38. But it does not commit because request.entries() is empty.
  8. There are no new records to replicate. So leader keeps sending heartbeats. But commitIndex does not get updated.

To Reproduce

I did not reproduce it yet. We have to find a way to simulate the scenario described above.

Expected behavior
Leader should commit as long as there is a quorum.

Environment:

  • Zeebe Version: 1.2.4
@deepthidevaki deepthidevaki added the kind/bug Categorizes an issue or PR as a bug label Dec 7, 2021
@npepinpe npepinpe added support Marks an issue as related to a customer support request Impact: Availability severity/mid Marks a bug as having a noticeable impact but with a known workaround labels Dec 8, 2021
@npepinpe npepinpe added this to Ready in Zeebe Dec 8, 2021
@deepthidevaki
Copy link
Contributor Author

Received more detailed log with trace level. It gives some more idea on why the "ack" got lost.

Actually the ack is not lost. But it is send after a delay.

2021-12-07 16:40:44.387 [] [raft-server-1-raft-partition-partition-4] DEBUG
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Rejected AppendRequest{term=44, leader=0, prevLogIndex=42, prevLogTerm=43, entries=0, commitIndex=42}: Previous index (42) is greater than the local log's last index (41)
2021-12-07 16:40:44.387 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Sending AppendResponse{status=OK, term=44, succeeded=false, lastLogIndex=41, lastSnapshotIndex=0}
      io.atomix.cluster.messaging.impl.RemoteServerConnection - Received message type raft-partition-partition-4-append from 10.184.0.200:26602
2021-12-07 16:40:44.389 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Received AppendRequest{term=44, leader=0, prevLogIndex=41, prevLogTerm=42, entries=2, commitIndex=42}

Here the AppendRequest is received, at 16:40:44

2021-12-07 16:40:44.389 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Appended IndexedRaftLogEntryImpl{index=42, term=43, entry=InitialEntry{}, record=io.atomix.raft.storage.log.PersistedRaftRecord@353f362c}
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Appended IndexedRaftLogEntryImpl{index=43, term=44, entry=InitialEntry{}, record=io.atomix.raft.storage.log.PersistedRaftRecord@4e971f7e}
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] INFO 
      io.atomix.raft.impl.RaftContext - RaftServer{raft-partition-partition-4} - Setting firstCommitIndex to 42. RaftServer is ready only after it has committed events upto this index
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] DEBUG
      io.atomix.raft.impl.DefaultRaftServer - RaftServer{raft-partition-partition-4} - Server started successfully!
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] INFO 
      io.atomix.raft.partition.impl.RaftPartitionServer - RaftPartitionServer{raft-partition-partition-4} - Successfully started server for partition PartitionId{id=4, group=raft-partition} in 3202ms
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] INFO 
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] DEBUG
2021-12-07 16:40:44.390 [] [raft-server-1-raft-partition-partition-4] INFO 
2021-12-07 16:40:44.392 [] [raft-server-1-raft-partition-partition-4] INFO 
      io.camunda.zeebe.broker.system - Detected 'HEALTHY' components. The current health status of components: {logstream-raft-partition-partition-4=HEALTHY}
      io.atomix.cluster.messaging.impl.RemoteServerConnection - Received message type raft-partition-partition-4-append from 10.184.0.200:26602
2021-12-07 16:40:53.688 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Committed entries up to index 42
2021-12-07 16:40:53.688 [] [raft-server-1-raft-partition-partition-4] TRACE
2021-12-07 16:40:53.690 [] [raft-server-1-raft-partition-partition-4] TRACE
      io.atomix.raft.roles.FollowerRole - RaftServer{raft-partition-partition-4}{role=FOLLOWER} - Sending AppendResponse{status=OK, term=44, succeeded=true, lastLogIndex=43, lastSnapshotIndex=0}

But the ack is sent only at 16:40:53 after more than 5s. By this time leader will timeout the request.

I think the reason for this delay is that raft thread is blocked as described in #8281

@deepthidevaki deepthidevaki self-assigned this Dec 9, 2021
@deepthidevaki deepthidevaki moved this from Ready to In progress in Zeebe Dec 10, 2021
@deepthidevaki deepthidevaki moved this from In progress to Review in progress in Zeebe Dec 10, 2021
@ghost ghost closed this as completed in fe843ca Dec 20, 2021
Zeebe automation moved this from Review in progress to Done Dec 20, 2021
ghost pushed a commit that referenced this issue Dec 20, 2021
8435: [Backport stable/1.2] Ensure leader always commits when replicated to quorum r=deepthidevaki a=github-actions[bot]

# Description
Backport of #8357 to `stable/1.2`.

closes #8324

Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com>
ghost pushed a commit that referenced this issue Dec 20, 2021
8411: Release 1.1.8 r=npepinpe a=npepinpe

## Description

This PR includes improvements/fixes to the release pipeline, as well as the release commits preparing for the next patch release. The `.gocompat.json` file is expected to be modified since the ordering changes, but that's a unfortunate side effect of the tool. It's still the same, and is checked by the pipeline (both release and test) beforehand.



8434: [Backport stable/1.1] Ensure leader always commits when replicated to quorum r=deepthidevaki a=github-actions[bot]

# Description
Backport of #8357 to `stable/1.1`.

closes #8324

Co-authored-by: Nicolas Pepin-Perreault <nicolas.pepin-perreault@camunda.com>
Co-authored-by: camunda-jenkins <ci@camunda.com>
Co-authored-by: Deepthi Devaki Akkoorath <deepthidevaki@gmail.com>
@korthout korthout added the version:1.3.0 Marks an issue as being completely or in parts released in 1.3.0 label Jan 4, 2022
@KerstinHebel KerstinHebel removed this from Done in Zeebe Mar 23, 2022
This issue was closed.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
kind/bug Categorizes an issue or PR as a bug severity/mid Marks a bug as having a noticeable impact but with a known workaround support Marks an issue as related to a customer support request version:1.3.0 Marks an issue as being completely or in parts released in 1.3.0
Projects
None yet
Development

Successfully merging a pull request may close this issue.

4 participants