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/03/15 05:39:06 UTC

[GitHub] [ozone] bharatviswa504 opened a new pull request #2041: HDDS-4897. Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

bharatviswa504 opened a new pull request #2041:
URL: https://github.com/apache/ozone/pull/2041


   ## What changes were proposed in this pull request?
   This PR implements
   1. Create SCM CertClient, which generates a public key, private key, and generates CSR with ClusterID, SCMID. 
   2. Modify DefaultCA Server to work in 2 modes, SELF_SIGNED_CA and INTERMEDIARY_CA.
   3. Modify SCMStorageConfig to persist SCM cert serial ID.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4897
   
   ## How was this patch tested?
   
   Added tests
   


----------------------------------------------------------------
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -668,7 +672,9 @@ private CertificateServer initializeCertificateServer(String clusterID,
             .setRatisServer(scmHAManager.getRatisServer())
             .setCRLSequenceId(getLastSequenceIdForCRL()).build();
 
-    return new DefaultCAServer(subject, clusterID, scmID, certStore);
+    return new DefaultCAServer(subject, clusterID, scmID, certStore,
+        new DefaultProfile(),

Review comment:
       When this PR code is integrated, when 2 CA Servers are started will change this. It will come in next in PR. 




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       {metadatadir}/scm/sub-ca/certs/certificate.crt
   {metadatadir}/scm/sub-ca/certs/CA-``<<ca-certid>>``.crt
   {metadatadir}/scm/sub-ca/certs/``<<cert-d>>``.crt
   
   To be simple, I have followed this approach. Certificat.crt will be used by sub CA, and cert client uses first 2, and also during load if same cert is loaded, it is already taken care by certclient, so we are good over there.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getSCMSignedCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getSCMSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSignedCert(CertificateClient client,

Review comment:
       I see it is completely different. As DefaultCAServer uses its own SelfSignedCertificate builder and persist, whereas this new method uses generate CSR and persist.  
   
   Where as taken care of dedup between getRootCASignedSCMCert and this method.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       >Also, when Primary starts two DefaultCA server in the follow up JIRA, one with RootCAProfile and one without?
   Yes.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMStorageConfig.java
##########
@@ -77,4 +78,20 @@ protected Properties getNodeProperties() {
     return scmProperties;
   }
 
+  /**
+   * Sets the SCM CA certificate serial id.

Review comment:
       Done




-- 
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 merged pull request #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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


   


-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -668,7 +672,9 @@ private CertificateServer initializeCertificateServer(String clusterID,
             .setRatisServer(scmHAManager.getRatisServer())
             .setCRLSequenceId(getLastSequenceIdForCRL()).build();
 
-    return new DefaultCAServer(subject, clusterID, scmID, certStore);
+    return new DefaultCAServer(subject, clusterID, scmID, certStore,
+        new DefaultProfile(),

Review comment:
       When this PR code is integrated, when 2 CA Servers are started will change this. It will come in next in PR.  As to make current tests pass, used DefaultProfile

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -668,7 +672,9 @@ private CertificateServer initializeCertificateServer(String clusterID,
             .setRatisServer(scmHAManager.getRatisServer())
             .setCRLSequenceId(getLastSequenceIdForCRL()).build();
 
-    return new DefaultCAServer(subject, clusterID, scmID, certStore);
+    return new DefaultCAServer(subject, clusterID, scmID, certStore,
+        new DefaultProfile(),

Review comment:
       When this PR code is integrated, when 2 CA Servers are started will change this. It will come in next in PR.  As to make current code work, used DefaultProfile




-- 
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] xiaoyuyao commented on a change in pull request #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    } catch (IOException | CertificateException  e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * This function creates/initializes a certificate server as needed.
+   * This function is idempotent, so calling this again and again after the
+   * server is initialized is not a problem.
+   *
+   * @param clusterID - Cluster ID
+   * @param scmID     - SCM ID
+   */
+  public static CertificateServer initializeRootCertificateServer(
+      String clusterID, String scmID, SCMCertStore scmCertStore)
+      throws IOException {
+    String subject = "scm-rootca@" + InetAddress.getLocalHost().getHostName();

Review comment:
       Can we keep the root ca subject scm@host to be backward compatible?

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
##########
@@ -442,19 +434,28 @@ private boolean checkIfCertificatesExist() {
       };
       break;
     case INITIALIZE:
-      consumer = (arg) -> {
-        try {
-          generateSelfSignedCA(arg);
-        } catch (NoSuchProviderException | NoSuchAlgorithmException
-            | IOException e) {
-          LOG.error("Unable to initialize CertificateServer.", e);
-        }
-        VerificationStatus newStatus = verifySelfSignedCA(arg);
-        if (newStatus != VerificationStatus.SUCCESS) {
-          LOG.error("Unable to initialize CertificateServer, failed in " +
-              "verification.");
-        }
-      };
+      if (type == CAType.SELF_SIGNED_CA) {
+        consumer = (arg) -> {
+          try {
+            generateSelfSignedCA(arg);
+          } catch (NoSuchProviderException | NoSuchAlgorithmException
+              | IOException e) {
+            LOG.error("Unable to initialize CertificateServer.", e);
+          }
+          VerificationStatus newStatus = verifySelfSignedCA(arg);
+          if (newStatus != VerificationStatus.SUCCESS) {
+            LOG.error("Unable to initialize CertificateServer, failed in " +
+                "verification.");
+          }
+        };
+      } else if (type == CAType.INTERMEDIARY_CA) {
+        // for this certificates are generated during bootstrap/start. If
+        // both certs are missing, something is wrong during bootstrap/start.
+        consumer = (arg) -> {

Review comment:
       Can we add some error log here?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    } catch (IOException | CertificateException  e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * This function creates/initializes a certificate server as needed.
+   * This function is idempotent, so calling this again and again after the
+   * server is initialized is not a problem.
+   *
+   * @param clusterID - Cluster ID
+   * @param scmID     - SCM ID
+   */
+  public static CertificateServer initializeRootCertificateServer(
+      String clusterID, String scmID, SCMCertStore scmCertStore)
+      throws IOException {
+    String subject = "scm-rootca@" + InetAddress.getLocalHost().getHostName();
+
+    return new DefaultCAServer(subject, clusterID, scmID, scmCertStore,
+        new DefaultCAProfile(),
+        Paths.get(SCM_CA_CERT_STORAGE_DIR, SCM_CA_PATH).toString());
+  }
+
+  private static PKCS10CertificationRequest generateCSR(
+      CertificateClient client, SCMStorageConfig scmStorageConfig,
+      OzoneConfiguration config, InetSocketAddress scmAddress,
+      String fetchedSCMId) throws IOException {
+    CertificateSignRequest.Builder builder = client.getCSRBuilder();
+    KeyPair keyPair = new KeyPair(client.getPublicKey(),
+        client.getPrivateKey());
+
+    // Get host name.
+    String hostname = scmAddress.getAddress().getHostName();
+
+    String subject = "scm@"+ hostname;

Review comment:
       this sub scm cert should have subject like scm-sub@host.

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getSCMSignedCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getSCMSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSignedCert(CertificateClient client,

Review comment:
       This seems to overlap with most of the existing  DefaultCAServer#generateSelfSignedCA, can we dedup the code?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMStorageConfig.java
##########
@@ -77,4 +78,20 @@ protected Properties getNodeProperties() {
     return scmProperties;
   }
 
+  /**
+   * Sets the SCM CA certificate serial id.

Review comment:
       can you highlight this is for sub scm ca cert id. 

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,

Review comment:
       NIT: move generateCSR around line 173 so that if rootCA init failed, no need to generate CSR. 

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Paths;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+
+/**
+ * SCM Certificate Client which is used for generate public, private Key and

Review comment:
       Can you add some comments to highlight this is only used for Sub-CA?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);

Review comment:
       Can we combine the exception handling if the logic are the same?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
##########
@@ -668,7 +672,9 @@ private CertificateServer initializeCertificateServer(String clusterID,
             .setRatisServer(scmHAManager.getRatisServer())
             .setCRLSequenceId(getLastSequenceIdForCRL()).build();
 
-    return new DefaultCAServer(subject, clusterID, scmID, certStore);
+    return new DefaultCAServer(subject, clusterID, scmID, certStore,
+        new DefaultProfile(),

Review comment:
       should we start with the DefaultCAProfile?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));

Review comment:
       should we check the result from RPC response before attempting further operations?

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       Can you elaborate the logic behind this? Are we assume only the root CA start with the DefaultCAProfile?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       Given we are using the SCMCertificateClient for both cases, I think they will be persisted to the same location. 

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    } catch (IOException | CertificateException  e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * This function creates/initializes a certificate server as needed.
+   * This function is idempotent, so calling this again and again after the
+   * server is initialized is not a problem.
+   *
+   * @param clusterID - Cluster ID
+   * @param scmID     - SCM ID
+   */
+  public static CertificateServer initializeRootCertificateServer(
+      String clusterID, String scmID, SCMCertStore scmCertStore)
+      throws IOException {
+    String subject = "scm-rootca@" + InetAddress.getLocalHost().getHostName();

Review comment:
       The sub scm can have scm-sub@host

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       I think we should store the sub-scm cert to a different location.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       Yes.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       My idea here is DefaultCAProfile when used by CA Server it will issue CA Certificate.
   And RootCA Server right now starts with DefaultCAProfile on fresh installed clusters, where as on upgraded cluster from non-HA rootCAServer will be started like before with DefaultProfile. The integration of DefaultCAServer and starting 2 CA will be taken care in further Jiras.




-- 
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 pull request #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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


   Thank You @xiaoyuyao for the review. I have addressed/replied to the review comments.


-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    } catch (IOException | CertificateException  e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * This function creates/initializes a certificate server as needed.
+   * This function is idempotent, so calling this again and again after the
+   * server is initialized is not a problem.
+   *
+   * @param clusterID - Cluster ID
+   * @param scmID     - SCM ID
+   */
+  public static CertificateServer initializeRootCertificateServer(
+      String clusterID, String scmID, SCMCertStore scmCertStore)
+      throws IOException {
+    String subject = "scm-rootca@" + InetAddress.getLocalHost().getHostName();

Review comment:
       Done

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      Thread.currentThread().interrupt();
+      throw new RuntimeException(e);
+    } catch (ExecutionException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    } catch (IOException | CertificateException  e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+
+  }
+
+  /**
+   * This function creates/initializes a certificate server as needed.
+   * This function is idempotent, so calling this again and again after the
+   * server is initialized is not a problem.
+   *
+   * @param clusterID - Cluster ID
+   * @param scmID     - SCM ID
+   */
+  public static CertificateServer initializeRootCertificateServer(
+      String clusterID, String scmID, SCMCertStore scmCertStore)
+      throws IOException {
+    String subject = "scm-rootca@" + InetAddress.getLocalHost().getHostName();
+
+    return new DefaultCAServer(subject, clusterID, scmID, scmCertStore,
+        new DefaultCAProfile(),
+        Paths.get(SCM_CA_CERT_STORAGE_DIR, SCM_CA_PATH).toString());
+  }
+
+  private static PKCS10CertificationRequest generateCSR(
+      CertificateClient client, SCMStorageConfig scmStorageConfig,
+      OzoneConfiguration config, InetSocketAddress scmAddress,
+      String fetchedSCMId) throws IOException {
+    CertificateSignRequest.Builder builder = client.getCSRBuilder();
+    KeyPair keyPair = new KeyPair(client.getPublicKey(),
+        client.getPrivateKey());
+
+    // Get host name.
+    String hostname = scmAddress.getAddress().getHostName();
+
+    String subject = "scm@"+ hostname;

Review comment:
       Done




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/SCMCertificateClient.java
##########
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.file.Paths;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+
+/**
+ * SCM Certificate Client which is used for generate public, private Key and

Review comment:
       Done

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
##########
@@ -442,19 +434,28 @@ private boolean checkIfCertificatesExist() {
       };
       break;
     case INITIALIZE:
-      consumer = (arg) -> {
-        try {
-          generateSelfSignedCA(arg);
-        } catch (NoSuchProviderException | NoSuchAlgorithmException
-            | IOException e) {
-          LOG.error("Unable to initialize CertificateServer.", e);
-        }
-        VerificationStatus newStatus = verifySelfSignedCA(arg);
-        if (newStatus != VerificationStatus.SUCCESS) {
-          LOG.error("Unable to initialize CertificateServer, failed in " +
-              "verification.");
-        }
-      };
+      if (type == CAType.SELF_SIGNED_CA) {
+        consumer = (arg) -> {
+          try {
+            generateSelfSignedCA(arg);
+          } catch (NoSuchProviderException | NoSuchAlgorithmException
+              | IOException e) {
+            LOG.error("Unable to initialize CertificateServer.", e);
+          }
+          VerificationStatus newStatus = verifySelfSignedCA(arg);
+          if (newStatus != VerificationStatus.SUCCESS) {
+            LOG.error("Unable to initialize CertificateServer, failed in " +
+                "verification.");
+          }
+        };
+      } else if (type == CAType.INTERMEDIARY_CA) {
+        // for this certificates are generated during bootstrap/start. If
+        // both certs are missing, something is wrong during bootstrap/start.
+        consumer = (arg) -> {

Review comment:
       Done




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,

Review comment:
       Done




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);
+
+      // Write to the location, so that Default CA server can read the
+      // certificate generated by cert client.
+      CertificateCodec certCodec =
+          new CertificateCodec(new SecurityConfig(config),
+              client.getComponentName());
+
+      certCodec.writeCertificate(certificateHolder);
+
+      // Persist scm cert serial ID.
+      scmStorageConfig.setScmCertSerialId(certificateHolder.getSerialNumber()
+          .toString());
+    } catch (InterruptedException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);

Review comment:
       done




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));

Review comment:
       Already ClientSideTranslator has performed this.  So, check if it has a certificate should be enough.
   
   ```
     private SCMSecurityResponse handleError(SCMSecurityResponse resp)
         throws SCMSecurityException {
       if (resp.getStatus() != SCMSecurityProtocolProtos.Status.OK) {
         throw new SCMSecurityException(resp.getMessage(),
             SCMSecurityException.ErrorCode.values()[resp.getStatus().ordinal()]);
       }
       return resp;
     }
   ```




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       sub-ca
   {metadatadir}/scm/sub-ca/certs/certificate.crt
   {metadatadir}/scm/sub-ca/certs/CA-``<<ca-certid>>``.crt
   {metadatadir}/scm/sub-ca/certs/``<<cert-d>>``.crt
   {metadatadir}/scm/sub-ca/keys/
   
   root-ca
   {metadatadir}/scm/ca/certs/certificate.crt
   {metadatadir}/scm/ca/keys/
   To be simple, I have followed this approach. Certificat.crt will be used by sub CA, and cert client uses first 2, and also during load if same cert is loaded, it is already taken care by certclient, so we are good over there.




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       My idea here is DefaultCAProfile when used by CA Server it will issue only CA Certificate.
   And RootCA Server right now starts with DefaultCAProfile on fresh installed clusters, where as on upgraded cluster from non-HA rootCAServer will be started like before with DefaultProfile. The integration of DefaultCAServer and starting 2 CA will be taken care in further Jiras.




-- 
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 pull request #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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


   Thank You @xiaoyuyao for the 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] xiaoyuyao commented on a change in pull request #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/PKIProfiles/DefaultCAProfile.java
##########
@@ -43,4 +47,38 @@
       VALIDATE_NAME_CONSTRAINTS = (e, b) -> TRUE;
   static final BiFunction<Extension, PKIProfile, Boolean>
       VALIDATE_CRL_DISTRIBUTION_POINTS = (e, b) -> TRUE;
-}
+
+
+  private static boolean validateBasicExtensions(Extension ext,
+      PKIProfile pkiProfile) {
+    BasicConstraints constraints =
+        BasicConstraints.getInstance(ext.getParsedValue());
+    if(constraints.isCA()) {

Review comment:
       bq. My idea here is DefaultCAProfile when used by CA Server it will issue only CA Certificate.
   Should we rename to RootCAProfile in this case?
   
   Also, when Primary starts two DefaultCA server in the follow up JIRA, one with RootCAProfile and one without?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       Since the integration code is not there yet, I agree if you pass different client for sub-ca and root-ca, we are good here. 




-- 
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 #2041: HDDS-4897. [SCM HA Security] Create SCM Cert Client and change DefaultCA to allow selfsigned and intermediary

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ha/HASecurityUtils.java
##########
@@ -0,0 +1,266 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership.  The ASF
+ * licenses this file to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,WITHOUT
+ * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
+ * License for the specific language governing permissions and limitations under
+ * the License.
+ */
+package org.apache.hadoop.hdds.scm.ha;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmNodeDetailsProto;
+import org.apache.hadoop.hdds.protocol.proto.SCMSecurityProtocolProtos.SCMGetCertResponseProto;
+import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.server.SCMCertStore;
+import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCAServer;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.DefaultCAProfile;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import org.apache.hadoop.hdds.security.x509.certificate.client.SCMCertificateClient;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.pkcs.PKCS10CertificationRequest;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeType.SCM;
+import static org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateApprover.ApprovalType.KERBEROS_TRUSTED;
+import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_CERT_STORAGE_DIR;
+import static org.apache.hadoop.ozone.OzoneConsts.SCM_CA_PATH;
+
+public final class HASecurityUtils {
+
+  private HASecurityUtils() {
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(HASecurityUtils.class);
+
+  /**
+   * Initialize Security which generates public, private key pair and get SCM
+   * signed certificate and persist to local disk.
+   * @param scmStorageConfig
+   * @param fetchedScmId
+   * @param conf
+   * @param scmAddress
+   * @throws IOException
+   */
+  public static void initializeSecurity(SCMStorageConfig scmStorageConfig,
+      String fetchedScmId, OzoneConfiguration conf,
+      InetSocketAddress scmAddress, boolean primaryscm)
+      throws IOException {
+    LOG.info("Initializing secure StorageContainerManager.");
+
+    CertificateClient certClient =
+        new SCMCertificateClient(new SecurityConfig(conf));
+    InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful.");
+      break;
+    case GETCERT:
+      if (!primaryscm) {
+        getRootCASignedSCMCert(certClient, conf, fetchedScmId, scmStorageConfig,
+            scmAddress);
+      } else {
+        getPrimarySCMSelfSignedCert(certClient, conf, fetchedScmId,
+            scmStorageConfig, scmAddress);
+      }
+      LOG.info("Successfully stored SCM signed certificate.");
+      break;
+    case FAILURE:
+      LOG.error("SCM security initialization failed.");
+      throw new RuntimeException("OM security initialization failed.");
+    case RECOVER:
+      LOG.error("SCM security initialization failed. SCM certificate is " +
+          "missing.");
+      throw new RuntimeException("SCM security initialization failed.");
+    default:
+      LOG.error("SCM security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("SCM security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   */
+  private static void getRootCASignedSCMCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    ScmNodeDetailsProto scmNodeDetailsProto =
+        ScmNodeDetailsProto.newBuilder()
+            .setClusterId(scmStorageConfig.getClusterID())
+            .setHostName(scmAddress.getHostName())
+            .setScmNodeId(fetchedSCMId).build();
+
+    SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+        HddsServerUtil.getScmSecurityClient(config);
+
+    SCMGetCertResponseProto response = secureScmClient.
+        getSCMCertChain(scmNodeDetailsProto, getEncodedString(csr));
+    String pemEncodedCert = response.getX509Certificate();
+
+    try {
+
+      // Store SCM CA certificate.
+      if (response.hasX509CACertificate()) {
+        String pemEncodedRootCert = response.getX509CACertificate();
+        client.storeCertificate(pemEncodedRootCert, true, true);
+        client.storeCertificate(pemEncodedCert, true);
+
+
+        // Write to the location, so that Default CA server can read the
+        // certificate generated by cert client.
+        CertificateCodec certCodec =
+            new CertificateCodec(new SecurityConfig(config),
+                client.getComponentName());
+
+        X509Certificate certificate =
+            CertificateCodec.getX509Certificate(pemEncodedCert);
+        certCodec.writeCertificate(certCodec.getCertificateHolder(certificate));
+
+        // Persist scm cert serial ID.
+        scmStorageConfig.setScmCertSerialId(certificate.getSerialNumber()
+            .toString());
+      } else {
+        throw new RuntimeException("Unable to retrieve SCM certificate chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static void getPrimarySCMSelfSignedCert(CertificateClient client,
+      OzoneConfiguration config, String fetchedSCMId,
+      SCMStorageConfig scmStorageConfig, InetSocketAddress scmAddress)
+      throws IOException {
+
+    PKCS10CertificationRequest csr = generateCSR(client, scmStorageConfig,
+        config, scmAddress, fetchedSCMId);
+
+    CertificateServer rootCAServer =
+        initializeRootCertificateServer(scmStorageConfig.getClusterID(),
+            scmStorageConfig.getScmId(), null);
+
+    rootCAServer.init(new SecurityConfig(config),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    try {
+      X509CertificateHolder certificateHolder = rootCAServer.
+          requestCertificate(csr, KERBEROS_TRUSTED, SCM).get();
+
+      X509CertificateHolder rootCAServerCACertificateHolder =
+          rootCAServer.getCACertificate();
+
+      String pemEncodedCert =
+          CertificateCodec.getPEMEncodedString(certificateHolder);
+
+      String pemEncodedRootCert =
+          CertificateCodec.getPEMEncodedString(rootCAServerCACertificateHolder);
+
+
+      client.storeCertificate(pemEncodedRootCert, true, true);
+      client.storeCertificate(pemEncodedCert, true);

Review comment:
       {metadatadir}/scm/sub-ca/certs/certificate.crt
   {metadatadir}/scm/sub-ca/certs/CA-<<ca-certid>>.crt
   {metadatadir}/scm/sub-ca/certs/<<cert-d>>.crt
   
   To be simple, I have followed this approach. Certificat.crt will be used by sub CA, and cert client uses first 2, and also during load if same cert is loaded, it is already taken care by certclient, so we are good over there.




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