bshashikant commented on a change in pull request #1953:
URL: https://github.com/apache/ozone/pull/1953#discussion_r583563075



##########
File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/SCMRatisServerImpl.java
##########
@@ -63,69 +61,120 @@
   private static final Logger LOG =
       LoggerFactory.getLogger(SCMRatisServerImpl.class);
 
-  private final RaftServer.Division division;
+  private final RaftServer server;
+  private final SCMStateMachine stateMachine;
   private final StorageContainerManager scm;
-  private final InetSocketAddress address;
   private final ClientId clientId = ClientId.randomId();
   private final AtomicLong callId = new AtomicLong();
 
   // TODO: Refactor and remove ConfigurationSource and use only
   //  SCMHAConfiguration.
-  SCMRatisServerImpl(final SCMHAConfiguration haConf,
-      final ConfigurationSource conf, final StorageContainerManager scm,
-      final DBTransactionBuffer buffer) throws IOException {
+  SCMRatisServerImpl(final ConfigurationSource conf,
+      final StorageContainerManager scm, final DBTransactionBuffer buffer)
+      throws IOException {
     this.scm = scm;
-    this.address = haConf.getRatisBindAddress();
-    RaftServer server = newRaftServer(scm.getClusterId(), scm.getScmId(),
-        scm.getSCMHANodeDetails(), conf)
-        .setStateMachine(new SCMStateMachine(scm, this, buffer)).build();
+    this.stateMachine = new SCMStateMachine(scm, this, buffer);
+    final RaftGroupId groupId = buildRaftGroupId(scm.getClusterId());
+    LOG.info("starting Raft server for scm:{}", scm.getScmId());
+    // During SCM startup, the bootstrapped node will be started just with
+    // groupId information, so that it won't trigger any leader election
+    // as it doesn't have any peer info.
+
+    // The primary SCM node which is initialized using scm --init command,
+    // will initialize the raft server with the peer info and it will be
+    // persisted in the raft log post leader election. Now, when the primary
+    // scm boots up, it has peer info embedded in the raft log and will
+    // trigger leader election.
+    this.server =
+        newRaftServer(scm.getScmId(), conf).setStateMachine(stateMachine)
+            .setGroup(RaftGroup.valueOf(groupId)).build();
+  }
+
+  public static void initialize(String clusterId, String scmId,
+      SCMNodeDetails details, OzoneConfiguration conf) throws IOException {
+    final RaftGroup group = buildRaftGroup(details, scmId, clusterId);
+    RaftServer server = newRaftServer(scmId, conf)
+        .setGroup(group).build();
+    server.start();
+    waitforLeaderToBeReady(server, 60000, group);
+    server.close();
+  }
+
+  private static void waitforLeaderToBeReady(RaftServer server, long timeout,
+      RaftGroup group)
+      throws IOException {
+    boolean ready;
+    long st = Time.monotonicNow();
+    do {
+      ready = server.getDivision(group.getGroupId()).getInfo().isLeaderReady();
+      if (!ready) {
+        try {
+          Thread.sleep(2000);
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+        }
+      }
+    } while (!ready && Time.monotonicNow() - st < timeout);
 
-    this.division =
-        server.getDivision(server.getGroups().iterator().next().getGroupId());
+    if (!ready) {
+      throw new IOException(String
+          .format("Ratis group %s is not ready in %d ms", group.getGroupId(),
+              timeout));
+    }
   }
 
-  public static RaftServer.Builder newRaftServer(final String clusterId,
-      final String scmId, final SCMHANodeDetails haDetails,
-      final ConfigurationSource conf) throws IOException {
-    final String scmNodeId = haDetails.getLocalNodeDetails().getNodeId();
+  private static RaftServer.Builder newRaftServer(final String scmId,
+      final ConfigurationSource conf) {
     final SCMHAConfiguration haConf = conf.getObject(SCMHAConfiguration.class);
-    SCMHAGroupBuilder haGrpBuilder = scmNodeId != null ?
-        new SCMHAGroupBuilder(haDetails, clusterId, scmId) :
-        new SCMHAGroupBuilder(haConf, conf, clusterId);
     final RaftProperties serverProperties =
         RatisUtil.newRaftProperties(haConf, conf);
-    return RaftServer.newBuilder().setServerId(haGrpBuilder.getPeerId())
-        .setGroup(haGrpBuilder.getRaftGroup()).setProperties(serverProperties)
-        .setStateMachine(new SCMStateMachine());
+    return RaftServer.newBuilder().setServerId(RaftPeerId.getRaftPeerId(scmId))
+        .setProperties(serverProperties)
+        .setStateMachine(new SCMStateMachine(false));
   }
 
   @Override
   public void start() throws IOException {
-    division.getRaftServer().start();
+    server.start();
+  }
+
+  public RaftServer.Division getDivision() {
+    try {
+      return server
+          .getDivision(server.getGroups().iterator().next().getGroupId());
+    } catch (Exception e) {
+      LOG.warn("Failed to get RaftServerDivision", e);
+      return null;
+    }
+  }
+
+  @VisibleForTesting
+  public SCMStateMachine getStateMachine() {
+    return stateMachine;
   }
 
   @Override
   public void registerStateMachineHandler(final RequestType handlerType,
                                           final Object handler) {
-    ((SCMStateMachine) division.getStateMachine())
-        .registerHandler(handlerType, handler);
+    stateMachine.registerHandler(handlerType, handler);
   }
 
   @Override
   public SCMRatisResponse submitRequest(SCMRatisRequest request)
       throws IOException, ExecutionException, InterruptedException {
     final RaftClientRequest raftClientRequest = RaftClientRequest.newBuilder()
         .setClientId(clientId)
-        .setServerId(division.getId())
-        .setGroupId(division.getGroup().getGroupId())
+        .setServerId(getDivision().getId())

Review comment:
       Division info can change depending upon server state. Its safe to use 
this via Ratis. Its a cached object away inside ratisServerImpl.




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to