You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/02/15 17:46:16 UTC

[GitHub] [ozone] bshashikant opened a new pull request #1921: HDDS-4822. Implement scm --bootstrap command.

bshashikant opened a new pull request #1921:
URL: https://github.com/apache/ozone/pull/1921


   ## What changes were proposed in this pull request?
   Implemented SCM --bootstrap command.
   
   During SCM --bootstrap, the bootstrapping SCM node will connect to primary SCM node (already running) and get the cluster Id. Once security is implemented, it will also fetch the CSR root certificates from primary SCM during SCM --bootstrap phase. 
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-4822
   
   ## How was this patch tested?
   Added basic Unit tests. Will add more tests once SCM HA config details are finalized.
   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GlenGeng commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
GlenGeng commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r580986666



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {

Review comment:
       Thanks @bharatviswa504 for the clarification.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578954612



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "
+                + "is {}", persistedClusterId, fetchedId);
+        return false;
+      }
+    } else {
+      try {
+        scmStorageConfig.setClusterId(fetchedId);
+        // It will write down the cluster Id fetched from already
+        // running SCM as well as the local SCM Id.
+
+        // SCM Node info containing hostname to scm Id mappings
+        // will be persisted into the version file once this node gets added
+        // to existing SCM ring post node regular start up.

Review comment:
       Question: Now we have scmId and scmNodeInfo with scmId and host mapping.
   Do we plan to remove scmId for the newer version or how it will be handled for upgrade?
   Will these be handled in new Jira?
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant merged pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant merged pull request #1921:
URL: https://github.com/apache/ozone/pull/1921


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578954612



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "
+                + "is {}", persistedClusterId, fetchedId);
+        return false;
+      }
+    } else {
+      try {
+        scmStorageConfig.setClusterId(fetchedId);
+        // It will write down the cluster Id fetched from already
+        // running SCM as well as the local SCM Id.
+
+        // SCM Node info containing hostname to scm Id mappings
+        // will be persisted into the version file once this node gets added
+        // to existing SCM ring post node regular start up.

Review comment:
       Question: Now we have scmId and scmNodeInfo with scmId and host mapping.
   Do we plan to remove scmId for the newer version or how it will be handled for upgrade?
   
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578972956



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "
+                + "is {}", persistedClusterId, fetchedId);
+        return false;
+      }
+    } else {
+      try {
+        scmStorageConfig.setClusterId(fetchedId);
+        // It will write down the cluster Id fetched from already
+        // running SCM as well as the local SCM Id.
+
+        // SCM Node info containing hostname to scm Id mappings
+        // will be persisted into the version file once this node gets added
+        // to existing SCM ring post node regular start up.

Review comment:
       SCMID will be exchanged during start up. Its not required during Bootstrap.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] GlenGeng commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
GlenGeng commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r580975651



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {

Review comment:
       `if (state == StorageState.INITIALIZED) {` ?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#issuecomment-784005820


   Thanks @bharatviswa504 for thr review.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r580979938



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {

Review comment:
       @GlenGeng 
   Shashii pointed out this during offline discussion and it is being addressed in HDDS-4718
   https://github.com/apache/ozone/pull/1953/files#diff-90e25a3409b1f5c4234a04a51db63e702b217aebe51e15e08525c76b2c6137d6R720




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r580979938



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {

Review comment:
       @GlenGeng 
   Thanks for pointing this.
   @bshashikant  pointed out this during the offline discussion and it is being addressed in HDDS-4718
   https://github.com/apache/ozone/pull/1953/files#diff-90e25a3409b1f5c4234a04a51db63e702b217aebe51e15e08525c76b2c6137d6R720




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578974029



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "
+                + "is {}", persistedClusterId, fetchedId);
+        return false;
+      }
+    } else {
+      try {
+        scmStorageConfig.setClusterId(fetchedId);
+        // It will write down the cluster Id fetched from already
+        // running SCM as well as the local SCM Id.
+
+        // SCM Node info containing hostname to scm Id mappings
+        // will be persisted into the version file once this node gets added
+        // to existing SCM ring post node regular start up.

Review comment:
       > Question: Now we have scmId and scmNodeInfo with scmId and host mapping.
   > Do we plan to remove scmId for the newer version or how it will be handled for upgrade?
   > Will these be handled in new Jira?
   
   We will remove it , while upgrading It needs to write down the SCMID:NODE Mappings for existing clusters. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r580979938



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {

Review comment:
       @GlenGeng 
   Thanks for pointing this.
   @bshashikant  pointed out this during the offline discussion and it is being addressed in HDDS-4718
   https://github.com/apache/ozone/pull/1953/files#diff-90e25a3409b1f5c4234a04a51db63e702b217aebe51e15e08525c76b2c6137d6R720
   
   To make forward progress, we committed this to the branch.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578974976



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);

Review comment:
       Yes, it will work. I did not find anyway to do that in failOverProxy provider.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578974446



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java
##########
@@ -112,6 +112,25 @@ public void initScm(@CommandLine.Option(names = { "--clusterid" },
     }
   }
 
+  /**
+   * This function implements a sub-command to allow the SCM to be
+   * initialized from the command line.
+   */
+  @CommandLine.Command(name = "--bootstrap",
+      customSynopsis = "ozone scm [global options] --bootstrap",

Review comment:
        [global options]  are hadoop generic options . I guess, maintained for legacy same as in scm --init.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #1921: HDDS-4822. Implement scm --bootstrap command.

Posted by GitBox <gi...@apache.org>.
bharatviswa504 commented on a change in pull request #1921:
URL: https://github.com/apache/ozone/pull/1921#discussion_r578956008



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java
##########
@@ -112,6 +112,25 @@ public void initScm(@CommandLine.Option(names = { "--clusterid" },
     }
   }
 
+  /**
+   * This function implements a sub-command to allow the SCM to be
+   * initialized from the command line.
+   */
+  @CommandLine.Command(name = "--bootstrap",
+      customSynopsis = "ozone scm [global options] --bootstrap",

Review comment:
       Question: [global options] what are these global options?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);

Review comment:
       If we remove the localSCMNodeId and construct failOverProxy provider we should be good here right?
   
   

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "

Review comment:
       Can we reword this
   but clusterId of SCM is not matching with leader SCM clusterId. 
   
   (Generally, this might be due to change in config ?)

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -681,6 +688,60 @@ public static String buildRpcServerStartMessage(String description,
     return rpcServer;
   }
 
+  /**
+   * Routine to bootstrap the StorageContainerManager. Thsi will connect to a
+   * running SCM instance which has valid cluster id and fetch the cluster id
+   * from there. SCM ids will be also be exchanged here.
+   *
+   * TODO: once SCM HA security is enabled, CSR cerificates will be fetched from
+   * running scm leader instance as well.
+   *
+   * @param conf OzoneConfiguration
+   * @return true if SCM bootstrap is successful, false otherwise.
+   * @throws IOException if init fails due to I/O error
+   */
+  public static boolean scmBootstrap(OzoneConfiguration conf)
+      throws IOException {
+    if (!SCMHAUtils.isSCMHAEnabled(conf)) {
+      LOG.error("Bootstrap is not supported without SCM HA.");
+      return false;
+    }
+    // The node here will try to fetch the cluster id from any of existing
+    // running SCM instances.
+    // TODO: need to avoid failover to local SCM Node here
+    final ScmInfo scmInfo = HAUtils.getScmInfo(conf);
+    SCMStorageConfig scmStorageConfig = new SCMStorageConfig(conf);
+    final String persistedClusterId = scmStorageConfig.getClusterID();
+    final String fetchedId = scmInfo.getClusterId();
+    Preconditions.checkNotNull(fetchedId);
+    StorageState state = scmStorageConfig.getState();
+    if (state != StorageState.INITIALIZED) {
+      Preconditions.checkNotNull(scmStorageConfig.getScmId());
+      if (fetchedId != persistedClusterId) {
+        LOG.error(
+            "Could not bootstrap as SCM is already initialized with cluster "
+                + "id {} but cluster id for existing running SCM instance "
+                + "is {}", persistedClusterId, fetchedId);
+        return false;
+      }
+    } else {
+      try {
+        scmStorageConfig.setClusterId(fetchedId);
+        // It will write down the cluster Id fetched from already
+        // running SCM as well as the local SCM Id.
+
+        // SCM Node info containing hostname to scm Id mappings
+        // will be persisted into the version file once this node gets added
+        // to existing SCM ring post node regular start up.

Review comment:
       SCM ID exchange and persist is missing?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org