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

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

Thanks for the review [~szetszwo].
{quote}I suggest implementing only one of the rpc such as gRPC. We may add the 
other implementations later.
 Need to change GrpcServerProtocolService for followers to take the request.
{quote}
done

{quote} In the follower, it may take a long time to do the actual install 
snapshot work. So, the leader may wait for a long time. Consider the following 
questions:
How long should the leader wait?
Should the follower report progress from time to time so that leader won't 
timeout?
Should the leader send appendEntries during waiting?{quote}
I am thinking that on the follower, once the notification is received to 
install the snapshot, and the state machine is informed, we should immediately 
return a response to the Leader that the request has been received. The actual 
install snapshot work by the State Machine should not be blocking the response 
to Leader. We could have a separate thread running on the RaftServerImpl which 
keeps track of the snapshot installations in progress so that SM is not 
notified multiple times. 
In the meanwhile, till the snapshot installation is finished, the Leader will 
keep trying to send appendEntries (or notifyInstallSnapshot requests) but the 
follower will ignore them.

Please let me know if this approach sounds good to you. 


> 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
>
>
> 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