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

Tsz Wo Nicholas Sze commented on RATIS-498:
-------------------------------------------

Thanks for the update.  Some more comments:

In RaftServerImpl.notifyStateMachineToInstallSnapshot
- change isSMSnapshotInstallationInProgress to 
AtomicReference<InstallSnapshotRequestProto> so that it can record the reqeust. 
 When there is another reqeust, we may compare if they are the same reqeust.
- We should use compareAndSet for isSMSnapshotInstallationInProgress.  
Otherwise, it is not atomic.
{code}
        // Check if there is already a snapshot installation in progress.
        if (isSMSnapshotInstallationInProgress.compareAndSet(false, true)) {
{code}
- the notifyStateMachineThread is not needed.  The state machine should take 
care thread management for the call.  The code should look like:
{code}
        
stateMachine.notifyInstallSnapshotFromLeader(firstAvailableLogIndex).thenAccept(reply
 -> {
          synchronized (this) {
            state.reloadStateMachine(reply, leaderTerm);
          }
        }).exceptionally(exception -> {
          // TODO: how to deal with the failure?
          LOG.error(getId() + ": State Machine failed to install snapshot", 
exception);
          return null;
        });
{code}

Other comments:
- In the current design, INSTALL_SNAPSHOT_NOTIFICATION is a short request so 
that it does not need START and COMPLETE in FollowerState.UpdateType.  Just use 
INSTALL_SNAPSHOT_NOTIFICATION(AtomicInteger::get); see REQUEST_VOTE.
- Use getFollowerId() in LogAppender.createInstallSnapshotNotificationRequest.
- Please add some tests.

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