[ https://issues.apache.org/jira/browse/RATIS-498?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16800680#comment-16800680 ]
Tsz Wo Nicholas Sze commented on RATIS-498: ------------------------------------------- In notifyStateMachineToInstallSnapshot, we should # Check if latest snapshot is already ahead of the requested index. # Check if another request is in progress. # Process the request. #* Also, since we need to reset inProgressInstallSnapshotRequest, it is better to use whenComplete as in your earlier patch. {code} private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot( InstallSnapshotRequestProto request, RaftPeerId leaderId) throws IOException { ... updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION); // Check if latest snapshot is already ahead of the requested index. final TermIndex latestSnapshot = state.getLatestInstalledSnapshot(); if (latestSnapshot != null && latestSnapshot.getIndex() >= firstAvailableLogIndex - 1) { // State Machine has already installed the snapshot. Return the latest snapshot index to the Leader. final InstallSnapshotReplyProto reply = ServerProtoUtils.toInstallSnapshotReplyProto( leaderId, getId(), groupId, currentTerm, InstallSnapshotResult.INSTALLED, latestSnapshot.getIndex()); LOG.info("{}: StateMachine latest installed snapshot index: {}. Reply: {}", getId(), latestSnapshot.getIndex(), reply); return reply; } // Check if another request is in progress if (!inProgressInstallSnapshotRequest.compareAndSet(null, firstAvailableLogTermIndex)) { return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(), groupId, currentTerm, InstallSnapshotResult.ERROR, -1); } // This is the first installSnapshot notify request for this term and // index. Notify the state machine to install the snapshot. LOG.debug("{}: notifying state machine to install snapshot. Next log " + "index is {} but the leader's first available log index is {}.", getId(), state.getLog().getNextIndex(), firstAvailableLogIndex); stateMachine.notifyInstallSnapshotFromLeader(firstAvailableLogTermIndex) .whenComplete((reply, exception) -> { if (!inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogTermIndex, null)) { LOG.error("{}: TermIndex mismatched: firstAvailableLogTermIndex = {} but inProgressInstallSnapshotRequest = {}", getId(), firstAvailableLogTermIndex, inProgressInstallSnapshotRequest); return; } if (reply != null) { synchronized (this) { stateMachine.pause(); state.reloadStateMachine(reply.getIndex(), leaderTerm); state.updateInstalledSnapshotIndex(reply); } } else { // TODO: how to deal with the failure? LOG.error(getId() + ": State Machine failed to install snapshot", exception); } }); LOG.debug("{}: StateMachine snapshot installation is in progress. " + "InProgress Request: {}", getId(), inProgressInstallSnapshotRequest.get()); return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(), groupId, currentTerm, InstallSnapshotResult.IN_PROGRESS, -1); } } {code} > 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 > > > 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)