Pārlūkot izejas kodu

HDDS-696. Bootstrap genesis SCM(CA) with self-signed certificate.
Contributed by Anu Engineer.

Anu Engineer 6 gadi atpakaļ
vecāks
revīzija
0c8829a9a1
22 mainītis faili ar 1381 papildinājumiem un 181 dzēšanām
  1. 22 35
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
  2. 67 17
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
  3. 8 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
  4. 373 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
  5. 11 7
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java
  6. 280 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java
  7. 22 0
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/package-info.java
  8. 3 3
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/CertificateSignRequest.java
  9. 57 31
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/SelfSignedCertificate.java
  10. 2 2
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/package-info.java
  11. 22 10
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java
  12. 106 24
      hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java
  13. 1 1
      hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java
  14. 118 0
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java
  15. 22 0
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/package-info.java
  16. 218 0
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateCodec.java
  17. 12 7
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestCertificateSignRequest.java
  18. 23 23
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java
  19. 2 2
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java
  20. 4 4
      hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestKeyCodec.java
  21. 2 3
      hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java
  22. 6 9
      hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java

+ 22 - 35
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java

@@ -1,19 +1,18 @@
 /**
- * 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
+ * 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.
+ * 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;
 
@@ -25,61 +24,45 @@ import org.apache.hadoop.utils.db.DBProfile;
  */
 public final class HddsConfigKeys {
 
-  /**
-   * Do not instantiate.
-   */
-  private HddsConfigKeys() {
-  }
-
   public static final String HDDS_HEARTBEAT_INTERVAL =
       "hdds.heartbeat.interval";
   public static final String HDDS_HEARTBEAT_INTERVAL_DEFAULT =
       "30s";
-
   public static final String HDDS_NODE_REPORT_INTERVAL =
       "hdds.node.report.interval";
   public static final String HDDS_NODE_REPORT_INTERVAL_DEFAULT =
       "60s";
-
   public static final String HDDS_CONTAINER_REPORT_INTERVAL =
       "hdds.container.report.interval";
   public static final String HDDS_CONTAINER_REPORT_INTERVAL_DEFAULT =
       "60s";
-
   public static final String HDDS_PIPELINE_REPORT_INTERVAL =
           "hdds.pipeline.report.interval";
   public static final String HDDS_PIPELINE_REPORT_INTERVAL_DEFAULT =
           "60s";
-
   public static final String HDDS_COMMAND_STATUS_REPORT_INTERVAL =
       "hdds.command.status.report.interval";
   public static final String HDDS_COMMAND_STATUS_REPORT_INTERVAL_DEFAULT =
       "60s";
-
   public static final String HDDS_CONTAINER_ACTION_MAX_LIMIT =
       "hdds.container.action.max.limit";
   public static final int HDDS_CONTAINER_ACTION_MAX_LIMIT_DEFAULT =
       20;
-
   public static final String HDDS_PIPELINE_ACTION_MAX_LIMIT =
       "hdds.pipeline.action.max.limit";
   public static final int HDDS_PIPELINE_ACTION_MAX_LIMIT_DEFAULT =
       20;
-
   // Configuration to allow volume choosing policy.
   public static final String HDDS_DATANODE_VOLUME_CHOOSING_POLICY =
       "hdds.datanode.volume.choosing.policy";
-
   // DB Profiles used by ROCKDB instances.
   public static final String HDDS_DB_PROFILE = "hdds.db.profile";
   public static final DBProfile HDDS_DEFAULT_DB_PROFILE = DBProfile.DISK;
-
   // Once a container usage crosses this threshold, it is eligible for
   // closing.
   public static final String HDDS_CONTAINER_CLOSE_THRESHOLD =
       "hdds.container.close.threshold";
   public static final float HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT = 0.9f;
-
   public static final String HDDS_SCM_CHILLMODE_ENABLED =
       "hdds.scm.chillmode.enabled";
   public static final boolean HDDS_SCM_CHILLMODE_ENABLED_DEFAULT = true;
@@ -97,11 +80,9 @@ public final class HddsConfigKeys {
   public static final String HDDS_SCM_CHILLMODE_THRESHOLD_PCT =
       "hdds.scm.chillmode.threshold.pct";
   public static final double HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT = 0.99;
-
   public static final String HDDS_LOCK_MAX_CONCURRENCY =
       "hdds.lock.max.concurrency";
   public static final int HDDS_LOCK_MAX_CONCURRENCY_DEFAULT = 100;
-
   // This configuration setting is used as a fallback location by all
   // Ozone/HDDS services for their metadata. It is useful as a single
   // config point for test/PoC clusters.
@@ -121,7 +102,6 @@ public final class HddsConfigKeys {
   public static final String HDDS_DEFAULT_SECURITY_PROVIDER = "BC";
   public static final String HDDS_KEY_DIR_NAME = "hdds.key.dir.name";
   public static final String HDDS_KEY_DIR_NAME_DEFAULT = "keys";
-
   // TODO : Talk to StorageIO classes and see if they can return a secure
   // storage location for each node.
   public static final String HDDS_METADATA_DIR_NAME = "hdds.metadata.dir";
@@ -131,7 +111,6 @@ public final class HddsConfigKeys {
   public static final String HDDS_PUBLIC_KEY_FILE_NAME = "hdds.public.key.file"
       + ".name";
   public static final String HDDS_PUBLIC_KEY_FILE_NAME_DEFAULT = "public.pem";
-
   /**
    * Maximum duration of certificates issued by SCM including Self-Signed Roots.
    * The formats accepted are based on the ISO-8601 duration format PnDTnHnMn.nS
@@ -140,12 +119,20 @@ public final class HddsConfigKeys {
   public static final String HDDS_X509_MAX_DURATION = "hdds.x509.max.duration";
   // Limit Certificate duration to a max value of 5 years.
   public static final String HDDS_X509_MAX_DURATION_DEFAULT= "P1865D";
-
   public static final String HDDS_X509_SIGNATURE_ALGO =
       "hdds.x509.signature.algorithm";
   public static final String HDDS_X509_SIGNATURE_ALGO_DEFAULT = "SHA256withRSA";
-
-  public static final String HDDS_GRPC_BLOCK_TOKEN_ENABLED = "hdds.grpc.block" +
-      ".token.enabled";
+  public static final String HDDS_GRPC_BLOCK_TOKEN_ENABLED =
+      "hdds.grpc.block.token.enabled";
   public static final boolean HDDS_GRPC_BLOCK_TOKEN_ENABLED_DEFAULT = false;
+  public static final String HDDS_X509_DIR_NAME = "hdds.x509.dir.name";
+  public static final String HDDS_X509_DIR_NAME_DEFAULT = "certs";
+  public static final String HDDS_X509_FILE_NAME = "hdds.x509.file.name";
+  public static final String HDDS_X509_FILE_NAME_DEFAULT = "certificate.crt";
+
+  /**
+   * Do not instantiate.
+   */
+  private HddsConfigKeys() {
+  }
 }

+ 67 - 17
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java

@@ -46,6 +46,10 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PRIVATE_KEY_FILE_NAME_D
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PUBLIC_KEY_FILE_NAME;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PUBLIC_KEY_FILE_NAME_DEFAULT;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SECURITY_PROVIDER;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_DIR_NAME;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_DIR_NAME_DEFAULT;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_FILE_NAME;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_FILE_NAME_DEFAULT;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_MAX_DURATION;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_MAX_DURATION_DEFAULT;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_SIGNATURE_ALGO;
@@ -54,7 +58,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
 
 /**
  * A class that deals with all Security related configs in HDDS.
- *
+ * <p>
  * This class allows security configs to be read and used consistently across
  * all of security related code base.
  */
@@ -73,6 +77,8 @@ public class SecurityConfig {
   private final Duration certDuration;
   private final String x509SignatureAlgo;
   private final Boolean grpcBlockTokenEnabled;
+  private final String certificateDir;
+  private final String certificateFileName;
 
   /**
    * Constructs a SecurityConfig.
@@ -108,6 +114,10 @@ public class SecurityConfig {
     this.certDuration = Duration.parse(durationString);
     this.x509SignatureAlgo = this.configuration.get(HDDS_X509_SIGNATURE_ALGO,
         HDDS_X509_SIGNATURE_ALGO_DEFAULT);
+    this.certificateDir = this.configuration.get(HDDS_X509_DIR_NAME,
+        HDDS_X509_DIR_NAME_DEFAULT);
+    this.certificateFileName = this.configuration.get(HDDS_X509_FILE_NAME,
+        HDDS_X509_FILE_NAME_DEFAULT);
 
     this.grpcBlockTokenEnabled = this.configuration.getBoolean(
         HDDS_GRPC_BLOCK_TOKEN_ENABLED,
@@ -127,8 +137,17 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the public key file name, This is used for storing the public
-   * keys on disk.
+   * Returns the Standard Certificate file name.
+   *
+   * @return String - Name of the Certificate File.
+   */
+  public String getCertificateFileName() {
+    return certificateFileName;
+  }
+
+  /**
+   * Returns the public key file name, This is used for storing the public keys
+   * on disk.
    *
    * @return String, File name used for public keys.
    */
@@ -137,8 +156,8 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the private key file name.This is used for storing the private
-   * keys on disk.
+   * Returns the private key file name.This is used for storing the private keys
+   * on disk.
    *
    * @return String, File name used for private keys.
    */
@@ -149,16 +168,47 @@ public class SecurityConfig {
   /**
    * Returns the File path to where keys are stored.
    *
-   * @return String Key location.
+   * @return path Key location.
    */
   public Path getKeyLocation() {
     return Paths.get(metadatDir, keyDir);
   }
 
+  /**
+   * Returns the File path to where keys are stored with an additional component
+   * name inserted in between.
+   *
+   * @param component - Component Name - String.
+   * @return Path location.
+   */
+  public Path getKeyLocation(String component) {
+    return Paths.get(metadatDir, component, keyDir);
+  }
+
+  /**
+   * Returns the File path to where keys are stored.
+   *
+   * @return path Key location.
+   */
+  public Path getCertificateLocation() {
+    return Paths.get(metadatDir, certificateDir);
+  }
+
+  /**
+   * Returns the File path to where keys are stored with an addition component
+   * name inserted in between.
+   *
+   * @param component - Component Name - String.
+   * @return Path location.
+   */
+  public Path getCertificateLocation(String component) {
+    return Paths.get(metadatDir, component, certificateDir);
+  }
+
   /**
    * Gets the Key Size, The default key size is 2048, since the default
-   * algorithm used is RSA. User can change this by setting the "hdds.key
-   * .len" in configuration.
+   * algorithm used is RSA. User can change this by setting the "hdds.key.len"
+   * in configuration.
    *
    * @return key size.
    */
@@ -177,8 +227,8 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the Key generation Algorithm used.  User can change this by
-   * setting the "hdds.key.algo" in configuration.
+   * Returns the Key generation Algorithm used.  User can change this by setting
+   * the "hdds.key.algo" in configuration.
    *
    * @return String Algo.
    */
@@ -188,8 +238,8 @@ public class SecurityConfig {
 
   /**
    * Returns the X.509 Signature Algorithm used. This can be changed by setting
-   * "hdds.x509.signature.algorithm" to the new name. The default algorithm
-   * is SHA256withRSA.
+   * "hdds.x509.signature.algorithm" to the new name. The default algorithm is
+   * SHA256withRSA.
    *
    * @return String
    */
@@ -207,11 +257,11 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the maximum length a certificate can be valid in SCM. The
-   * default value is 5 years. This can be changed by setting
-   * "hdds.x509.max.duration" in configuration. The formats accepted are
-   * based on the ISO-8601 duration format PnDTnHnMn.nS
-   *
+   * Returns the maximum length a certificate can be valid in SCM. The default
+   * value is 5 years. This can be changed by setting "hdds.x509.max.duration"
+   * in configuration. The formats accepted are based on the ISO-8601 duration
+   * format PnDTnHnMn.nS
+   * <p>
    * Default value is 5 years and written as P1865D.
    *
    * @return Duration.

+ 8 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java

@@ -20,10 +20,12 @@
 package org.apache.hadoop.hdds.security.x509.certificate.authority;
 
 import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
-import org.apache.hadoop.hdds.security.x509.certificates.CertificateSignRequest;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
 import org.bouncycastle.cert.X509CertificateHolder;
 
+import java.io.IOException;
+import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
 import java.util.concurrent.Future;
 
@@ -47,16 +49,19 @@ public interface CertificateServer {
    * Returns the CA Certificate for this CA.
    *
    * @return X509CertificateHolder - Certificate for this CA.
-   * @throws SCMSecurityException -- usually thrown if this CA is not
+   * @throws CertificateException - usually thrown if this CA is not
    *                              initialized.
+   * @throws IOException - on Error.
    */
   X509CertificateHolder getCACertificate()
-      throws SCMSecurityException;
+      throws CertificateException, IOException;
 
   /**
    * Request a Certificate based on Certificate Signing Request.
    *
    * @param csr - Certificate Signing Request.
+   * @param approver - An Enum which says what kind of approval process to
+   * follow.
    * @return A future that will have this certificate when this request is
    * approved.
    * @throws SCMSecurityException - on Error.

+ 373 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java

@@ -0,0 +1,373 @@
+/*
+ * 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.authority;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+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.security.x509.certificates.utils.SelfSignedCertificate;
+import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
+import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.concurrent.Future;
+import java.util.function.Consumer;
+
+/**
+ * The default CertificateServer used by SCM. This has no dependencies on any
+ * external system, this allows us to bootstrap a CertificateServer from
+ * Scratch.
+ * <p>
+ * Details =======
+ * <p>
+ * The Default CA server is one of the many possible implementations of an SCM
+ * Certificate Authority.
+ * <p>
+ * A certificate authority needs the Root Certificates and its private key to
+ * operate.  The init function of the DefaultCA Server detects four possible
+ * states the System can be in.
+ * <p>
+ * 1.  Success - This means that the expected Certificates and Keys are in
+ * place, and the CA was able to read those files into memory.
+ * <p>
+ * 2. Missing Keys - This means that private keys are missing. This is an error
+ * state which SCM CA cannot recover from. The cluster might have been
+ * initialized earlier and for some reason, we are not able to find the private
+ * keys for the CA. Eventually we will have 2 ways to recover from this state,
+ * first one is to copy the SCM CA private keys from a backup. Second one is to
+ * rekey the whole cluster. Both of these are improvements we will support in
+ * future.
+ * <p>
+ * 3. Missing Certificate - Similar to Missing Keys, but the root certificates
+ * are missing.
+ * <p>
+ * 4. Initialize - We don't have keys or certificates. DefaultCA assumes that
+ * this is a system bootup and will generate the keys and certificates
+ * automatically.
+ * <p>
+ * The init() follows the following logic,
+ * <p>
+ * 1. Compute the Verification Status -- Success, Missing Keys, Missing Certs or
+ * Initialize.
+ * <p>
+ * 2. ProcessVerificationStatus - Returns a Lambda, based on the Verification
+ * Status.
+ * <p>
+ * 3. Invoke the Lambda function.
+ * <p>
+ * At the end of the init function, we have functional CA. This function can be
+ * invoked as many times since we will regenerate the keys and certs only if
+ * both of them are missing.
+ */
+public class DefaultCAServer implements CertificateServer {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DefaultCAServer.class);
+  private final String subject;
+  private final String clusterID;
+  private final String scmID;
+  private String componentName = Paths.get("scm", "ca").toString();
+  private Path caKeysPath;
+  private Path caRootX509Path;
+  private SecurityConfig config;
+
+  /**
+   * Create an Instance of DefaultCAServer.
+   *
+   * @param subject - String Subject
+   * @param clusterID - String ClusterID
+   * @param scmID - String SCMID.
+   */
+  public DefaultCAServer(String subject, String clusterID, String scmID) {
+    this.subject = subject;
+    this.clusterID = clusterID;
+    this.scmID = scmID;
+  }
+
+  @Override
+  public void init(SecurityConfig securityConfig, CAType type)
+      throws SCMSecurityException {
+    caKeysPath = securityConfig.getKeyLocation(componentName);
+    caRootX509Path = securityConfig.getCertificateLocation(componentName);
+    this.config = securityConfig;
+
+    /* In future we will spilt this code to have different kind of CAs.
+     * Right now, we have only self-signed CertificateServer.
+     */
+
+    if (type == CAType.SELF_SIGNED_CA) {
+      VerificationStatus status = verifySelfSignedCA(securityConfig);
+      Consumer<SecurityConfig> caInitializer =
+          processVerificationStatus(status);
+      caInitializer.accept(securityConfig);
+      return;
+    }
+
+    LOG.error("We support only Self-Signed CAs for now.");
+    throw new IllegalStateException("Not implemented functionality requested.");
+  }
+
+  @Override
+  public X509CertificateHolder getCACertificate() throws
+      CertificateException, IOException {
+    CertificateCodec certificateCodec =
+        new CertificateCodec(config, componentName);
+    return certificateCodec.readCertificate();
+  }
+
+  @Override
+  public Future<X509CertificateHolder> requestCertificate(
+      CertificateSignRequest csr, CertificateApprover approver)
+      throws SCMSecurityException {
+    return null;
+  }
+
+  @Override
+  public Future<Boolean> revokeCertificate(X509Certificate certificate,
+      CertificateApprover approver) throws SCMSecurityException {
+    return null;
+  }
+
+  /**
+   * Generates a Self Signed CertificateServer. These are the steps in
+   * generating a Self-Signed CertificateServer.
+   * <p>
+   * 1. Generate a Private/Public Key Pair. 2. Persist to a protected location.
+   * 3. Generate a SelfSigned Root CertificateServer certificate.
+   *
+   * @param securityConfig - Config.
+   */
+  private void generateSelfSignedCA(SecurityConfig securityConfig) throws
+      NoSuchAlgorithmException, NoSuchProviderException, IOException {
+    KeyPair keyPair = generateKeys(securityConfig);
+    generateRootCertificate(securityConfig, keyPair);
+  }
+
+  /**
+   * Verify Self-Signed CertificateServer. 1. Check if the Certificate exist. 2.
+   * Check if the key pair exists.
+   *
+   * @param securityConfig -- Config
+   * @return Verification Status
+   */
+  private VerificationStatus verifySelfSignedCA(SecurityConfig securityConfig) {
+    /*
+    The following is the truth table for the States.
+    True means we have that file False means it is missing.
+    +--------------+--------+--------+--------------+
+    | Certificates |  Keys  | Result |   Function   |
+    +--------------+--------+--------+--------------+
+    | True         | True   | True   | Success      |
+    | False        | False  | True   | Initialize   |
+    | True         | False  | False  | Missing Key  |
+    | False        | True   | False  | Missing Cert |
+    +--------------+--------+--------+--------------+
+
+    This truth table maps to ~(certs xor keys) or certs == keys
+     */
+    boolean keyStatus = checkIfKeysExist();
+    boolean certStatus = checkIfCertificatesExist();
+
+    if ((certStatus == keyStatus) && (certStatus)) {
+      return VerificationStatus.SUCCESS;
+    }
+
+    if ((certStatus == keyStatus) && (!certStatus)) {
+      return VerificationStatus.INITIALIZE;
+    }
+
+    // At this point certStatus is not equal to keyStatus.
+    if (certStatus) {
+      return VerificationStatus.MISSING_KEYS;
+    }
+
+    return VerificationStatus.MISSING_CERTIFICATE;
+  }
+
+  /**
+   * Returns Keys status.
+   *
+   * @return True if the key files exist.
+   */
+  private boolean checkIfKeysExist() {
+    if (!Files.exists(caKeysPath)) {
+      return false;
+    }
+
+    if (!Files.exists(Paths.get(caKeysPath.toString(),
+        this.config.getPrivateKeyFileName()))) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Returns certificate Status.
+   *
+   * @return True if the Certificate files exist.
+   */
+  private boolean checkIfCertificatesExist() {
+    if (!Files.exists(caRootX509Path)) {
+      return false;
+    }
+    if (!Files.exists(Paths.get(caRootX509Path.toString(),
+        this.config.getCertificateFileName()))) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Based on the Status of the verification, we return a lambda that gets
+   * executed by the init function of the CA.
+   *
+   * @param status - Verification Status.
+   */
+  @VisibleForTesting
+  Consumer<SecurityConfig> processVerificationStatus(
+      VerificationStatus status) {
+    Consumer<SecurityConfig> consumer = null;
+    switch (status) {
+    case SUCCESS:
+      consumer = (arg) -> LOG.info("CertificateServer validation is " +
+          "successful");
+      break;
+    case MISSING_KEYS:
+      consumer = (arg) -> {
+        LOG.error("We have found the Certificate for this CertificateServer, " +
+            "but keys used by this CertificateServer is missing. This is a " +
+            "non-recoverable error. Please restart the system after locating " +
+            "the Keys used by the CertificateServer.");
+        LOG.error("Exiting due to unrecoverable CertificateServer error.");
+        throw new IllegalStateException("Missing Keys, cannot continue.");
+      };
+      break;
+    case MISSING_CERTIFICATE:
+      consumer = (arg) -> {
+        LOG.error("We found the keys, but the root certificate for this " +
+            "CertificateServer is missing. Please restart SCM after locating " +
+            "the " +
+            "Certificates.");
+        LOG.error("Exiting due to unrecoverable CertificateServer error.");
+        throw new IllegalStateException("Missing Root Certs, cannot continue.");
+      };
+      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.");
+        }
+      };
+      break;
+    default:
+      /* Make CheckStyle happy */
+      break;
+    }
+    return consumer;
+  }
+
+  /**
+   * Generates a KeyPair for the Certificate.
+   *
+   * @param securityConfig - SecurityConfig.
+   * @return Key Pair.
+   * @throws NoSuchProviderException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws IOException              - on Error.
+   */
+  private KeyPair generateKeys(SecurityConfig securityConfig)
+      throws NoSuchProviderException, NoSuchAlgorithmException, IOException {
+    HDDSKeyGenerator keyGenerator = new HDDSKeyGenerator(securityConfig);
+    KeyPair keys = keyGenerator.generateKey();
+    KeyCodec keyPEMWriter = new KeyCodec(securityConfig,
+        componentName);
+    keyPEMWriter.writeKey(keys);
+    return keys;
+  }
+
+  /**
+   * Generates a self-signed Root Certificate for CA.
+   *
+   * @param securityConfig - SecurityConfig
+   * @param key - KeyPair.
+   * @throws IOException          - on Error.
+   * @throws SCMSecurityException - on Error.
+   */
+  private void generateRootCertificate(SecurityConfig securityConfig,
+      KeyPair key) throws IOException, SCMSecurityException {
+    Preconditions.checkNotNull(this.config);
+    LocalDate beginDate = LocalDate.now().atStartOfDay().toLocalDate();
+    LocalDateTime temp = LocalDateTime.of(beginDate, LocalTime.MIDNIGHT);
+    LocalDate endDate =
+        temp.plus(securityConfig.getMaxCertificateDuration()).toLocalDate();
+    X509CertificateHolder selfSignedCertificate =
+        SelfSignedCertificate
+            .newBuilder()
+            .setSubject(this.subject)
+            .setScmID(this.scmID)
+            .setClusterID(this.clusterID)
+            .setBeginDate(beginDate)
+            .setEndDate(endDate)
+            .makeCA()
+            .setConfiguration(securityConfig.getConfiguration())
+            .setKey(key)
+            .build();
+
+    CertificateCodec certCodec =
+        new CertificateCodec(config, componentName);
+    certCodec.writeCertificate(selfSignedCertificate);
+  }
+
+  /**
+   * This represents the verification status of the CA. Based on this enum
+   * appropriate action is taken in the Init.
+   */
+  @VisibleForTesting
+  enum VerificationStatus {
+    SUCCESS, /* All artifacts needed by CertificateServer is present */
+    MISSING_KEYS, /* Private key is missing, certificate Exists.*/
+    MISSING_CERTIFICATE, /* Keys exist, but root certificate missing.*/
+    INITIALIZE /* All artifacts are missing, we should init the system. */
+  }
+}

+ 11 - 7
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java

@@ -19,7 +19,7 @@
 
 package org.apache.hadoop.hdds.security.x509.certificate.client;
 
-import org.apache.hadoop.hdds.security.x509.certificates.CertificateSignRequest;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
 import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
 
 import java.io.InputStream;
@@ -64,7 +64,7 @@ public interface CertificateClient {
 
   /**
    * Verifies if this certificate is part of a trusted chain.
-   *
+   * @param certificate - certificate.
    * @return true if it trusted, false otherwise.
    */
   boolean verifyCertificate(X509Certificate certificate);
@@ -74,7 +74,9 @@ public interface CertificateClient {
    * key.
    *
    * @param stream - Data stream to sign.
+   * @param component - name of the component.
    * @return byte array - containing the signature.
+   * @throws CertificateException - on Error.
    */
   byte[] signDataStream(InputStream stream, String component)
       throws CertificateException;
@@ -82,6 +84,7 @@ public interface CertificateClient {
   /**
    * Verifies a digital Signature, given the signature and the certificate of
    * the signer.
+   *
    * @param stream - Data Stream.
    * @param signature - Byte Array containing the signature.
    * @param cert - Certificate of the Signer.
@@ -123,7 +126,7 @@ public interface CertificateClient {
    *
    * @param key - private key
    * @param component - name of the component.
-   * @throws CertificateException
+   * @throws CertificateException - on Error.
    */
   void storePrivateKey(PrivateKey key, String component)
       throws CertificateException;
@@ -132,7 +135,8 @@ public interface CertificateClient {
    * Stores the public key of a specified component.
    *
    * @param key - public key
-   * @throws CertificateException
+   * @param component - name of the component.
+   * @throws CertificateException - on Error.
    */
   void storePublicKey(PublicKey key, String component)
       throws CertificateException;
@@ -142,7 +146,7 @@ public interface CertificateClient {
    *
    * @param certificate - X509 Certificate
    * @param component - Name of the component.
-   * @throws CertificateException
+   * @throws CertificateException - on Error.
    */
   void storeCertificate(X509Certificate certificate, String component)
       throws CertificateException;
@@ -152,7 +156,7 @@ public interface CertificateClient {
    *
    * @param certStore - Cert Store.
    * @param component - Trust Chain.
-   * @throws CertificateException
+   * @throws CertificateException - on Error.
    */
   void storeTrustChain(CertStore certStore,
       String component) throws CertificateException;
@@ -162,7 +166,7 @@ public interface CertificateClient {
    *
    * @param certificates - List of Certificates.
    * @param component - String component.
-   * @throws CertificateException
+   * @throws CertificateException - on Error.
    */
   void storeTrustChain(List<X509Certificate> certificates,
       String component) throws CertificateException;

+ 280 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/CertificateCodec.java

@@ -0,0 +1,280 @@
+/*
+ * 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.utils;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.bouncycastle.openssl.jcajce.JcaPEMWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StringWriter;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.attribute.PosixFilePermission;
+import java.security.cert.CertificateEncodingException;
+import java.security.cert.CertificateException;
+import java.security.cert.CertificateFactory;
+import java.security.cert.X509Certificate;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.nio.file.attribute.PosixFilePermission.OWNER_EXECUTE;
+import static java.nio.file.attribute.PosixFilePermission.OWNER_READ;
+import static java.nio.file.attribute.PosixFilePermission.OWNER_WRITE;
+
+/**
+ * A class used to read and write X.509 certificates  PEM encoded Streams.
+ */
+public class CertificateCodec {
+  public static final String BEGIN_CERT = "-----BEGIN CERTIFICATE-----";
+  public static final String END_CERT = "-----END CERTIFICATE-----";
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(CertificateCodec.class);
+  private static final JcaX509CertificateConverter CERTIFICATE_CONVERTER
+      = new JcaX509CertificateConverter();
+  private final SecurityConfig securityConfig;
+  private final Path location;
+  private Set<PosixFilePermission> permissionSet =
+      Stream.of(OWNER_READ, OWNER_WRITE, OWNER_EXECUTE)
+          .collect(Collectors.toSet());
+  /**
+   * Creates an CertificateCodec.
+   *
+   * @param config - Security Config.
+   * @param component - Component String.
+   */
+  public CertificateCodec(SecurityConfig config, String component) {
+    this.securityConfig = config;
+    this.location = securityConfig.getCertificateLocation(component);
+  }
+
+  /**
+   * Creates an CertificateCodec.
+   *
+   * @param configuration - Configuration
+   */
+  public CertificateCodec(Configuration configuration) {
+    Preconditions.checkNotNull(configuration, "Config cannot be null");
+    this.securityConfig = new SecurityConfig(configuration);
+    this.location = securityConfig.getCertificateLocation();
+  }
+
+  /**
+   * Returns a X509 Certificate from the Certificate Holder.
+   *
+   * @param holder - Holder
+   * @return X509Certificate.
+   * @throws CertificateException - on Error.
+   */
+  public static X509Certificate getX509Certificate(X509CertificateHolder holder)
+      throws CertificateException {
+    return CERTIFICATE_CONVERTER.getCertificate(holder);
+  }
+
+  /**
+   * Get Certificate location.
+   *
+   * @return Path
+   */
+  public Path getLocation() {
+    return location;
+  }
+
+  /**
+   * Returns the Certificate as a PEM encoded String.
+   *
+   * @param x509CertHolder - X.509 Certificate Holder.
+   * @return PEM Encoded Certificate String.
+   * @throws SCMSecurityException - On failure to create a PEM String.
+   */
+  public String getPEMEncodedString(X509CertificateHolder x509CertHolder)
+      throws SCMSecurityException {
+    try {
+      StringWriter stringWriter = new StringWriter();
+      try (JcaPEMWriter pemWriter = new JcaPEMWriter(stringWriter)) {
+        pemWriter.writeObject(getX509Certificate(x509CertHolder));
+      }
+      return stringWriter.toString();
+    } catch (CertificateException | IOException e) {
+      LOG.error("Error in encoding certificate." + x509CertHolder
+          .getSubject().toString(), e);
+      throw new SCMSecurityException("PEM Encoding failed for certificate." +
+          x509CertHolder.getSubject().toString(), e);
+    }
+  }
+
+  /**
+   * Gets the X.509 Certificate from PEM encoded String.
+   *
+   * @param pemEncodedString - PEM encoded String.
+   * @return X509Certificate  - Certificate.
+   * @throws CertificateException - Thrown on Failure.
+   * @throws IOException          - Thrown on Failure.
+   */
+  public X509Certificate getX509Certificate(String pemEncodedString)
+      throws CertificateException, IOException {
+    CertificateFactory fact = CertificateFactory.getInstance("X.509");
+    try (InputStream input = IOUtils.toInputStream(pemEncodedString, UTF_8)) {
+      return (X509Certificate) fact.generateCertificate(input);
+    }
+  }
+
+  /**
+   * Write the Certificate pointed to the location by the configs.
+   *
+   * @param xCertificate - Certificate to write.
+   * @throws SCMSecurityException - on Error.
+   * @throws IOException - on Error.
+   */
+  public void writeCertificate(X509CertificateHolder xCertificate)
+      throws SCMSecurityException, IOException {
+    String pem = getPEMEncodedString(xCertificate);
+    writeCertificate(location.toAbsolutePath(),
+        this.securityConfig.getCertificateFileName(), pem, false);
+  }
+
+  /**
+   * Write the Certificate to the specific file.
+   *
+   * @param xCertificate - Certificate to write.
+   * @param fileName - file name to write to.
+   * @param overwrite - boolean value, true means overwrite an existing
+   * certificate.
+   * @throws SCMSecurityException - On Error.
+   * @throws IOException          - On Error.
+   */
+  public void writeCertificate(X509CertificateHolder xCertificate,
+      String fileName, boolean overwrite)
+      throws SCMSecurityException, IOException {
+    String pem = getPEMEncodedString(xCertificate);
+    writeCertificate(location.toAbsolutePath(), fileName, pem, overwrite);
+  }
+
+  /**
+   * Helper function that writes data to the file.
+   *
+   * @param basePath - Base Path where the file needs to written to.
+   * @param fileName - Certificate file name.
+   * @param pemEncodedCertificate - pemEncoded Certificate file.
+   * @param force - Overwrite if the file exists.
+   * @throws IOException - on Error.
+   */
+  public synchronized void writeCertificate(Path basePath, String fileName,
+      String pemEncodedCertificate, boolean force)
+      throws IOException {
+    File certificateFile =
+        Paths.get(basePath.toString(), fileName).toFile();
+    if (certificateFile.exists() && !force) {
+      throw new SCMSecurityException("Specified certificate file already " +
+          "exists.Please use force option if you want to overwrite it.");
+    }
+    if (!basePath.toFile().exists()) {
+      if (!basePath.toFile().mkdirs()) {
+        LOG.error("Unable to create file path. Path: {}", basePath);
+        throw new IOException("Creation of the directories failed."
+            + basePath.toString());
+      }
+    }
+    try (FileOutputStream file = new FileOutputStream(certificateFile)) {
+      IOUtils.write(pemEncodedCertificate, file, UTF_8);
+    }
+
+    Files.setPosixFilePermissions(certificateFile.toPath(), permissionSet);
+  }
+
+  /**
+   * Rertuns a default certificate using the default paths for this component.
+   *
+   * @return X509CertificateHolder.
+   * @throws SCMSecurityException - on Error.
+   * @throws CertificateException - on Error.
+   * @throws IOException          - on Error.
+   */
+  public X509CertificateHolder readCertificate() throws
+      CertificateException, IOException {
+    return readCertificate(this.location.toAbsolutePath(),
+        this.securityConfig.getCertificateFileName());
+  }
+
+  /**
+   * Returns the certificate from the specific PEM encoded file.
+   *
+   * @param basePath - base path
+   * @param fileName - fileName
+   * @return X%09 Certificate
+   * @throws IOException          - on Error.
+   * @throws SCMSecurityException - on Error.
+   * @throws CertificateException - on Error.
+   */
+  public synchronized X509CertificateHolder readCertificate(Path basePath,
+      String fileName) throws IOException, CertificateException {
+    File certificateFile = Paths.get(basePath.toString(), fileName).toFile();
+    return getX509CertificateHolder(certificateFile);
+  }
+
+  /**
+   * Helper function to read certificate.
+   *
+   * @param certificateFile - Full path to certificate file.
+   * @return X509CertificateHolder
+   * @throws IOException          - On Error.
+   * @throws CertificateException - On Error.
+   */
+  private X509CertificateHolder getX509CertificateHolder(File certificateFile)
+      throws IOException, CertificateException {
+    if (!certificateFile.exists()) {
+      throw new IOException("Unable to find the requested certificate. Path: "
+          + certificateFile.toString());
+    }
+    CertificateFactory fact = CertificateFactory.getInstance("X.509");
+    try (FileInputStream is = new FileInputStream(certificateFile)) {
+      return getCertificateHolder(
+          (X509Certificate) fact.generateCertificate(is));
+    }
+  }
+
+  /**
+   * Returns the Certificate holder from X509Ceritificate class.
+   *
+   * @param x509cert - Certificate class.
+   * @return X509CertificateHolder
+   * @throws CertificateEncodingException - on Error.
+   * @throws IOException                  - on Error.
+   */
+  public X509CertificateHolder getCertificateHolder(X509Certificate x509cert)
+      throws CertificateEncodingException, IOException {
+    return new X509CertificateHolder(x509cert.getEncoded());
+  }
+}

+ 22 - 0
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/utils/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Certificate Utils.
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.utils;

+ 3 - 3
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/CertificateSignRequest.java → hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/CertificateSignRequest.java

@@ -5,7 +5,7 @@
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
  *      http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -16,7 +16,7 @@
  * limitations under the License.
  *
  */
-package org.apache.hadoop.hdds.security.x509.certificates;
+package org.apache.hadoop.hdds.security.x509.certificates.utils;
 
 import com.google.common.base.Preconditions;
 import org.apache.hadoop.conf.Configuration;
@@ -49,7 +49,7 @@ import java.util.Optional;
 
 /**
  * A certificate sign request object that wraps operations to build a
- * PKCS10CertificationRequest to CA.
+ * PKCS10CertificationRequest to CertificateServer.
  */
 public final class CertificateSignRequest {
   private final KeyPair keyPair;

+ 57 - 31
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/SelfSignedCertificate.java → hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/SelfSignedCertificate.java

@@ -5,7 +5,7 @@
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
  *      http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -17,7 +17,7 @@
  *
  */
 
-package org.apache.hadoop.hdds.security.x509.certificates;
+package org.apache.hadoop.hdds.security.x509.certificates.utils;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
@@ -27,9 +27,11 @@ import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
 import org.apache.hadoop.util.Time;
 import org.apache.logging.log4j.util.Strings;
+import org.bouncycastle.asn1.DEROctetString;
 import org.bouncycastle.asn1.x500.X500Name;
 import org.bouncycastle.asn1.x509.BasicConstraints;
 import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.KeyUsage;
 import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
 import org.bouncycastle.cert.CertIOException;
 import org.bouncycastle.cert.X509CertificateHolder;
@@ -38,28 +40,33 @@ import org.bouncycastle.operator.ContentSigner;
 import org.bouncycastle.operator.OperatorCreationException;
 import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
 
+import java.io.IOException;
 import java.math.BigInteger;
 import java.security.KeyPair;
 import java.time.Duration;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
 import java.util.Date;
 
 /**
- * A Self Signed Certificate with CA basic constraint can be used to boot-strap
- * a certificate infra-structure, if no external certificate is provided.
+ * A Self Signed Certificate with CertificateServer basic constraint can be used
+ * to bootstrap a certificate infrastructure, if no external certificate is
+ * provided.
  */
 public final class SelfSignedCertificate {
   private static final String NAME_FORMAT = "CN=%s,OU=%s,O=%s";
   private String subject;
   private String clusterID;
   private String scmID;
-  private Date beginDate;
-  private Date endDate;
+  private LocalDate beginDate;
+  private LocalDate endDate;
   private KeyPair key;
   private SecurityConfig config;
-  private boolean isCA;
 
   /**
    * Private Ctor invoked only via Builder Interface.
+   *
    * @param subject - Subject
    * @param scmID - SCM ID
    * @param clusterID - Cluster ID
@@ -67,11 +74,10 @@ public final class SelfSignedCertificate {
    * @param endDate - Not After
    * @param configuration - SCM Config
    * @param keyPair - KeyPair
-   * @param ca - isCA?
    */
   private SelfSignedCertificate(String subject, String scmID, String clusterID,
-      Date beginDate, Date endDate, SecurityConfig configuration,
-      KeyPair keyPair, boolean ca) {
+      LocalDate beginDate, LocalDate endDate, SecurityConfig configuration,
+      KeyPair keyPair) {
     this.subject = subject;
     this.clusterID = clusterID;
     this.scmID = scmID;
@@ -79,7 +85,6 @@ public final class SelfSignedCertificate {
     this.endDate = endDate;
     config = configuration;
     this.key = keyPair;
-    this.isCA = ca;
   }
 
   @VisibleForTesting
@@ -91,8 +96,8 @@ public final class SelfSignedCertificate {
     return new Builder();
   }
 
-  private X509CertificateHolder generateCertificate()
-      throws OperatorCreationException, CertIOException {
+  private X509CertificateHolder generateCertificate(boolean isCA)
+      throws OperatorCreationException, IOException {
     // For the Root Certificate we form the name from Subject, SCM ID and
     // Cluster ID.
     String dnName = String.format(getNameFormat(), subject, scmID, clusterID);
@@ -115,12 +120,27 @@ public final class SelfSignedCertificate {
       serial = new BigInteger(Long.toString(Time.monotonicNow()));
     }
 
+    ZoneOffset zoneOffset =
+        beginDate.atStartOfDay(ZoneOffset.systemDefault()).getOffset();
+
+    // Valid from the Start of the day when we generate this Certificate.
+    Date validFrom =
+        Date.from(beginDate.atTime(LocalTime.MIN).toInstant(zoneOffset));
+
+    // Valid till end day finishes.
+    Date validTill =
+        Date.from(endDate.atTime(LocalTime.MAX).toInstant(zoneOffset));
+
     X509v3CertificateBuilder builder = new X509v3CertificateBuilder(name,
-        serial, beginDate, endDate, name, publicKeyInfo);
+        serial, validFrom, validTill, name, publicKeyInfo);
 
     if (isCA) {
       builder.addExtension(Extension.basicConstraints, true,
           new BasicConstraints(true));
+      int keyUsageFlag = KeyUsage.keyCertSign | KeyUsage.cRLSign;
+      KeyUsage keyUsage = new KeyUsage(keyUsageFlag);
+      builder.addExtension(Extension.keyUsage, false,
+          new DEROctetString(keyUsage));
     }
     return builder.build(contentSigner);
   }
@@ -132,8 +152,8 @@ public final class SelfSignedCertificate {
     private String subject;
     private String clusterID;
     private String scmID;
-    private Date beginDate;
-    private Date endDate;
+    private LocalDate beginDate;
+    private LocalDate endDate;
     private KeyPair key;
     private SecurityConfig config;
     private boolean isCA;
@@ -163,13 +183,13 @@ public final class SelfSignedCertificate {
       return this;
     }
 
-    public Builder setBeginDate(Date date) {
-      this.beginDate = new Date(date.toInstant().toEpochMilli());
+    public Builder setBeginDate(LocalDate date) {
+      this.beginDate = date;
       return this;
     }
 
-    public Builder setEndDate(Date date) {
-      this.endDate = new Date(date.toInstant().toEpochMilli());
+    public Builder setEndDate(LocalDate date) {
+      this.endDate = date;
       return this;
     }
 
@@ -178,7 +198,8 @@ public final class SelfSignedCertificate {
       return this;
     }
 
-    public X509CertificateHolder build() throws SCMSecurityException {
+    public X509CertificateHolder build()
+        throws SCMSecurityException, IOException {
       Preconditions.checkNotNull(key, "Key cannot be null");
       Preconditions.checkArgument(Strings.isNotBlank(subject), "Subject " +
           "cannot be blank");
@@ -187,22 +208,27 @@ public final class SelfSignedCertificate {
       Preconditions.checkArgument(Strings.isNotBlank(scmID), "SCM ID cannot " +
           "be blank");
 
-      Preconditions.checkArgument(beginDate.before(endDate), "Certificate " +
+      Preconditions.checkArgument(beginDate.isBefore(endDate), "Certificate " +
           "begin date should be before end date");
 
-      Duration certDuration = Duration.between(beginDate.toInstant(),
-          endDate.toInstant());
-      Preconditions.checkArgument(
-          certDuration.compareTo(config.getMaxCertificateDuration()) < 0,
-          "Certificate life time cannot be greater than max configured value.");
-
+      // We just read the beginDate and EndDate as Start of the Day and
+      // confirm that we do not violate the maxDuration Config.
+      Duration certDuration = Duration.between(beginDate.atStartOfDay(),
+          endDate.atStartOfDay());
+      Duration maxDuration = config.getMaxCertificateDuration();
+      if (certDuration.compareTo(maxDuration) > 0) {
+        throw new SCMSecurityException("The cert duration violates the " +
+            "maximum configured value. Please check the hdds.x509.max" +
+            ".duration config key. Current Value: " + certDuration +
+            " config: " + maxDuration);
+      }
 
       SelfSignedCertificate rootCertificate =
           new SelfSignedCertificate(this.subject,
-          this.scmID, this.clusterID, this.beginDate, this.endDate,
-          this.config, key, isCA);
+              this.scmID, this.clusterID, this.beginDate, this.endDate,
+              this.config, key);
       try {
-        return rootCertificate.generateCertificate();
+        return rootCertificate.generateCertificate(isCA);
       } catch (OperatorCreationException | CertIOException e) {
         throw new CertificateException("Unable to create root certificate.",
             e.getCause());

+ 2 - 2
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java → hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/utils/package-info.java

@@ -5,7 +5,7 @@
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
  *      http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -19,4 +19,4 @@
 /**
  * Utils for Certificates.
  */
-package org.apache.hadoop.hdds.security.x509.certificates;
+package org.apache.hadoop.hdds.security.x509.certificates.utils;

+ 22 - 10
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java

@@ -28,7 +28,9 @@ import java.security.KeyPairGenerator;
 import java.security.NoSuchAlgorithmException;
 import java.security.NoSuchProviderException;
 
-/** A class to generate Key Pair for use with Certificates. */
+/**
+ * A class to generate Key Pair for use with Certificates.
+ */
 public class HDDSKeyGenerator {
   private static final Logger LOG =
       LoggerFactory.getLogger(HDDSKeyGenerator.class);
@@ -43,8 +45,18 @@ public class HDDSKeyGenerator {
     this.securityConfig = new SecurityConfig(configuration);
   }
 
+  /**
+   * Constructor that takes a SecurityConfig as the Argument.
+   *
+   * @param config - SecurityConfig
+   */
+  public HDDSKeyGenerator(SecurityConfig config) {
+    this.securityConfig = config;
+  }
+
   /**
    * Returns the Security config used for this object.
+   *
    * @return SecurityConfig
    */
   public SecurityConfig getSecurityConfig() {
@@ -55,10 +67,10 @@ public class HDDSKeyGenerator {
    * Use Config to generate key.
    *
    * @return KeyPair
-   * @throws NoSuchProviderException - On Error, due to missing Java
-   * dependencies.
+   * @throws NoSuchProviderException  - On Error, due to missing Java
+   *                                  dependencies.
    * @throws NoSuchAlgorithmException - On Error,  due to missing Java
-   * dependencies.
+   *                                  dependencies.
    */
   public KeyPair generateKey() throws NoSuchProviderException,
       NoSuchAlgorithmException {
@@ -71,10 +83,10 @@ public class HDDSKeyGenerator {
    *
    * @param size - int, valid key sizes.
    * @return KeyPair
-   * @throws NoSuchProviderException - On Error, due to missing Java
-   * dependencies.
+   * @throws NoSuchProviderException  - On Error, due to missing Java
+   *                                  dependencies.
    * @throws NoSuchAlgorithmException - On Error,  due to missing Java
-   * dependencies.
+   *                                  dependencies.
    */
   public KeyPair generateKey(int size) throws
       NoSuchProviderException, NoSuchAlgorithmException {
@@ -89,10 +101,10 @@ public class HDDSKeyGenerator {
    * @param algorithm - Algorithm to use
    * @param provider - Security provider.
    * @return KeyPair.
-   * @throws NoSuchProviderException - On Error, due to missing Java
-   * dependencies.
+   * @throws NoSuchProviderException  - On Error, due to missing Java
+   *                                  dependencies.
    * @throws NoSuchAlgorithmException - On Error,  due to missing Java
-   * dependencies.
+   *                                  dependencies.
    */
   public KeyPair generateKey(int size, String algorithm, String provider)
       throws NoSuchProviderException, NoSuchAlgorithmException {

+ 106 - 24
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyPEMWriter.java → hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java

@@ -20,24 +20,33 @@ package org.apache.hadoop.hdds.security.x509.keys;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
-import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.output.FileWriterWithEncoding;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 import org.bouncycastle.util.io.pem.PemObject;
+import org.bouncycastle.util.io.pem.PemReader;
 import org.bouncycastle.util.io.pem.PemWriter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.StringReader;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.nio.file.attribute.PosixFilePermission;
+import java.security.KeyFactory;
 import java.security.KeyPair;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.PKCS8EncodedKeySpec;
 import java.util.Set;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
@@ -48,33 +57,44 @@ import static java.nio.file.attribute.PosixFilePermission.OWNER_READ;
 import static java.nio.file.attribute.PosixFilePermission.OWNER_WRITE;
 
 /**
- * We store all Key material in good old PEM files.
- * This helps in avoiding dealing will persistent
- * Java KeyStore issues. Also when debugging,
- * general tools like OpenSSL can be used to read and
- * decode these files.
+ * We store all Key material in good old PEM files. This helps in avoiding
+ * dealing will persistent Java KeyStore issues. Also when debugging, general
+ * tools like OpenSSL can be used to read and decode these files.
  */
-public class HDDSKeyPEMWriter {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(HDDSKeyPEMWriter.class);
+public class KeyCodec {
+  public final static String PRIVATE_KEY = "PRIVATE KEY";
+  public final static String PUBLIC_KEY = "PUBLIC KEY";
+  public final static Charset DEFAULT_CHARSET = StandardCharsets.UTF_8;
+  private final static  Logger LOG =
+      LoggerFactory.getLogger(KeyCodec.class);
   private final Path location;
   private final SecurityConfig securityConfig;
   private Set<PosixFilePermission> permissionSet =
-      Stream.of(OWNER_READ, OWNER_WRITE,  OWNER_EXECUTE)
+      Stream.of(OWNER_READ, OWNER_WRITE, OWNER_EXECUTE)
           .collect(Collectors.toSet());
   private Supplier<Boolean> isPosixFileSystem;
-  public final static String PRIVATE_KEY = "PRIVATE KEY";
-  public final static String PUBLIC_KEY = "PUBLIC KEY";
-  public static final Charset DEFAULT_CHARSET = StandardCharsets.UTF_8;
-  /*
-    Creates an HDDS Key Writer.
 
-    @param configuration - Configuration
+  /**
+   * Creates an KeyCodec.
+   *
+   * @param config - Security Config.
+   * @param component - Component String.
+   */
+  public KeyCodec(SecurityConfig config, String component) {
+    this.securityConfig = config;
+    isPosixFileSystem = KeyCodec::isPosix;
+    this.location = securityConfig.getKeyLocation(component);
+  }
+
+  /**
+   * Creates an HDDS Key Writer.
+   *
+   * @param configuration - Configuration
    */
-  public HDDSKeyPEMWriter(Configuration configuration) throws IOException {
+  public KeyCodec(Configuration configuration) {
     Preconditions.checkNotNull(configuration, "Config cannot be null");
     this.securityConfig = new SecurityConfig(configuration);
-    isPosixFileSystem = HDDSKeyPEMWriter::isPosix;
+    isPosixFileSystem = KeyCodec::isPosix;
     this.location = securityConfig.getKeyLocation();
   }
 
@@ -90,6 +110,7 @@ public class HDDSKeyPEMWriter {
 
   /**
    * Returns the Permission set.
+   *
    * @return Set
    */
   @VisibleForTesting
@@ -99,6 +120,7 @@ public class HDDSKeyPEMWriter {
 
   /**
    * Returns the Security config used for this object.
+   *
    * @return SecurityConfig
    */
   public SecurityConfig getSecurityConfig() {
@@ -108,8 +130,8 @@ public class HDDSKeyPEMWriter {
   /**
    * This function is used only for testing.
    *
-   * @param isPosixFileSystem - Sets a boolean function for mimicking
-   * files systems that are not posix.
+   * @param isPosixFileSystem - Sets a boolean function for mimicking files
+   * systems that are not posix.
    */
   @VisibleForTesting
   public void setIsPosixFileSystem(Supplier<Boolean> isPosixFileSystem) {
@@ -153,6 +175,66 @@ public class HDDSKeyPEMWriter {
         securityConfig.getPublicKeyFileName(), overwrite);
   }
 
+  /**
+   * Reads a Private Key from the PEM Encoded Store.
+   *
+   * @param basePath - Base Path, Directory where the Key is stored.
+   * @param keyFileName - File Name of the private key
+   * @return PrivateKey Object.
+   * @throws IOException - on Error.
+   */
+  private PKCS8EncodedKeySpec readKey(Path basePath, String keyFileName)
+      throws IOException {
+    File fileName = Paths.get(basePath.toString(), keyFileName).toFile();
+    String keyData = FileUtils.readFileToString(fileName, DEFAULT_CHARSET);
+    final byte[] pemContent;
+    try (PemReader pemReader = new PemReader(new StringReader(keyData))) {
+      PemObject keyObject = pemReader.readPemObject();
+      pemContent = keyObject.getContent();
+    }
+    return new PKCS8EncodedKeySpec(pemContent);
+  }
+
+  /**
+   * Returns a Private Key from a PEM encoded file.
+   *
+   * @param basePath - base path
+   * @param privateKeyFileName - private key file name.
+   * @return PrivateKey
+   * @throws InvalidKeySpecException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws IOException              - on Error.
+   */
+  public PrivateKey readPrivateKey(Path basePath, String privateKeyFileName)
+      throws InvalidKeySpecException, NoSuchAlgorithmException, IOException {
+    PKCS8EncodedKeySpec encodedKeySpec = readKey(basePath, privateKeyFileName);
+    final KeyFactory keyFactory =
+        KeyFactory.getInstance(securityConfig.getProvider());
+    final PrivateKey privateKey =
+        keyFactory.generatePrivate(encodedKeySpec);
+    return privateKey;
+  }
+
+  /**
+   * Returns a public key from a PEM encoded file.
+   *
+   * @param basePath - base path.
+   * @param publicKeyFileName - public key file name.
+   * @return PublicKey
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws InvalidKeySpecException  - on Error.
+   * @throws IOException              - on Error.
+   */
+  public PublicKey readPublicKey(Path basePath, String publicKeyFileName)
+      throws NoSuchAlgorithmException, InvalidKeySpecException, IOException {
+    PKCS8EncodedKeySpec encodedKeySpec = readKey(basePath, publicKeyFileName);
+    final KeyFactory keyFactory =
+        KeyFactory.getInstance(securityConfig.getProvider());
+    final PublicKey publicKey =
+        keyFactory.generatePublic(encodedKeySpec);
+    return publicKey;
+  }
+
   /**
    * Helper function that actually writes data to the files.
    *
@@ -190,9 +272,9 @@ public class HDDSKeyPEMWriter {
   }
 
   /**
-   * Checks if private and public key file already exists. Throws IOException
-   * if file exists and force flag is set to false, else will delete the
-   * existing file.
+   * Checks if private and public key file already exists. Throws IOException if
+   * file exists and force flag is set to false, else will delete the existing
+   * file.
    *
    * @param privateKeyFile - Private key file.
    * @param force - forces overwriting the keys.

+ 1 - 1
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/StorageInfo.java

@@ -62,7 +62,7 @@ public class StorageInfo {
    * @param cT
    *          Cluster creation Time
 
-   * @throws IOException
+   * @throws IOException - on Error.
    */
   public StorageInfo(NodeType type, String cid, long cT)
       throws IOException {

+ 118 - 0
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java

@@ -0,0 +1,118 @@
+/*
+ * 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.authority;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.util.function.Consumer;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
+import static org.junit.Assert.*;
+
+/**
+ * Tests the Default CA Server.
+ */
+public class TestDefaultCAServer {
+  private static OzoneConfiguration conf = new OzoneConfiguration();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void init() throws IOException {
+    conf.set(OZONE_METADATA_DIRS, temporaryFolder.newFolder().toString());
+  }
+
+  @Test
+  public void testInit() throws SCMSecurityException, CertificateException,
+      IOException {
+    SecurityConfig securityConfig = new SecurityConfig(conf);
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        RandomStringUtils.randomAlphabetic(4),
+        RandomStringUtils.randomAlphabetic(4));
+    testCA.init(securityConfig, CertificateServer.CAType.SELF_SIGNED_CA);
+    X509CertificateHolder first = testCA.getCACertificate();
+    assertNotNull(first);
+    //Init is idempotent.
+    testCA.init(securityConfig, CertificateServer.CAType.SELF_SIGNED_CA);
+    X509CertificateHolder second = testCA.getCACertificate();
+    assertEquals(first, second);
+
+    // we only support Self Signed CA for now.
+    try {
+      testCA.init(securityConfig, CertificateServer.CAType.INTERMEDIARY_CA);
+      fail("code should not reach here, exception should have been thrown.");
+    } catch (IllegalStateException e) {
+      // This is a run time exception, hence it is not caught by the junit
+      // expected Exception.
+      assertTrue(e.toString().contains("Not implemented"));
+    }
+  }
+
+  @Test
+  public void testMissingCertificate() {
+    SecurityConfig securityConfig = new SecurityConfig(conf);
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        RandomStringUtils.randomAlphabetic(4),
+        RandomStringUtils.randomAlphabetic(4));
+    Consumer<SecurityConfig> caInitializer =
+        ((DefaultCAServer) testCA).processVerificationStatus(
+        DefaultCAServer.VerificationStatus.MISSING_CERTIFICATE);
+    try {
+
+      caInitializer.accept(securityConfig);
+      fail("code should not reach here, exception should have been thrown.");
+    } catch (IllegalStateException e) {
+      // This also is a runtime exception. Hence not caught by junit expected
+      // exception.
+      assertTrue(e.toString().contains("Missing Root Certs"));
+    }
+  }
+
+  @Test
+  public void testMissingKey() {
+    SecurityConfig securityConfig = new SecurityConfig(conf);
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        RandomStringUtils.randomAlphabetic(4),
+        RandomStringUtils.randomAlphabetic(4));
+    Consumer<SecurityConfig> caInitializer =
+        ((DefaultCAServer) testCA).processVerificationStatus(
+            DefaultCAServer.VerificationStatus.MISSING_KEYS);
+    try {
+
+      caInitializer.accept(securityConfig);
+      fail("code should not reach here, exception should have been thrown.");
+    } catch (IllegalStateException e) {
+      // This also is a runtime exception. Hence not caught by junit expected
+      // exception.
+      assertTrue(e.toString().contains("Missing Keys"));
+    }
+
+  }
+}

+ 22 - 0
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/package-info.java

@@ -0,0 +1,22 @@
+/*
+ * 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.
+ *
+ */
+/**
+ * Tests for Default CA.
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.authority;

+ 218 - 0
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/utils/TestCertificateCodec.java

@@ -0,0 +1,218 @@
+/*
+ * 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.utils;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.SelfSignedCertificate;
+import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.time.LocalDate;
+import java.time.temporal.ChronoUnit;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests the Certificate codecs.
+ */
+public class TestCertificateCodec {
+  private static OzoneConfiguration conf = new OzoneConfiguration();
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  @Before
+  public void init() throws IOException {
+    conf.set(OZONE_METADATA_DIRS, temporaryFolder.newFolder().toString());
+  }
+
+  /**
+   * This test converts a X509Certificate Holder object to a PEM encoded String,
+   * then creates a new X509Certificate object to verify that we are able to
+   * serialize and deserialize correctly. we follow up with converting these
+   * objects to standard JCA x509Certificate objects.
+   *
+   * @throws NoSuchProviderException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws IOException              - on Error.
+   * @throws SCMSecurityException     - on Error.
+   * @throws CertificateException     - on Error.
+   */
+  @Test
+  public void testGetPEMEncodedString()
+      throws NoSuchProviderException, NoSuchAlgorithmException,
+      IOException, SCMSecurityException, CertificateException {
+    HDDSKeyGenerator keyGenerator =
+        new HDDSKeyGenerator(conf);
+    X509CertificateHolder cert =
+        SelfSignedCertificate.newBuilder()
+            .setSubject(RandomStringUtils.randomAlphabetic(4))
+            .setClusterID(RandomStringUtils.randomAlphabetic(4))
+            .setScmID(RandomStringUtils.randomAlphabetic(4))
+            .setBeginDate(LocalDate.now())
+            .setEndDate(LocalDate.now().plus(1, ChronoUnit.DAYS))
+            .setConfiguration(keyGenerator.getSecurityConfig()
+                .getConfiguration())
+            .setKey(keyGenerator.generateKey())
+            .makeCA()
+            .build();
+    CertificateCodec codec = new CertificateCodec(conf);
+    String pemString = codec.getPEMEncodedString(cert);
+    assertTrue(pemString.startsWith(CertificateCodec.BEGIN_CERT));
+    assertTrue(pemString.endsWith(CertificateCodec.END_CERT + "\n"));
+
+    // Read back the certificate and verify that all the comparisons pass.
+    X509CertificateHolder newCert =
+        codec.getCertificateHolder(codec.getX509Certificate(pemString));
+    assertEquals(cert, newCert);
+
+    // Just make sure we can decode both these classes to Java Std. lIb classes.
+    X509Certificate firstCert = CertificateCodec.getX509Certificate(cert);
+    X509Certificate secondCert = CertificateCodec.getX509Certificate(newCert);
+    assertEquals(firstCert, secondCert);
+  }
+
+  /**
+   * tests writing and reading certificates in PEM encoded form.
+   *
+   * @throws NoSuchProviderException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws IOException              - on Error.
+   * @throws SCMSecurityException     - on Error.
+   * @throws CertificateException     - on Error.
+   */
+  @Test
+  public void testwriteCertificate() throws NoSuchProviderException,
+      NoSuchAlgorithmException, IOException, SCMSecurityException,
+      CertificateException {
+    HDDSKeyGenerator keyGenerator =
+        new HDDSKeyGenerator(conf);
+    X509CertificateHolder cert =
+        SelfSignedCertificate.newBuilder()
+            .setSubject(RandomStringUtils.randomAlphabetic(4))
+            .setClusterID(RandomStringUtils.randomAlphabetic(4))
+            .setScmID(RandomStringUtils.randomAlphabetic(4))
+            .setBeginDate(LocalDate.now())
+            .setEndDate(LocalDate.now().plus(1, ChronoUnit.DAYS))
+            .setConfiguration(keyGenerator.getSecurityConfig()
+                .getConfiguration())
+            .setKey(keyGenerator.generateKey())
+            .makeCA()
+            .build();
+    CertificateCodec codec = new CertificateCodec(conf);
+    String pemString = codec.getPEMEncodedString(cert);
+    File basePath = temporaryFolder.newFolder();
+    if (!basePath.exists()) {
+      Assert.assertTrue(basePath.mkdirs());
+    }
+    codec.writeCertificate(basePath.toPath(), "pemcertificate.crt",
+        pemString, false);
+    X509CertificateHolder certHolder =
+        codec.readCertificate(basePath.toPath(), "pemcertificate.crt");
+    assertNotNull(certHolder);
+    assertEquals(cert.getSerialNumber(), certHolder.getSerialNumber());
+  }
+
+  /**
+   * Tests reading and writing certificates in DER form.
+   *
+   * @throws IOException              - on Error.
+   * @throws SCMSecurityException     - on Error.
+   * @throws CertificateException     - on Error.
+   * @throws NoSuchProviderException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   */
+  @Test
+  public void testwriteCertificateDefault()
+      throws IOException, SCMSecurityException, CertificateException,
+      NoSuchProviderException, NoSuchAlgorithmException {
+    HDDSKeyGenerator keyGenerator =
+        new HDDSKeyGenerator(conf);
+    X509CertificateHolder cert =
+        SelfSignedCertificate.newBuilder()
+            .setSubject(RandomStringUtils.randomAlphabetic(4))
+            .setClusterID(RandomStringUtils.randomAlphabetic(4))
+            .setScmID(RandomStringUtils.randomAlphabetic(4))
+            .setBeginDate(LocalDate.now())
+            .setEndDate(LocalDate.now().plus(1, ChronoUnit.DAYS))
+            .setConfiguration(keyGenerator.getSecurityConfig()
+                .getConfiguration())
+            .setKey(keyGenerator.generateKey())
+            .makeCA()
+            .build();
+    CertificateCodec codec = new CertificateCodec(conf);
+    codec.writeCertificate(cert);
+    X509CertificateHolder certHolder = codec.readCertificate();
+    assertNotNull(certHolder);
+    assertEquals(cert.getSerialNumber(), certHolder.getSerialNumber());
+  }
+
+  /**
+   * Tests writing to non-default certificate file name.
+   *
+   * @throws IOException              - on Error.
+   * @throws SCMSecurityException     - on Error.
+   * @throws NoSuchProviderException  - on Error.
+   * @throws NoSuchAlgorithmException - on Error.
+   * @throws CertificateException     - on Error.
+   */
+  @Test
+  public void writeCertificate2() throws IOException, SCMSecurityException,
+      NoSuchProviderException, NoSuchAlgorithmException, CertificateException {
+    HDDSKeyGenerator keyGenerator =
+        new HDDSKeyGenerator(conf);
+    X509CertificateHolder cert =
+        SelfSignedCertificate.newBuilder()
+            .setSubject(RandomStringUtils.randomAlphabetic(4))
+            .setClusterID(RandomStringUtils.randomAlphabetic(4))
+            .setScmID(RandomStringUtils.randomAlphabetic(4))
+            .setBeginDate(LocalDate.now())
+            .setEndDate(LocalDate.now().plus(1, ChronoUnit.DAYS))
+            .setConfiguration(keyGenerator.getSecurityConfig()
+                .getConfiguration())
+            .setKey(keyGenerator.generateKey())
+            .makeCA()
+            .build();
+    CertificateCodec codec =
+        new CertificateCodec(keyGenerator.getSecurityConfig(), "ca");
+    codec.writeCertificate(cert, "newcert.crt", false);
+    // Rewrite with force support
+    codec.writeCertificate(cert, "newcert.crt", true);
+    X509CertificateHolder x509CertificateHolder =
+        codec.readCertificate(codec.getLocation(), "newcert.crt");
+    assertNotNull(x509CertificateHolder);
+
+  }
+}

+ 12 - 7
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestCertificateSignRequest.java

@@ -1,4 +1,4 @@
-/**
+/*
  * 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
@@ -6,20 +6,22 @@
  * 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>
+ *
+ *      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.certificates;
+package org.apache.hadoop.hdds.security.x509.certificate.utils;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
 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.keys.HDDSKeyGenerator;
 import org.apache.hadoop.hdds.security.x509.keys.SecurityUtil;
 import org.bouncycastle.asn1.ASN1Sequence;
@@ -45,12 +47,15 @@ import java.util.UUID;
 
 import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
 
+/**
+ * Certificate Signing Request.
+ */
 public class TestCertificateSignRequest {
 
-  private SecurityConfig securityConfig;
   private static OzoneConfiguration conf = new OzoneConfiguration();
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private SecurityConfig securityConfig;
 
   @Before
   public void init() throws IOException {
@@ -257,7 +262,7 @@ public class TestCertificateSignRequest {
     // Verify CSR with attribute for extensions
     Assert.assertEquals(1, csr.getAttributes().length);
   }
-  
+
   @Test
   public void testCsrSerialization() throws NoSuchProviderException,
       NoSuchAlgorithmException, SCMSecurityException, IOException {

+ 23 - 23
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java

@@ -5,7 +5,7 @@
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
  *      http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -17,11 +17,12 @@
  *
  */
 
-package org.apache.hadoop.hdds.security.x509.certificates;
+package org.apache.hadoop.hdds.security.x509.certificate.utils;
 
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
 import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.SelfSignedCertificate;
 import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
 import org.bouncycastle.asn1.x509.Extension;
 import org.bouncycastle.cert.X509CertificateHolder;
@@ -41,8 +42,8 @@ import java.security.NoSuchProviderException;
 import java.security.SignatureException;
 import java.security.cert.CertificateException;
 import java.security.cert.X509Certificate;
-import java.time.Duration;
-import java.time.Instant;
+import java.time.LocalDate;
+import java.time.temporal.ChronoUnit;
 import java.util.Date;
 import java.util.UUID;
 
@@ -52,10 +53,10 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.OZONE_METADATA_DIRS;
  * Test Class for Root Certificate generation.
  */
 public class TestRootCertificate {
-  private SecurityConfig securityConfig;
   private static OzoneConfiguration conf = new OzoneConfiguration();
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private SecurityConfig securityConfig;
 
   @Before
   public void init() throws IOException {
@@ -67,10 +68,9 @@ public class TestRootCertificate {
   public void testAllFieldsAreExpected()
       throws SCMSecurityException, NoSuchProviderException,
       NoSuchAlgorithmException, CertificateException,
-      SignatureException, InvalidKeyException {
-    Instant now = Instant.now();
-    Date notBefore = Date.from(now);
-    Date notAfter = Date.from(now.plus(Duration.ofDays(365)));
+      SignatureException, InvalidKeyException, IOException {
+    LocalDate notBefore = LocalDate.now();
+    LocalDate notAfter = notBefore.plus(365, ChronoUnit.DAYS);
     String clusterID = UUID.randomUUID().toString();
     String scmID = UUID.randomUUID().toString();
     String subject = "testRootCert";
@@ -96,13 +96,15 @@ public class TestRootCertificate {
 
 
     // Make sure that NotBefore is before the current Date
-    Date invalidDate = Date.from(now.minus(Duration.ofDays(1)));
+    Date invalidDate = java.sql.Date.valueOf(
+        notBefore.minus(1, ChronoUnit.DAYS));
     Assert.assertFalse(
         certificateHolder.getNotBefore()
             .before(invalidDate));
 
     //Make sure the end date is honored.
-    invalidDate = Date.from(now.plus(Duration.ofDays(366)));
+    invalidDate = java.sql.Date.valueOf(
+        notAfter.plus(1, ChronoUnit.DAYS));
     Assert.assertFalse(
         certificateHolder.getNotAfter()
             .after(invalidDate));
@@ -113,7 +115,8 @@ public class TestRootCertificate {
     Assert.assertEquals(certificateHolder.getIssuer().toString(), dnName);
     Assert.assertEquals(certificateHolder.getSubject().toString(), dnName);
 
-    // We did not ask for this Certificate to be a CA certificate, hence that
+    // We did not ask for this Certificate to be a CertificateServer
+    // certificate, hence that
     // extension should be null.
     Assert.assertNull(
         certificateHolder.getExtension(Extension.basicConstraints));
@@ -128,10 +131,9 @@ public class TestRootCertificate {
   @Test
   public void testCACert()
       throws SCMSecurityException, NoSuchProviderException,
-      NoSuchAlgorithmException {
-    Instant now = Instant.now();
-    Date notBefore = Date.from(now);
-    Date notAfter = Date.from(now.plus(Duration.ofDays(365)));
+      NoSuchAlgorithmException, IOException {
+    LocalDate notBefore = LocalDate.now();
+    LocalDate notAfter = notBefore.plus(365, ChronoUnit.DAYS);
     String clusterID = UUID.randomUUID().toString();
     String scmID = UUID.randomUUID().toString();
     String subject = "testRootCert";
@@ -151,7 +153,8 @@ public class TestRootCertificate {
             .makeCA();
 
     X509CertificateHolder certificateHolder = builder.build();
-    // This time we asked for a CA Certificate, make sure that extension is
+    // This time we asked for a CertificateServer Certificate, make sure that
+    // extension is
     // present and valid.
     Extension basicExt =
         certificateHolder.getExtension(Extension.basicConstraints);
@@ -167,10 +170,9 @@ public class TestRootCertificate {
   @Test
   public void testInvalidParamFails()
       throws SCMSecurityException, NoSuchProviderException,
-      NoSuchAlgorithmException {
-    Instant now = Instant.now();
-    Date notBefore = Date.from(now);
-    Date notAfter = Date.from(now.plus(Duration.ofDays(365)));
+      NoSuchAlgorithmException, IOException {
+    LocalDate notBefore = LocalDate.now();
+    LocalDate notAfter = notBefore.plus(365, ChronoUnit.DAYS);
     String clusterID = UUID.randomUUID().toString();
     String scmID = UUID.randomUUID().toString();
     String subject = "testRootCert";
@@ -253,6 +255,4 @@ public class TestRootCertificate {
     // Assert that we can create a certificate with all sane params.
     Assert.assertNotNull(builder.build());
   }
-
-
 }

+ 2 - 2
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java

@@ -5,7 +5,7 @@
  * 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
+ * with the License.  You may obtain a copy of the License at
  *
  *      http://www.apache.org/licenses/LICENSE-2.0
  *
@@ -19,4 +19,4 @@
 /**
  * Test classes for Certificate utilities.
  */
-package org.apache.hadoop.hdds.security.x509.certificates;
+package org.apache.hadoop.hdds.security.x509.certificate.utils;

+ 4 - 4
hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyPEMWriter.java → hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestKeyCodec.java

@@ -51,7 +51,7 @@ import org.junit.rules.TemporaryFolder;
 /**
  * Test class for HDDS pem writer.
  */
-public class TestHDDSKeyPEMWriter {
+public class TestKeyCodec {
 
   @Rule
   public TemporaryFolder temporaryFolder = new TemporaryFolder();
@@ -82,7 +82,7 @@ public class TestHDDSKeyPEMWriter {
       throws NoSuchProviderException, NoSuchAlgorithmException,
       IOException, InvalidKeySpecException {
     KeyPair keys = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    KeyCodec pemWriter = new KeyCodec(configuration);
     pemWriter.writeKey(keys);
 
     // Assert that locations have been created.
@@ -171,7 +171,7 @@ public class TestHDDSKeyPEMWriter {
   public void testReWriteKey()
       throws Exception {
     KeyPair kp = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    KeyCodec pemWriter = new KeyCodec(configuration);
     SecurityConfig secConfig = pemWriter.getSecurityConfig();
     pemWriter.writeKey(kp);
 
@@ -205,7 +205,7 @@ public class TestHDDSKeyPEMWriter {
   public void testWriteKeyInNonPosixFS()
       throws Exception {
     KeyPair kp = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    KeyCodec pemWriter = new KeyCodec(configuration);
     pemWriter.setIsPosixFileSystem(() -> false);
 
     // Assert key rewrite fails in non Posix file system.

+ 2 - 3
hadoop-hdds/common/src/test/java/org/apache/hadoop/utils/db/TestDBStoreBuilder.java

@@ -31,7 +31,6 @@ import org.junit.rules.TemporaryFolder;
 import java.io.File;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
-import java.util.Arrays;
 
 /**
  * Tests RDBStore creation.
@@ -131,7 +130,7 @@ public class TestDBStoreBuilder {
             RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
         firstTable.put(key, value);
         byte[] temp = firstTable.get(key);
-        Assert.assertTrue(Arrays.equals(value, temp));
+        Assert.assertArrayEquals(value, temp);
       }
 
       try (Table secondTable = dbStore.getTable("Second")) {
@@ -161,7 +160,7 @@ public class TestDBStoreBuilder {
             RandomStringUtils.random(9).getBytes(StandardCharsets.UTF_8);
         firstTable.put(key, value);
         byte[] temp = firstTable.get(key);
-        Assert.assertTrue(Arrays.equals(value, temp));
+        Assert.assertArrayEquals(value, temp);
       }
 
       try (Table secondTable = dbStore.getTable("Second")) {

+ 6 - 9
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java

@@ -46,7 +46,7 @@ import org.apache.hadoop.hdds.scm.server.SCMStorage;
 import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
 import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
 import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
-import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyPEMWriter;
+import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.Client;
 import org.apache.hadoop.ipc.RPC;
@@ -88,7 +88,7 @@ public final class TestSecureOzoneCluster {
 
   private static final String TEST_USER = "testUgiUser";
   private static final int CLIENT_TIMEOUT = 2 * 1000;
-  private Logger LOGGER = LoggerFactory
+  private Logger logger = LoggerFactory
       .getLogger(TestSecureOzoneCluster.class);
 
   @Rule
@@ -127,9 +127,9 @@ public final class TestSecureOzoneCluster {
       createCredentialsInKDC(conf, miniKdc);
       generateKeyPair(conf);
     } catch (IOException e) {
-      LOGGER.error("Failed to initialize TestSecureOzoneCluster", e);
+      logger.error("Failed to initialize TestSecureOzoneCluster", e);
     } catch (Exception e) {
-      LOGGER.error("Failed to initialize TestSecureOzoneCluster", e);
+      logger.error("Failed to initialize TestSecureOzoneCluster", e);
     }
   }
 
@@ -148,7 +148,7 @@ public final class TestSecureOzoneCluster {
       }
       FileUtils.deleteQuietly(metaDirPath.toFile());
     } catch (Exception e) {
-      LOGGER.error("Failed to stop TestSecureOzoneCluster", e);
+      logger.error("Failed to stop TestSecureOzoneCluster", e);
     }
   }
 
@@ -452,7 +452,7 @@ public final class TestSecureOzoneCluster {
   private void generateKeyPair(OzoneConfiguration config) throws Exception {
     HDDSKeyGenerator keyGenerator = new HDDSKeyGenerator(conf);
     keyPair = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(config);
+    KeyCodec pemWriter = new KeyCodec(config);
     pemWriter.writeKey(keyPair, true);
   }
 
@@ -463,8 +463,6 @@ public final class TestSecureOzoneCluster {
    */
   @Test
   public void testDelegationTokenRenewal() throws Exception {
-    // Capture logs for assertions.
-    LogCapturer logs = LogCapturer.captureLogs(Server.AUDITLOG);
     GenericTestUtils
         .setLogLevel(LoggerFactory.getLogger(Server.class.getName()), INFO);
 
@@ -480,7 +478,6 @@ public final class TestSecureOzoneCluster {
     om.start();
 
     UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-    String username = ugi.getUserName();
 
     // Get first OM client which will authenticate via Kerberos
     omClient = new OzoneManagerProtocolClientSideTranslatorPB(