You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by GitBox <gi...@apache.org> on 2022/03/29 13:04:51 UTC

[GitHub] [ratis] szetszwo commented on a change in pull request #573: RATIS-1481. make state upgradate in notifyStateMachineToInstallSnapshot serialized

szetszwo commented on a change in pull request #573:
URL: https://github.com/apache/ratis/pull/573#discussion_r837442203



##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
##########
@@ -289,24 +284,34 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot(
         }
       }
 
+      final long inProgressInstallSnapshotIndexValue = getInProgressInstallSnapshotIndex();
+      Preconditions.assertTrue(
+          inProgressInstallSnapshotIndexValue <= firstAvailableLogIndex && inProgressInstallSnapshotIndexValue > 0,
+          "inProgressInstallSnapshotRequest: %s is not eligible, firstAvailableLogIndex: %s",
+          getInProgressInstallSnapshotIndex(), firstAvailableLogIndex);
+
       // If the snapshot is null or unavailable, return SNAPSHOT_UNAVAILABLE.
       if (isSnapshotNull.compareAndSet(true, false)) {
         LOG.info("{}: InstallSnapshot notification result: {}", getMemberId(),
             InstallSnapshotResult.SNAPSHOT_UNAVAILABLE);
-        inProgressInstallSnapshotIndex.compareAndSet(firstAvailableLogIndex, 0);
+        inProgressInstallSnapshotIndex.set(0);
         return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(),
             currentTerm, InstallSnapshotResult.SNAPSHOT_UNAVAILABLE, -1);
       }
 
       // If a snapshot has been installed, return SNAPSHOT_INSTALLED with the installed snapshot index and reset
-      // installedSnapshotIndex to 0.
-      long latestInstalledSnapshotIndex = this.installedSnapshotIndex.getAndSet(0);
-      if (latestInstalledSnapshotIndex > 0) {
+      // installedSnapshotIndex to (0,0).
+      final TermIndex latestInstalledSnapshotTermIndex = this.installedSnapshotTermIndex.get();

Review comment:
       We should use getAndSet or getAndUpdate.  Otherwise, the operation is not atomic.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
##########
@@ -235,50 +236,44 @@ private InstallSnapshotReplyProto notifyStateMachineToInstallSnapshot(
 
         // For the cases where RaftConf is empty on newly started peer with empty peer list,
         // we retrieve leader info from installSnapShotRequestProto.
-        final RoleInfoProto proto = leaderProto == null || server.getRaftConf().getPeer(state.getLeaderId()) != null?
-            server.getRoleInfoProto(): getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto));
+        final RoleInfoProto roleInfoProto = leaderProto == null || server.getRaftConf().getPeer(state.
+            getLeaderId()) != null? server.getRoleInfoProto(): getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto));

Review comment:
       Please revert this.  It only has name change and format change.

##########
File path: ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
##########
@@ -119,13 +119,14 @@ private InstallSnapshotReplyProto installSnapshotImpl(InstallSnapshotRequestProt
     if (reply != null) {
       if (request.hasLastRaftConfigurationLogEntryProto()) {
         // Set the configuration included in the snapshot
-        final LogEntryProto proto = request.getLastRaftConfigurationLogEntryProto();
-        LOG.info("{}: set new configuration {} from snapshot", getMemberId(), proto);
-
-        state.setRaftConf(proto);
-        state.writeRaftConfiguration(proto);
-        server.getStateMachine().event().notifyConfigurationChanged(
-            proto.getTerm(), proto.getIndex(), proto.getConfigurationEntry());
+        final LogEntryProto newConfLogEntryProto = request.getLastRaftConfigurationLogEntryProto();

Review comment:
       Please keep using "proto" as the name so that we can keep the lines short.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@ratis.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org