HDDS-4063. Fix InstallSnapshot in OM HA (#1294)

(cherry picked from commit cc5901fa2bc7d543e9220b0249aa8310eae3aa8c)
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
index 2a98db6..d6d2be6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerRatisServer.java
@@ -407,6 +407,7 @@
         StorageUnit.BYTES);
     RaftServerConfigKeys.Log.setSegmentSizeMax(properties,
         SizeInBytes.valueOf(raftSegmentSize));
+    RaftServerConfigKeys.Log.setPurgeUptoSnapshotIndex(properties, true);
 
     // Set RAFT segment pre-allocated size
     final int raftSegmentPreallocatedSize = (int) conf.getStorageSize(
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
index 3f7429a..fd70315 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
@@ -363,21 +363,11 @@
   public CompletableFuture<TermIndex> notifyInstallSnapshotFromLeader(
       RaftProtos.RoleInfoProto roleInfoProto, TermIndex firstTermIndexInLog) {
 
-    String leaderNodeId = RaftPeerId.valueOf(roleInfoProto.getSelf().getId())
-        .toString();
-
-    LOG.info("Received install snapshot notificaiton form OM leader: {} with " +
+    String leaderNodeId = RaftPeerId.valueOf(roleInfoProto.getFollowerInfo()
+        .getLeaderInfo().getId().getId()).toString();
+    LOG.info("Received install snapshot notification from OM leader: {} with " +
             "term index: {}", leaderNodeId, firstTermIndexInLog);
 
-    if (!roleInfoProto.getRole().equals(RaftProtos.RaftPeerRole.LEADER)) {
-      // A non-leader Ratis server should not send this notification.
-      LOG.error("Received Install Snapshot notification from non-leader OM " +
-          "node: {}. Ignoring the notification.", leaderNodeId);
-      return completeExceptionally(new OMException("Received notification to " +
-          "install snaphost from non-leader OM node",
-          OMException.ResultCodes.RATIS_ERROR));
-    }
-
     CompletableFuture<TermIndex> future = CompletableFuture.supplyAsync(
         () -> ozoneManager.installSnapshotFromLeader(leaderNodeId),
         installSnapshotExecutor);