You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Tsz-wo Sze (Jira)" <ji...@apache.org> on 2022/03/30 05:47:00 UTC

[jira] [Updated] (RATIS-1481) notifyStateMachineToInstallSnapshot stuck in IN_PROGRESS

     [ https://issues.apache.org/jira/browse/RATIS-1481?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Tsz-wo Sze updated RATIS-1481:
------------------------------
    Component/s: server

> notifyStateMachineToInstallSnapshot stuck in IN_PROGRESS
> --------------------------------------------------------
>
>                 Key: RATIS-1481
>                 URL: https://issues.apache.org/jira/browse/RATIS-1481
>             Project: Ratis
>          Issue Type: Bug
>          Components: server
>            Reporter: Xu Shao Hong
>            Assignee: Xu Shao Hong
>            Priority: Major
>          Time Spent: 6h 50m
>  Remaining Estimate: 0h
>
> The phenomenon shows that in the ozone cluster, OM fails to install the snapshot. From the OM log, OM state machine has done its part(eg. download Checkpoint, install, load).
> First,
> {code:java}
> stateMachine.followerEvent().notifyInstallSnapshotFromLeader(roleInfoProto, firstAvailableLogTermIndex).whenComplete(...) {code}
>  it is an async action of {*}CompletableFuture{*}. Normally, the follower should be able to receive the future Installsnapshot request and tell back once it has already installed snapshot. But I found that the leader will not send Installsnapshot requests anymore.
>  
> During whenComplete stage, these followings will be executed, which would update the snapshot index and commit index.
> {code:java}
> stateMachine.pause();
> state.updateInstalledSnapshotIndex(reply);
> state.reloadStateMachine(reply.getIndex());
> installedSnapshotIndex.set(reply.getIndex()); {code}
>  
>  
> In the process of appendEntriesAsync, checkInconsistentAppendEntries will return inconsistency as the snapshot is still in progress. Once the actual upgrade of snapshot index and commit index takes place, the leader receives the inconsistency with the new index and then won't send installsnapshot requests anymore as the check of shouldNotifyToInstallSnapshot() will be null.
>  
> Meanwhile, due to the async action of CompletableFuture, the follower raft server has not yet sent the SNAPSHOT_INSTALLED to leader according to the previous installsnapshot request and cannot receive future requests. This lead to an infinite loop of failed appendEntries and disappeared installshot progress



--
This message was sent by Atlassian Jira
(v8.20.1#820001)