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

Hanisha Koneru commented on RATIS-498:
--------------------------------------

Thanks for the review [~szetszwo].

I think we should first check if a request is in progress and then check if the 
latest snapshot is already ahead of the requested index. 
Otherwise, the leader might have an inconsistent state of the follower. Let's 
say we have a scenario - 3 nodes - N1 (leader), N2 and N3.
 # N3 has a snapshot upto index 60 and no logs after that.
 # N3 gets installSnapshot notification from N1 with firstAvailableLogIndex = 
100.
 # N3 notifies SM to install snapshot. SM is installing snapshot.
 # Leader election happens and N2 becomes the new leader.
 # N2 sends install snapshot notification to N3 with firstAvailableLogIndex = 
50.
 # Since N2 already has snapshot upto index 60, it sends "ALREADY_INSTALLED" 
response back to N2.
 # N2 updates the snapshot index of N3 and proceeds with appendEntries
 # The initial install snapshot request on N3 completes and changes the state 
of N3 but the new leader N2 is not notified.

> Notify Follower to Install Snapshot through state machine
> ---------------------------------------------------------
>
>                 Key: RATIS-498
>                 URL: https://issues.apache.org/jira/browse/RATIS-498
>             Project: Ratis
>          Issue Type: New Feature
>          Components: server
>            Reporter: Hanisha Koneru
>            Assignee: Hanisha Koneru
>            Priority: Major
>         Attachments: RATIS-498.000.patch, RATIS-498.001.patch, 
> RATIS-498.002.patch, RATIS-498.003.patch, RATIS-498.004.patch, 
> RATIS-498.005.patch
>
>
> When a lagging Follower wants to catch up with the Leader, and the Leader 
> only has logs with start index greater than the Followers's last log index, 
> then the leader sends an InstallSnapshotRequest to the the Follower. 
> The aim of this Jira is to allow State Machine to decouple snapshot 
> installation from the Ratis server. When Leader does not have the logs to get 
> the Follower up to speed, it should notify the Follower to install a snapshot 
> (if install snapshot through Log Appender is disabled). The Follower in turn 
> notifies its state machine that a snapshot is required to catch up with the 
> leader.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to