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

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

How about we modify the proto as below (please feel free to modify it)?
{code:java}
message InstallSnapshotRequestProto {
  message SnapshotChunkProto {
    string requestId = 1; // an identifier for chunked-requests.
    uint32 requestIndex = 2; // the index for this request chunk. Starts from 0.
    RaftConfigurationProto raftConfiguration = 3;
    uint64 leaderTerm = 4;
    TermIndexProto termIndex = 5;
    repeated FileChunkProto fileChunks = 6;
    uint64 totalSize = 7;
    bool done = 8; // whether this is the final chunk for the same req.
  }

  message NotificationProto {
    TermIndexProto firstAvailableTermIndex = 1; // first available log index to 
notify Follower to install snapshot
  }

  RaftRpcRequestProto serverRequest = 1;

  oneof InstallSnapshotRequestBody {
    SnapshotChunkProto snapshotChunk = 2;
    NotificationProto notification = 3;
  }
}
{code}
When a follower receives an InstallSnapshotRequestProto, if snapshot 
installation is disabled, it checks whether the proto has a notification. If 
snapshot installation is enabled, it checks whether the proto has a 
snapshotChunk.
{quote}And, if leader and follower have different settings for 
installSnapshotEnabled, I am thinking we could return a new 
InstallSnapshotResult result code (CONF_MISMATCH) to the leader. But there 
would not be a way to fix this error, right? Should the follower be shutdown in 
this case?
{quote}
Returning CONF_MISMATCH sounds good. I think the follower should not be 
shutdown since it could be the case that the leader is misconfigured. Let's 
just print a big warning for now.

BTW, since this sounds quite involved, how about we work on it in a separated 
JIRA?

> 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, RATIS-498.006.patch, RATIS-498.007.patch, 
> RATIS-498.008.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