You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Jing Zhao (JIRA)" <ji...@apache.org> on 2017/12/11 03:12:00 UTC

[jira] [Comment Edited] (RATIS-160) Retry cache should handle leader change after log commit

    [ https://issues.apache.org/jira/browse/RATIS-160?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16285467#comment-16285467 ] 

Jing Zhao edited comment on RATIS-160 at 12/11/17 3:11 AM:
-----------------------------------------------------------

The reason I do not like adding "entryInLog" is that its semantic meaning is unclear. Actually in the current patch this boolean field only captures whether the retry cache entry is in an in-memory log segment. It cannot tell us whether the entry has be persisted in the disk, or has been committed to the raft group. Note the corresponding log entry can still be omitted or even truncated. Therefore even if we record "entryInLog" as true, we actually cannot guarantee anything.

Do you think the proposed fix in my previous comment can fix the issue? If yes that should be a simple fix. In the meanwhile, we should add a unit test to reproduce the scenario you observed. But that may be complicated so we can do it in a separate jira.


was (Author: jingzhao):
The reason I do not like adding "entryInLog" is that its semantic meaning is unclear. Actually the current patch only captures whether the retry cache entry is in an in-memory log segment. It cannot tell us whether the entry has be persisted in the disk, or has been committed to the raft group. Note the corresponding log entry can still be omitted or even truncated. Therefore even if we record "entryInLog" as true, we actually cannot guarantee anything.

Do you think the proposed fix in my previous comment can fix the issue? If yes that should be a simple fix. In the meanwhile, we should add a unit test to reproduce the scenario you observed. But that may be complicated so we can do it in a separate jira.

> Retry cache should handle leader change after log commit
> --------------------------------------------------------
>
>                 Key: RATIS-160
>                 URL: https://issues.apache.org/jira/browse/RATIS-160
>             Project: Ratis
>          Issue Type: Bug
>            Reporter: Lokesh Jain
>            Assignee: Lokesh Jain
>         Attachments: RATIS-160.001.patch
>
>
> This jira is in relation to the below exception seen in the logs. 
> {code:java}
> java.lang.IllegalStateException: retry cache entry should be pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
>         at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>         at org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
>         at org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
>         at org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
>         at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
>         at java.lang.Thread.run(Thread.java:748)
> Exception in thread "StateMachineUpdater-s3" org.apache.ratis.util.ExitUtils$ExitException: StateMachineUpdater-s3: the StateMachineUpdater hits Throwable
>         at org.apache.ratis.util.ExitUtils.terminate(ExitUtils.java:94)
>         at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:175)
>         at java.lang.Thread.run(Thread.java:748)
> Caused by: java.lang.IllegalStateException: retry cache entry should be pending: client-89341C13-2136-4EF3-BD8A-73C2526B7703:1777:done
>         at org.apache.ratis.util.Preconditions.assertTrue(Preconditions.java:60)
>         at org.apache.ratis.server.impl.RetryCache.getOrCreateEntry(RetryCache.java:169)
>         at org.apache.ratis.server.impl.RaftServerImpl.replyPendingRequest(RaftServerImpl.java:915)
>         at org.apache.ratis.server.impl.RaftServerImpl.applyLogToStateMachine(RaftServerImpl.java:974)
>         at org.apache.ratis.server.impl.StateMachineUpdater.run(StateMachineUpdater.java:151)
>         ... 1 more
> {code}
> This occurs when leader commits a log entry but is not able to send a reply to the client before leader is changed. When the new leader gets the request it sends the append entry request to the followers whose cache already has the  corresponding entry leading to the above exception.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)