You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by aj...@apache.org on 2019/02/12 03:00:00 UTC

[hadoop] branch trunk updated: HDDS-1012. Add Default CertificateClient implementation. Contributed by Ajay Kumar

This is an automated email from the ASF dual-hosted git repository.

ajay pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new d48e61d  HDDS-1012. Add Default CertificateClient implementation. Contributed by Ajay Kumar
d48e61d is described below

commit d48e61dd3603417073c83ec94e5939dc94d051a7
Author: Ajay Kumar <aj...@apache.org>
AuthorDate: Mon Feb 11 18:59:47 2019 -0800

    HDDS-1012. Add Default CertificateClient implementation. Contributed by Ajay Kumar
---
 .../security/exception/SCMSecurityException.java   |   3 +-
 .../x509/certificate/client/CertificateClient.java |  71 ++-
 .../certificate/client/DNCertificateClient.java}   |  31 +-
 .../client/DefaultCertificateClient.java           | 631 +++++++++++++++++++++
 .../certificate/client/OMCertificateClient.java    | 104 ++++
 .../x509/exceptions/CertificateException.java      |  34 ++
 .../hadoop/hdds/security/x509/keys/KeyCodec.java   |  47 +-
 .../org/apache/hadoop/ozone/OzoneSecurityUtil.java |  17 +
 .../client/TestCertificateClientInit.java          | 205 +++++++
 .../client/TestDefaultCertificateClient.java       | 336 +++++++++++
 .../hadoop/ozone/TestSecureOzoneCluster.java       |   8 +-
 .../ozone/client/CertificateClientTestImpl.java    |  41 +-
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |   6 +-
 13 files changed, 1451 insertions(+), 83 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
index f545325..bbe25a9 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
@@ -73,6 +73,7 @@ public class SCMSecurityException extends IOException {
    */
   public enum ErrorCode {
     DEFAULT,
-    MISSING_BLOCK_TOKEN
+    MISSING_BLOCK_TOKEN,
+    BLOCK_TOKEN_VERIFICATION_FAILED
   }
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java
index e33c9b6..f3ad16a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/CertificateClient.java
@@ -39,28 +39,26 @@ public interface CertificateClient {
    * Returns the private key of the specified component if it exists on the
    * local system.
    *
-   * @param component - String name like DN, OM, SCM etc.
    * @return private key or Null if there is no data.
    */
-  PrivateKey getPrivateKey(String component);
+  PrivateKey getPrivateKey();
 
   /**
    * Returns the public key of the specified component if it exists on the local
    * system.
    *
-   * @param component - String name like DN, OM, SCM etc.
    * @return public key or Null if there is no data.
    */
-  PublicKey getPublicKey(String component);
+  PublicKey getPublicKey();
 
   /**
    * Returns the certificate  of the specified component if it exists on the
    * local system.
    *
-   * @param component - String name like DN, OM, SCM etc.
+
    * @return certificate or Null if there is no data.
    */
-  X509Certificate getCertificate(String component);
+  X509Certificate getCertificate();
 
   /**
    * Verifies if this certificate is part of a trusted chain.
@@ -74,13 +72,14 @@ 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)
+  byte[] signDataStream(InputStream stream)
       throws CertificateException;
 
+  byte[] signData(byte[] data) throws CertificateException;
+
   /**
    * Verifies a digital Signature, given the signature and the certificate of
    * the signer.
@@ -91,7 +90,7 @@ public interface CertificateClient {
    * @return true if verified, false if not.
    */
   boolean verifySignature(InputStream stream, byte[] signature,
-      X509Certificate cert);
+      X509Certificate cert) throws CertificateException;
 
   /**
    * Verifies a digital Signature, given the signature and the certificate of
@@ -102,7 +101,7 @@ public interface CertificateClient {
    * @return true if verified, false if not.
    */
   boolean verifySignature(byte[] data, byte[] signature,
-      X509Certificate cert);
+      X509Certificate cert) throws CertificateException;
 
   /**
    * Returns a CSR builder that can be used to creates a Certificate sigining
@@ -122,53 +121,53 @@ public interface CertificateClient {
   X509Certificate queryCertificate(String query);
 
   /**
-   * Stores the private key of a specified component.
+   * Stores the Certificate.
    *
-   * @param key - private key
-   * @param component - name of the component.
+   * @param certificate - X509 Certificate
+
    * @throws CertificateException - on Error.
    */
-  void storePrivateKey(PrivateKey key, String component)
+  void storeCertificate(X509Certificate certificate)
       throws CertificateException;
 
   /**
-   * Stores the public key of a specified component.
+   * Stores the trusted chain of certificates.
    *
-   * @param key - public key
-   * @param component - name of the component.
+   * @param certStore - Cert Store.
    * @throws CertificateException - on Error.
    */
-  void storePublicKey(PublicKey key, String component)
-      throws CertificateException;
+  void storeTrustChain(CertStore certStore) throws CertificateException;
 
   /**
-   * Stores the Certificate of a specific component.
+   * Stores the trusted chain of certificates.
    *
-   * @param certificate - X509 Certificate
-   * @param component - Name of the component.
+   * @param certificates - List of Certificates.
+
    * @throws CertificateException - on Error.
    */
-  void storeCertificate(X509Certificate certificate, String component)
+  void storeTrustChain(List<X509Certificate> certificates)
       throws CertificateException;
 
   /**
-   * Stores the trusted chain of certificates for a specific component.
+   * Initialize certificate client.
    *
-   * @param certStore - Cert Store.
-   * @param component - Trust Chain.
-   * @throws CertificateException - on Error.
-   */
-  void storeTrustChain(CertStore certStore,
-      String component) throws CertificateException;
+   * */
+  InitResponse init() throws CertificateException;
 
   /**
-   * Stores the trusted chain of certificates for a specific component.
+   * Represents initialization response of client.
+   * 1. SUCCESS: Means client is initialized successfully and all required
+   *              files are in expected state.
+   * 2. FAILURE: Initialization failed due to some unrecoverable error.
+   * 3. GETCERT: Bootstrap of keypair is successful but certificate is not
+   *             found. Client should request SCM signed certificate.
    *
-   * @param certificates - List of Certificates.
-   * @param component - String component.
-   * @throws CertificateException - on Error.
    */
-  void storeTrustChain(List<X509Certificate> certificates,
-      String component) throws CertificateException;
+  enum InitResponse {
+    SUCCESS,
+    FAILURE,
+    GETCERT,
+    RECOVER
+  }
 
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java
similarity index 51%
copy from hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java
copy to hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java
index d51d829..d41c6fe 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DNCertificateClient.java
@@ -7,37 +7,34 @@
  * "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
+ *      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.ozone;
-
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_DEFAULT;
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
+package org.apache.hadoop.hdds.security.x509.certificate.client;
 
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
 /**
- * Ozone security Util class.
+ * Certificate client for DataNodes.
  */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class OzoneSecurityUtil {
+public class DNCertificateClient extends DefaultCertificateClient {
 
-  private OzoneSecurityUtil() {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(DNCertificateClient.class);
+  DNCertificateClient(SecurityConfig securityConfig, String component) {
+    super(securityConfig, component, LOG);
   }
 
-  public static boolean isSecurityEnabled(Configuration conf) {
-    return conf.getBoolean(OZONE_SECURITY_ENABLED_KEY,
-        OZONE_SECURITY_ENABLED_DEFAULT);
+  public Logger getLogger() {
+    return LOG;
   }
-
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java
new file mode 100644
index 0000000..b407d77
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/DefaultCertificateClient.java
@@ -0,0 +1,631 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.lang3.RandomStringUtils;
+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.exceptions.CertificateException;
+import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
+import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.slf4j.Logger;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.security.InvalidKeyException;
+import java.security.KeyPair;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.Signature;
+import java.security.SignatureException;
+import java.security.cert.CertStore;
+import java.security.cert.CertificateEncodingException;
+import java.security.cert.X509Certificate;
+import java.security.spec.InvalidKeySpecException;
+import java.util.List;
+import java.util.Objects;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+import static org.apache.hadoop.hdds.security.x509.exceptions.CertificateException.ErrorCode.*;
+
+/**
+ * Default Certificate client implementation. It provides certificate
+ * operations that needs to be performed by certificate clients in the Ozone
+ * eco-system.
+ */
+public abstract class DefaultCertificateClient implements CertificateClient {
+
+  private final Logger logger;
+  private final SecurityConfig securityConfig;
+  private final String component;
+  private final KeyCodec keyCodec;
+  private PrivateKey privateKey;
+  private PublicKey publicKey;
+  private X509Certificate x509Certificate;
+
+
+  DefaultCertificateClient(SecurityConfig securityConfig, String component,
+      Logger log) {
+    Objects.requireNonNull(securityConfig);
+    Objects.requireNonNull(component);
+    this.component = component;
+    this.securityConfig = securityConfig;
+    keyCodec = new KeyCodec(securityConfig, component);
+    this.logger = log;
+  }
+
+  /**
+   * Returns the private key of the specified component if it exists on the
+   * local system.
+   *
+   * @return private key or Null if there is no data.
+   */
+  @Override
+  public PrivateKey getPrivateKey() {
+    if (privateKey != null) {
+      return privateKey;
+    }
+
+    Path keyPath = securityConfig.getKeyLocation(component);
+    if (OzoneSecurityUtil.checkIfFileExist(keyPath,
+        securityConfig.getPrivateKeyFileName())) {
+      try {
+        privateKey = keyCodec.readPrivateKey();
+      } catch (InvalidKeySpecException | NoSuchAlgorithmException
+          | IOException e) {
+        getLogger().error("Error while getting private key for {}",
+            component, e);
+      }
+    }
+    return privateKey;
+  }
+
+  /**
+   * Returns the public key of the specified component if it exists on the
+   * local system.
+   *
+   * @return public key or Null if there is no data.
+   */
+  @Override
+  public PublicKey getPublicKey() {
+    if (publicKey != null) {
+      return publicKey;
+    }
+
+    Path keyPath = securityConfig.getKeyLocation(component);
+    if (OzoneSecurityUtil.checkIfFileExist(keyPath,
+        securityConfig.getPublicKeyFileName())) {
+      try {
+        publicKey = keyCodec.readPublicKey();
+      } catch (InvalidKeySpecException | NoSuchAlgorithmException
+          | IOException e) {
+        getLogger().error("Error while getting private key for {}",
+            component, e);
+      }
+    }
+    return publicKey;
+  }
+
+  /**
+   * Returns the certificate  of the specified component if it exists on the
+   * local system.
+   *
+   * @return certificate or Null if there is no data.
+   */
+  @Override
+  public X509Certificate getCertificate() {
+    if(x509Certificate != null){
+      return x509Certificate;
+    }
+
+    Path certPath = securityConfig.getCertificateLocation(component);
+    if (OzoneSecurityUtil.checkIfFileExist(certPath,
+        securityConfig.getCertificateFileName())) {
+      CertificateCodec certificateCodec =
+          new CertificateCodec(securityConfig, component);
+      try {
+        X509CertificateHolder x509CertificateHolder =
+            certificateCodec.readCertificate();
+        x509Certificate =
+            CertificateCodec.getX509Certificate(x509CertificateHolder);
+      } catch (java.security.cert.CertificateException | IOException e) {
+        getLogger().error("Error reading certificate for {}", component, e);
+      }
+    }
+    return x509Certificate;
+  }
+
+  /**
+   * Verifies if this certificate is part of a trusted chain.
+   *
+   * @param certificate - certificate.
+   * @return true if it trusted, false otherwise.
+   */
+  @Override
+  public boolean verifyCertificate(X509Certificate certificate) {
+    throw new UnsupportedOperationException("Operation not supported.");
+  }
+
+  /**
+   * Creates digital signature over the data stream using the components
+   * private key.
+   *
+   * @param stream - Data stream to sign.
+   * @throws CertificateException - on Error.
+   */
+  @Override
+  public byte[] signDataStream(InputStream stream)
+      throws CertificateException {
+    try {
+      Signature sign = Signature.getInstance(securityConfig.getSignatureAlgo(),
+          securityConfig.getProvider());
+      sign.initSign(getPrivateKey());
+      byte[] buffer = new byte[1024 * 4];
+
+      int len;
+      while (-1 != (len = stream.read(buffer))) {
+        sign.update(buffer, 0, len);
+      }
+      return sign.sign();
+    } catch (NoSuchAlgorithmException | NoSuchProviderException
+        | InvalidKeyException | SignatureException | IOException e) {
+      getLogger().error("Error while signing the stream", e);
+      throw new CertificateException("Error while signing the stream", e,
+          CRYPTO_SIGN_ERROR);
+    }
+  }
+
+  /**
+   * Creates digital signature over the data stream using the components
+   * private key.
+   *
+   * @param data        - Data to sign.
+   * @throws CertificateException - on Error.
+   */
+  @Override
+  public byte[] signData(byte[] data) throws CertificateException {
+    try {
+      Signature sign = Signature.getInstance(securityConfig.getSignatureAlgo(),
+          securityConfig.getProvider());
+
+      sign.initSign(getPrivateKey());
+      sign.update(data);
+
+      return sign.sign();
+    } catch (NoSuchAlgorithmException | NoSuchProviderException
+        | InvalidKeyException | SignatureException e) {
+      getLogger().error("Error while signing the stream", e);
+      throw new CertificateException("Error while signing the stream", e,
+          CRYPTO_SIGN_ERROR);
+    }
+  }
+
+  /**
+   * 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.
+   * @return true if verified, false if not.
+   */
+  @Override
+  public boolean verifySignature(InputStream stream, byte[] signature,
+      X509Certificate cert) throws CertificateException {
+    try {
+      Signature sign = Signature.getInstance(securityConfig.getSignatureAlgo(),
+          securityConfig.getProvider());
+      sign.initVerify(cert);
+      byte[] buffer = new byte[1024 * 4];
+
+      int len;
+      while (-1 != (len = stream.read(buffer))) {
+        sign.update(buffer, 0, len);
+      }
+      return sign.verify(signature);
+    } catch (NoSuchAlgorithmException | NoSuchProviderException
+        | InvalidKeyException | SignatureException | IOException e) {
+      getLogger().error("Error while signing the stream", e);
+      throw new CertificateException("Error while signing the stream", e,
+          CRYPTO_SIGNATURE_VERIFICATION_ERROR);
+    }
+  }
+
+  /**
+   * Verifies a digital Signature, given the signature and the certificate of
+   * the signer.
+   *
+   * @param data - Data in byte array.
+   * @param signature - Byte Array containing the signature.
+   * @param cert - Certificate of the Signer.
+   * @return true if verified, false if not.
+   */
+  @Override
+  public boolean verifySignature(byte[] data, byte[] signature,
+      X509Certificate cert) throws CertificateException {
+    try {
+      Signature sign = Signature.getInstance(securityConfig.getSignatureAlgo(),
+          securityConfig.getProvider());
+      sign.initVerify(cert);
+      sign.update(data);
+      return sign.verify(signature);
+    } catch (NoSuchAlgorithmException | NoSuchProviderException
+        | InvalidKeyException | SignatureException e) {
+      getLogger().error("Error while signing the stream", e);
+      throw new CertificateException("Error while signing the stream", e,
+          CRYPTO_SIGNATURE_VERIFICATION_ERROR);
+    }
+  }
+
+  /**
+   * Verifies a digital Signature, given the signature and the certificate of
+   * the signer.
+   *
+   * @param data - Data in byte array.
+   * @param signature - Byte Array containing the signature.
+   * @param pubKey - Certificate of the Signer.
+   * @return true if verified, false if not.
+   */
+  private boolean verifySignature(byte[] data, byte[] signature,
+      PublicKey pubKey) throws CertificateException {
+    try {
+      Signature sign = Signature.getInstance(securityConfig.getSignatureAlgo(),
+          securityConfig.getProvider());
+      sign.initVerify(pubKey);
+      sign.update(data);
+      return sign.verify(signature);
+    } catch (NoSuchAlgorithmException | NoSuchProviderException
+        | InvalidKeyException | SignatureException e) {
+      getLogger().error("Error while signing the stream", e);
+      throw new CertificateException("Error while signing the stream", e,
+          CRYPTO_SIGNATURE_VERIFICATION_ERROR);
+    }
+  }
+
+  /**
+   * Returns a CSR builder that can be used to creates a Certificate signing
+   * request.
+   *
+   * @return CertificateSignRequest.Builder
+   */
+  @Override
+  public CertificateSignRequest.Builder getCSRBuilder() {
+    return new CertificateSignRequest.Builder();
+  }
+
+  /**
+   * Get the certificate of well-known entity from SCM.
+   *
+   * @param query - String Query, please see the implementation for the
+   * discussion on the query formats.
+   * @return X509Certificate or null if not found.
+   */
+  @Override
+  public X509Certificate queryCertificate(String query) {
+    // TODO:
+    throw new UnsupportedOperationException("Operation not supported");
+  }
+
+  /**
+   * Stores the Certificate  for this client. Don't use this api to add
+   * trusted certificates of other components.
+   *
+   * @param certificate - X509 Certificate
+   * @throws CertificateException - on Error.
+   */
+  @Override
+  public void storeCertificate(X509Certificate certificate)
+      throws CertificateException {
+    CertificateCodec certificateCodec = new CertificateCodec(securityConfig,
+        component);
+    try {
+      certificateCodec.writeCertificate(
+          new X509CertificateHolder(certificate.getEncoded()));
+    } catch (IOException | CertificateEncodingException e) {
+      throw new CertificateException("Error while storing certificate.", e,
+          CERTIFICATE_ERROR);
+    }
+  }
+
+  /**
+   * Stores the trusted chain of certificates for a specific component.
+   *
+   * @param ks                    - Key Store.
+   * @throws CertificateException - on Error.
+   */
+  @Override
+  public synchronized void storeTrustChain(CertStore ks)
+      throws CertificateException {
+    throw new UnsupportedOperationException("Operation not supported.");
+  }
+
+
+  /**
+   * Stores the trusted chain of certificates for a specific component.
+   *
+   * @param certificates - List of Certificates.
+   * @throws CertificateException - on Error.
+   */
+  @Override
+  public synchronized void storeTrustChain(List<X509Certificate> certificates)
+      throws CertificateException {
+    throw new UnsupportedOperationException("Operation not supported.");
+  }
+
+  /**
+   * Defines 8 cases of initialization.
+   * Each case specifies objects found.
+   * 0. NONE                  Keypair as well as certificate not found.
+   * 1. CERT                  Certificate found but keypair missing.
+   * 2. PUBLIC_KEY            Public key found but private key and
+   *                          certificate is missing.
+   * 3. PUBLICKEY_CERT        Only public key and certificate is present.
+   * 4. PRIVATE_KEY           Only private key is present.
+   * 5. PRIVATEKEY_CERT       Only private key and certificate is present.
+   * 6. PUBLICKEY_PRIVATEKEY  indicates private and public key were read
+   *                          successfully from configured location but
+   *                          Certificate.
+   * 7. All                   Keypair as well as certificate is present.
+   *
+   * */
+  protected enum InitCase {
+    NONE,
+    CERT,
+    PUBLIC_KEY,
+    PUBLICKEY_CERT,
+    PRIVATE_KEY,
+    PRIVATEKEY_CERT,
+    PUBLICKEY_PRIVATEKEY,
+    ALL
+  }
+
+  /**
+   *
+   * Initializes client by performing following actions.
+   * 1. Create key dir if not created already.
+   * 2. Generates and stores a keypair.
+   * 3. Try to recover public key if private key and certificate is present
+   *    but public key is missing.
+   *
+   * Truth table:
+   *  +--------------+-----------------+--------------+----------------+
+   *  | Private Key  | Public Keys     | Certificate  |   Result       |
+   *  +--------------+-----------------+--------------+----------------+
+   *  | False  (0)   | False   (0)     | False  (0)   |   GETCERT  000 |
+   *  | False  (0)   | False   (0)     | True   (1)   |   FAILURE  001 |
+   *  | False  (0)   | True    (1)     | False  (0)   |   FAILURE  010 |
+   *  | False  (0)   | True    (1)     | True   (1)   |   FAILURE  011 |
+   *  | True   (1)   | False   (0)     | False  (0)   |   FAILURE  100 |
+   *  | True   (1)   | False   (0)     | True   (1)   |   SUCCESS  101 |
+   *  | True   (1)   | True    (1)     | False  (0)   |   GETCERT  110 |
+   *  | True   (1)   | True    (1)     | True   (1)   |   SUCCESS  111 |
+   *  +--------------+-----------------+--------------+----------------+
+   *
+   * @return InitResponse
+   * Returns FAILURE in following cases:
+   * 1. If private key is missing but public key or certificate is available.
+   * 2. If public key and certificate is missing.
+   *
+   * Returns SUCCESS in following cases:
+   * 1. If keypair as well certificate is available.
+   * 2. If private key and certificate is available and public key is
+   *    recovered successfully.
+   *
+   * Returns GETCERT in following cases:
+   * 1. First time when keypair and certificate is not available, keypair
+   *    will be generated and stored at configured location.
+   * 2. When keypair (public/private key) is available but certificate is
+   *    missing.
+   *
+   */
+  @Override
+  public synchronized InitResponse init() throws CertificateException {
+    int initCase = 0;
+    PrivateKey pvtKey= getPrivateKey();
+    PublicKey pubKey = getPublicKey();
+    X509Certificate certificate = getCertificate();
+
+    if(pvtKey != null){
+      initCase = initCase | 1<<2;
+    }
+    if(pubKey != null){
+      initCase = initCase | 1<<1;
+    }
+    if(certificate != null){
+      initCase = initCase | 1;
+    }
+    getLogger().info("Certificate client init case: {}", initCase);
+    Preconditions.checkArgument(initCase < 8, "Not a " +
+        "valid case.");
+    InitCase init = InitCase.values()[initCase];
+    return handleCase(init);
+  }
+
+  /**
+   * Default handling of each {@link InitCase}.
+   * */
+  protected InitResponse handleCase(InitCase init)
+      throws CertificateException {
+    switch (init) {
+    case NONE:
+      getLogger().info("Creating keypair for client as keypair and " +
+          "certificate not found.");
+      bootstrapClientKeys();
+      return GETCERT;
+    case CERT:
+      getLogger().error("Private key not found, while certificate is still" +
+          " present. Delete keypair and try again.");
+      return FAILURE;
+    case PUBLIC_KEY:
+      getLogger().error("Found public key but private key and certificate " +
+          "missing.");
+      return FAILURE;
+    case PRIVATE_KEY:
+      getLogger().info("Found private key but public key and certificate " +
+          "is missing.");
+      // TODO: Recovering public key from private might be possible in some
+      //  cases.
+      return FAILURE;
+    case PUBLICKEY_CERT:
+      getLogger().error("Found public key and certificate but private " +
+          "key is missing.");
+      return FAILURE;
+    case PRIVATEKEY_CERT:
+      getLogger().info("Found private key and certificate but public key" +
+          " missing.");
+      if (recoverPublicKey()) {
+        return SUCCESS;
+      } else {
+        getLogger().error("Public key recovery failed.");
+        return FAILURE;
+      }
+    case PUBLICKEY_PRIVATEKEY:
+      getLogger().info("Found private and public key but certificate is" +
+          " missing.");
+      if (validateKeyPair(getPublicKey())) {
+        return GETCERT;
+      } else {
+        getLogger().info("Keypair validation failed.");
+        return FAILURE;
+      }
+    case ALL:
+      getLogger().info("Found certificate file along with KeyPair.");
+      if (validateKeyPairAndCertificate()) {
+        return SUCCESS;
+      } else {
+        return FAILURE;
+      }
+    default:
+      getLogger().error("Unexpected case: {}, Private key:{} , " +
+          "public key:{}, certificate:{}", init,
+          ((init.ordinal() & 1 << 2) == 1), ((init.ordinal() & 1 << 1) == 1),
+          ((init.ordinal() & 1 << 0) == 1));
+      return FAILURE;
+    }
+  }
+
+  /**
+   * Validate keypair and certificate.
+   * */
+  protected boolean validateKeyPairAndCertificate() throws
+      CertificateException {
+    if (validateKeyPair(getPublicKey())) {
+      getLogger().info("Keypair validated.");
+      // TODO: Certificates cryptographic validity can be checked as well.
+      if (validateKeyPair(getCertificate().getPublicKey())) {
+        getLogger().info("Keypair validated with certificate.");
+      } else {
+        getLogger().error("Stored certificate is generated with different " +
+            "private key.");
+        return false;
+      }
+    } else {
+      getLogger().error("Keypair validation failed.");
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Tries to recover public key from certificate. Also validates recovered
+   * public key.
+   * */
+  protected boolean recoverPublicKey() throws CertificateException {
+    PublicKey pubKey = getCertificate().getPublicKey();
+    try {
+
+      if(validateKeyPair(pubKey)){
+        keyCodec.writePublicKey(pubKey);
+        publicKey = pubKey;
+      } else {
+        getLogger().error("Can't recover public key " +
+            "corresponding to private key.", BOOTSTRAP_ERROR);
+        return false;
+      }
+    } catch (IOException e) {
+      throw new CertificateException("Error while trying to recover " +
+          "public key.", e, BOOTSTRAP_ERROR);
+    }
+    return true;
+  }
+
+  /**
+   * Validates public and private key of certificate client.
+   *
+   * @param pubKey
+   * */
+  protected boolean validateKeyPair(PublicKey pubKey)
+      throws CertificateException {
+    byte[] challenge = RandomStringUtils.random(1000).getBytes();
+    byte[]  sign = signDataStream(new ByteArrayInputStream(challenge));
+    return verifySignature(challenge, sign, pubKey);
+  }
+
+  /**
+   * Bootstrap the client by creating keypair and storing it in configured
+   * location.
+   * */
+  protected void bootstrapClientKeys() throws CertificateException {
+    Path keyPath = securityConfig.getKeyLocation(component);
+    if (Files.notExists(keyPath)) {
+      try {
+        Files.createDirectories(keyPath);
+      } catch (IOException e) {
+        throw new CertificateException("Error while creating directories " +
+            "for certificate storage.", BOOTSTRAP_ERROR);
+      }
+    }
+    KeyPair keyPair = createKeyPair();
+    privateKey = keyPair.getPrivate();
+    publicKey = keyPair.getPublic();
+  }
+
+  protected KeyPair createKeyPair() throws CertificateException {
+    HDDSKeyGenerator keyGenerator = new HDDSKeyGenerator(securityConfig);
+    KeyPair keyPair = null;
+    try {
+      keyPair = keyGenerator.generateKey();
+      keyCodec.writePublicKey(keyPair.getPublic());
+      keyCodec.writePrivateKey(keyPair.getPrivate());
+    } catch (NoSuchProviderException | NoSuchAlgorithmException
+        | IOException e) {
+      getLogger().error("Error while bootstrapping certificate client for {}",
+          component, e);
+      throw new CertificateException("Error while bootstrapping certificate " +
+          "client for" + component, BOOTSTRAP_ERROR);
+    }
+    return keyPair;
+  }
+
+  public Logger getLogger() {
+    return logger;
+  }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/OMCertificateClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/OMCertificateClient.java
new file mode 100644
index 0000000..bddcb37
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/OMCertificateClient.java
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+
+/**
+ * Certificate client for OzoneManager.
+ */
+public class OMCertificateClient extends DefaultCertificateClient {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OMCertificateClient.class);
+
+  OMCertificateClient(SecurityConfig securityConfig, String component) {
+    super(securityConfig, component, LOG);
+  }
+
+  protected InitResponse handleCase(InitCase init) throws
+      CertificateException {
+    switch (init) {
+    case NONE:
+      LOG.info("Creating keypair for client as keypair and certificate not " +
+          "found.");
+      bootstrapClientKeys();
+      return GETCERT;
+    case CERT:
+      LOG.error("Private key not found, while certificate is still present." +
+          "Delete keypair and try again.");
+      return FAILURE;
+    case PUBLIC_KEY:
+      LOG.error("Found public key but private key and certificate missing.");
+      return FAILURE;
+    case PRIVATE_KEY:
+      LOG.info("Found private key but public key and certificate is missing.");
+      // TODO: Recovering public key from private might be possible in some
+      //  cases.
+      return FAILURE;
+    case PUBLICKEY_CERT:
+      LOG.error("Found public key and certificate but private key is " +
+          "missing.");
+      return FAILURE;
+    case PRIVATEKEY_CERT:
+      LOG.info("Found private key and certificate but public key missing.");
+      if (recoverPublicKey()) {
+        return SUCCESS;
+      } else {
+        LOG.error("Public key recovery failed.");
+        return FAILURE;
+      }
+    case PUBLICKEY_PRIVATEKEY:
+      LOG.info("Found private and public key but certificate is missing.");
+      if (validateKeyPair(getPublicKey())) {
+        return RECOVER;
+      } else {
+        LOG.error("Keypair validation failed.");
+        return FAILURE;
+      }
+    case ALL:
+      LOG.info("Found certificate file along with KeyPair.");
+      if (validateKeyPairAndCertificate()) {
+        return SUCCESS;
+      } else {
+        return FAILURE;
+      }
+    default:
+      LOG.error("Unexpected case: {}, Private key:{} , " +
+              "public key:{}, certificate:{}", init,
+          ((init.ordinal() & 1 << 2) == 1), ((init.ordinal() & 1 << 1) == 1),
+          ((init.ordinal() & 1 << 0) == 1));
+      return FAILURE;
+    }
+  }
+
+  public Logger getLogger() {
+    return LOG;
+  }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java
index 5a8bf86..bbdbec7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
  */
 public class CertificateException extends SCMSecurityException {
 
+  private ErrorCode errorCode;
   /**
    * Ctor.
    * @param message - Error Message.
@@ -45,9 +46,42 @@ public class CertificateException extends SCMSecurityException {
 
   /**
    * Ctor.
+   * @param message - Message.
+   * @param cause  - Actual cause.
+   * @param errorCode
+   */
+  public CertificateException(String message, Throwable cause,
+      ErrorCode errorCode) {
+    super(message, cause);
+    this.errorCode = errorCode;
+  }
+
+  /**
+   * Ctor.
+   * @param message - Message.
+   * @param errorCode
+   */
+  public CertificateException(String message, ErrorCode errorCode) {
+    super(message);
+    this.errorCode = errorCode;
+  }
+
+  /**
+   * Ctor.
    * @param cause - Base Exception.
    */
   public CertificateException(Throwable cause) {
     super(cause);
   }
+
+  /**
+   * Error codes to make it easy to decode these exceptions.
+   */
+  public enum ErrorCode {
+    KEYSTORE_ERROR,
+    CRYPTO_SIGN_ERROR,
+    CERTIFICATE_ERROR,
+    BOOTSTRAP_ERROR,
+    CRYPTO_SIGNATURE_VERIFICATION_ERROR
+  }
 }
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java
index efd8d74..583c5d6 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/KeyCodec.java
@@ -151,6 +151,51 @@ public class KeyCodec {
   }
 
   /**
+   * Writes a given private key using the default config options.
+   *
+   * @param key - Key to write to file.
+   * @throws IOException - On I/O failure.
+   */
+  public void writePrivateKey(PrivateKey key) throws IOException {
+    File privateKeyFile =
+        Paths.get(location.toString(),
+            securityConfig.getPrivateKeyFileName()).toFile();
+
+    if (Files.exists(privateKeyFile.toPath())) {
+      throw new IOException("Private key already exist.");
+    }
+
+    try (PemWriter privateKeyWriter = new PemWriter(new
+        FileWriterWithEncoding(privateKeyFile, DEFAULT_CHARSET))) {
+      privateKeyWriter.writeObject(
+          new PemObject(PRIVATE_KEY, key.getEncoded()));
+    }
+    Files.setPosixFilePermissions(privateKeyFile.toPath(), permissionSet);
+  }
+
+  /**
+   * Writes a given public key using the default config options.
+   *
+   * @param key - Key to write to file.
+   * @throws IOException - On I/O failure.
+   */
+  public void writePublicKey(PublicKey key) throws IOException {
+    File publicKeyFile = Paths.get(location.toString(),
+        securityConfig.getPublicKeyFileName()).toFile();
+
+    if (Files.exists(publicKeyFile.toPath())) {
+      throw new IOException("Private key already exist.");
+    }
+
+    try (PemWriter keyWriter = new PemWriter(new
+        FileWriterWithEncoding(publicKeyFile, DEFAULT_CHARSET))) {
+      keyWriter.writeObject(
+          new PemObject(PUBLIC_KEY, key.getEncoded()));
+    }
+    Files.setPosixFilePermissions(publicKeyFile.toPath(), permissionSet);
+  }
+
+  /**
    * Writes a given key using default config options.
    *
    * @param keyPair - Key pair to write
@@ -275,7 +320,7 @@ public class KeyCodec {
    * @throws IOException - On I/O failure.
    */
   private synchronized void writeKey(Path basePath, KeyPair keyPair,
-                                     String privateKeyFileName, String publicKeyFileName, boolean force)
+      String privateKeyFileName, String publicKeyFileName, boolean force)
       throws IOException {
     checkPreconditions(basePath);
 
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java
index d51d829..c1f2906 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneSecurityUtil.java
@@ -25,6 +25,10 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+
 /**
  * Ozone security Util class.
  */
@@ -40,4 +44,17 @@ public final class OzoneSecurityUtil {
         OZONE_SECURITY_ENABLED_DEFAULT);
   }
 
+  /**
+   * Returns Keys status.
+   *
+   * @return True if the key files exist.
+   */
+  public static boolean checkIfFileExist(Path path, String fileName) {
+    if (Files.exists(path) && Files.exists(Paths.get(path.toString(),
+        fileName))) {
+      return true;
+    }
+    return false;
+  }
+
 }
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestCertificateClientInit.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestCertificateClientInit.java
new file mode 100644
index 0000000..e076267
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestCertificateClientInit.java
@@ -0,0 +1,205 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+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.keys.HDDSKeyGenerator;
+import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
+import org.apache.hadoop.ozone.OzoneSecurityUtil;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameter;
+
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.security.KeyPair;
+import java.security.cert.X509Certificate;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_METADATA_DIR_NAME;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for {@link DefaultCertificateClient}.
+ */
+@RunWith(Parameterized.class)
+public class TestCertificateClientInit {
+
+  private CertificateClient dnCertificateClient;
+  private CertificateClient omCertificateClient;
+  private static final String COMP = "test";
+  private HDDSKeyGenerator keyGenerator;
+  private Path metaDirPath;
+  private SecurityConfig securityConfig;
+  private KeyCodec keyCodec;
+
+  @Parameter
+  public boolean pvtKeyPresent;
+  @Parameter(1)
+  public boolean pubKeyPresent;
+  @Parameter(2)
+  public boolean certPresent;
+  @Parameter(3)
+  public InitResponse expectedResult;
+
+  @Parameterized.Parameters
+  public static Collection<Object[]> initData() {
+    return Arrays.asList(new Object[][]{
+        {false, false, false, GETCERT},
+        {false, false, true, FAILURE},
+        {false, true, false, FAILURE},
+        {true, false, false, FAILURE},
+        {false, true, true, FAILURE},
+        {true, true, false, GETCERT},
+        {true, false, true, SUCCESS},
+        {true, true, true, SUCCESS}});
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    OzoneConfiguration config = new OzoneConfiguration();
+    final String path = GenericTestUtils
+        .getTempPath(UUID.randomUUID().toString());
+    metaDirPath = Paths.get(path, "test");
+    config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
+    securityConfig = new SecurityConfig(config);
+    dnCertificateClient = new DNCertificateClient(securityConfig, COMP);
+    omCertificateClient = new OMCertificateClient(securityConfig, COMP);
+    keyGenerator = new HDDSKeyGenerator(securityConfig);
+    keyCodec = new KeyCodec(securityConfig, COMP);
+    Files.createDirectories(securityConfig.getKeyLocation(COMP));
+  }
+
+  @After
+  public void tearDown() {
+    dnCertificateClient = null;
+    omCertificateClient = null;
+    FileUtils.deleteQuietly(metaDirPath.toFile());
+  }
+
+
+  @Test
+  public void testInitDatanode() throws Exception {
+    KeyPair keyPair = keyGenerator.generateKey();
+    if (pvtKeyPresent) {
+      keyCodec.writePrivateKey(keyPair.getPrivate());
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getPrivateKeyFileName()).toFile());
+    }
+
+    if (pubKeyPresent) {
+      if (dnCertificateClient.getPublicKey() == null) {
+        keyCodec.writePublicKey(keyPair.getPublic());
+      }
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getPublicKeyFileName()).toFile());
+    }
+
+    if (certPresent) {
+      X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
+          "CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
+
+      CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
+      codec.writeCertificate(new X509CertificateHolder(
+          x509Certificate.getEncoded()));
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getCertificateFileName()).toFile());
+    }
+    InitResponse response = dnCertificateClient.init();
+
+    assertTrue(response.equals(expectedResult));
+
+    if (!response.equals(FAILURE)) {
+      assertTrue(OzoneSecurityUtil.checkIfFileExist(
+          securityConfig.getKeyLocation(COMP),
+          securityConfig.getPrivateKeyFileName()));
+      assertTrue(OzoneSecurityUtil.checkIfFileExist(
+          securityConfig.getKeyLocation(COMP),
+          securityConfig.getPublicKeyFileName()));
+    }
+  }
+
+  @Test
+  public void testInitOzoneManager() throws Exception {
+    KeyPair keyPair = keyGenerator.generateKey();
+    if (pvtKeyPresent) {
+      keyCodec.writePrivateKey(keyPair.getPrivate());
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getPrivateKeyFileName()).toFile());
+    }
+
+    if (pubKeyPresent) {
+      if (omCertificateClient.getPublicKey() == null) {
+        keyCodec.writePublicKey(keyPair.getPublic());
+      }
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getPublicKeyFileName()).toFile());
+    }
+
+    if (certPresent) {
+      X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
+          "CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
+
+      CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
+      codec.writeCertificate(new X509CertificateHolder(
+          x509Certificate.getEncoded()));
+    } else {
+      FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+          .toString(), securityConfig.getCertificateFileName()).toFile());
+    }
+    InitResponse response = omCertificateClient.init();
+
+    if (pvtKeyPresent && pubKeyPresent & !certPresent) {
+      assertTrue(response.equals(RECOVER));
+    } else {
+      assertTrue(response.equals(expectedResult));
+    }
+
+    if (!response.equals(FAILURE)) {
+      assertTrue(OzoneSecurityUtil.checkIfFileExist(
+          securityConfig.getKeyLocation(COMP),
+          securityConfig.getPrivateKeyFileName()));
+      assertTrue(OzoneSecurityUtil.checkIfFileExist(
+          securityConfig.getKeyLocation(COMP),
+          securityConfig.getPublicKeyFileName()));
+    }
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java
new file mode 100644
index 0000000..7a3f4f2
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/client/TestDefaultCertificateClient.java
@@ -0,0 +1,336 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+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.PrivateKey;
+import java.security.PublicKey;
+import java.security.Signature;
+import java.security.cert.X509Certificate;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.lang3.RandomStringUtils;
+
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_METADATA_DIR_NAME;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test class for {@link DefaultCertificateClient}.
+ */
+public class TestDefaultCertificateClient {
+
+  private OMCertificateClient omCertClient;
+  private DNCertificateClient dnCertClient;
+  private static final String COMP = "test";
+  private HDDSKeyGenerator keyGenerator;
+  private Path metaDirPath;
+  private SecurityConfig securityConfig;
+  private final static String UTF = "UTF-8";
+  private KeyCodec keyCodec;
+
+  @Before
+  public void setUp() throws Exception {
+    OzoneConfiguration config = new OzoneConfiguration();
+    final String path = GenericTestUtils
+        .getTempPath(UUID.randomUUID().toString());
+    metaDirPath = Paths.get(path, "test");
+    config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
+    securityConfig = new SecurityConfig(config);
+    getCertClient();
+    keyGenerator = new HDDSKeyGenerator(securityConfig);
+    keyCodec = new KeyCodec(securityConfig, COMP);
+    Files.createDirectories(securityConfig.getKeyLocation(COMP));
+  }
+
+  private void getCertClient() {
+    omCertClient = new OMCertificateClient(securityConfig, COMP);
+    dnCertClient = new DNCertificateClient(securityConfig, COMP);
+  }
+
+  @After
+  public void tearDown() {
+    omCertClient = null;
+    FileUtils.deleteQuietly(metaDirPath.toFile());
+  }
+
+  /**
+   * Tests: 1. getPrivateKey 2. getPublicKey 3. storePrivateKey 4.
+   * storePublicKey
+   */
+  @Test
+  public void testKeyOperations() throws Exception {
+    PrivateKey pvtKey = omCertClient.getPrivateKey();
+    PublicKey publicKey = omCertClient.getPublicKey();
+    assertNull(publicKey);
+    assertNull(pvtKey);
+
+    KeyPair keyPair = generateKeyPairFiles();
+    pvtKey = omCertClient.getPrivateKey();
+    assertNotNull(pvtKey);
+    assertEquals(pvtKey, keyPair.getPrivate());
+
+    publicKey = omCertClient.getPublicKey();
+    assertNotNull(publicKey);
+    assertEquals(publicKey, keyPair.getPublic());
+  }
+
+  private KeyPair generateKeyPairFiles() throws Exception {
+    KeyPair keyPair = keyGenerator.generateKey();
+    keyCodec.writePrivateKey(keyPair.getPrivate());
+    keyCodec.writePublicKey(keyPair.getPublic());
+    return keyPair;
+  }
+
+  /**
+   * Tests: 1. storeCertificate 2. getCertificate 3. verifyCertificate
+   */
+  @Test
+  public void testCertificateOps() throws Exception {
+    X509Certificate cert = omCertClient.getCertificate();
+    assertNull(cert);
+
+    X509Certificate x509Certificate = generateX509Cert(null);
+    omCertClient.storeCertificate(x509Certificate);
+
+    cert = omCertClient.getCertificate();
+    assertNotNull(cert);
+    assertTrue(cert.getEncoded().length > 0);
+    assertEquals(cert, x509Certificate);
+
+    // TODO: test verifyCertificate once implemented.
+  }
+
+  private X509Certificate generateX509Cert(KeyPair keyPair) throws Exception {
+    if (keyPair == null) {
+      keyPair = generateKeyPairFiles();
+    }
+    return KeyStoreTestUtil.generateCertificate("CN=Test", keyPair, 30,
+        securityConfig.getSignatureAlgo());
+  }
+
+  @Test
+  public void testSignDataStream() throws Exception {
+    String data = RandomStringUtils.random(100);
+    // Expect error when there is no private key to sign.
+    LambdaTestUtils.intercept(IOException.class, "Error while " +
+            "signing the stream",
+        () -> omCertClient.signDataStream(IOUtils.toInputStream(data,
+            UTF)));
+
+    KeyPair keyPair = generateKeyPairFiles();
+    byte[] sign = omCertClient.signDataStream(IOUtils.toInputStream(data,
+        UTF));
+    validateHash(sign, data.getBytes());
+  }
+
+  /**
+   * Validate hash using public key of KeyPair.
+   */
+  private void validateHash(byte[] hash, byte[] data)
+      throws Exception {
+    Signature rsaSignature =
+        Signature.getInstance(securityConfig.getSignatureAlgo(),
+            securityConfig.getProvider());
+    rsaSignature.initVerify(omCertClient.getPublicKey());
+    rsaSignature.update(data);
+    Assert.assertTrue(rsaSignature.verify(hash));
+  }
+
+  /**
+   * Tests: 1. verifySignature
+   */
+  @Test
+  public void verifySignatureStream() throws Exception {
+    String data = RandomStringUtils.random(500, UTF);
+
+    X509Certificate x509Certificate = generateX509Cert(null);
+    byte[] sign = omCertClient.signDataStream(IOUtils.toInputStream(data,
+        UTF));
+
+    // Positive tests.
+    assertTrue(omCertClient.verifySignature(data.getBytes(), sign,
+        x509Certificate));
+    assertTrue(omCertClient.verifySignature(IOUtils.toInputStream(data, UTF),
+        sign, x509Certificate));
+
+    // Negative tests.
+    assertFalse(omCertClient.verifySignature(data.getBytes(),
+        "abc".getBytes(), x509Certificate));
+    assertFalse(omCertClient.verifySignature(IOUtils.toInputStream(data,
+        UTF), "abc".getBytes(), x509Certificate));
+
+  }
+
+  /**
+   * Tests: 1. verifySignature
+   */
+  @Test
+  public void verifySignatureDataArray() throws Exception {
+    String data = RandomStringUtils.random(500, UTF);
+    X509Certificate x509Certificate = generateX509Cert(null);
+    byte[] sign = omCertClient.signData(data.getBytes());
+
+    // Positive tests.
+    assertTrue(omCertClient.verifySignature(data.getBytes(), sign,
+        x509Certificate));
+    assertTrue(omCertClient.verifySignature(IOUtils.toInputStream(data, UTF),
+        sign, x509Certificate));
+
+    // Negative tests.
+    assertFalse(omCertClient.verifySignature(data.getBytes(),
+        "abc".getBytes(), x509Certificate));
+    assertFalse(omCertClient.verifySignature(IOUtils.toInputStream(data,
+        UTF), "abc".getBytes(), x509Certificate));
+
+  }
+
+  @Test
+  public void queryCertificate() throws Exception {
+    LambdaTestUtils.intercept(UnsupportedOperationException.class,
+        "Operation not supported",
+        () -> omCertClient.queryCertificate(""));
+  }
+
+  @Test
+  public void testInitCertAndKeypairValidationFailures() throws Exception {
+
+    GenericTestUtils.LogCapturer dnClientLog = GenericTestUtils.LogCapturer
+        .captureLogs(dnCertClient.getLogger());
+    GenericTestUtils.LogCapturer omClientLog = GenericTestUtils.LogCapturer
+        .captureLogs(omCertClient.getLogger());
+    KeyPair keyPair = keyGenerator.generateKey();
+    KeyPair keyPair2 = keyGenerator.generateKey();
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Case 1. Expect failure when keypair validation fails.
+    FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+        .toString(), securityConfig.getPrivateKeyFileName()).toFile());
+    keyCodec.writePrivateKey(keyPair.getPrivate());
+
+    FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+        .toString(), securityConfig.getPublicKeyFileName()).toFile());
+    keyCodec.writePublicKey(keyPair2.getPublic());
+
+    // Check for DN.
+    assertEquals(dnCertClient.init(), FAILURE);
+    assertTrue(dnClientLog.getOutput().contains("Keypair validation " +
+        "failed"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Check for OM.
+    assertEquals(omCertClient.init(), FAILURE);
+    assertTrue(omClientLog.getOutput().contains("Keypair validation " +
+        "failed"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Case 2. Expect failure when certificate is generated from different
+    // private key and keypair validation fails.
+    getCertClient();
+    FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+        .toString(), securityConfig.getCertificateFileName()).toFile());
+    X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
+        "CN=Test", keyGenerator.generateKey(), 10,
+        securityConfig.getSignatureAlgo());
+    CertificateCodec codec = new CertificateCodec(securityConfig, COMP);
+    codec.writeCertificate(new X509CertificateHolder(
+        x509Certificate.getEncoded()));
+
+    // Check for DN.
+    assertEquals(dnCertClient.init(), FAILURE);
+    assertTrue(dnClientLog.getOutput().contains("Keypair validation " +
+        "failed"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Check for OM.
+    assertEquals(omCertClient.init(), FAILURE);
+    assertTrue(omClientLog.getOutput().contains("Keypair validation failed"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Case 3. Expect failure when certificate is generated from different
+    // private key and certificate validation fails.
+
+    // Re write the correct public key.
+    getCertClient();
+    FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+        .toString(), securityConfig.getPublicKeyFileName()).toFile());
+    keyCodec.writePublicKey(keyPair.getPublic());
+
+    // Check for DN.
+    assertEquals(dnCertClient.init(), FAILURE);
+    assertTrue(dnClientLog.getOutput().contains("Stored certificate is " +
+        "generated with different"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    //Check for OM.
+    assertEquals(omCertClient.init(), FAILURE);
+    assertTrue(omClientLog.getOutput().contains("Stored certificate is " +
+        "generated with different"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+
+    // Case 4. Failure when public key recovery fails.
+    getCertClient();
+    FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation(COMP)
+        .toString(), securityConfig.getPublicKeyFileName()).toFile());
+
+    // Check for DN.
+    assertEquals(dnCertClient.init(), FAILURE);
+    assertTrue(dnClientLog.getOutput().contains("Can't recover public key"));
+
+    // Check for OM.
+    assertEquals(omCertClient.init(), FAILURE);
+    assertTrue(omClientLog.getOutput().contains("Can't recover public key"));
+    dnClientLog.clearOutput();
+    omClientLog.clearOutput();
+  }
+
+}
\ No newline at end of file
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
index f98ee95..d54da2b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.minikdc.MiniKdc;
 import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.ozone.client.CertificateClientTestImpl;
 import org.apache.hadoop.ozone.om.OMConfigKeys;
 import org.apache.hadoop.ozone.om.OMStorage;
 import org.apache.hadoop.ozone.om.OzoneManager;
@@ -78,7 +79,6 @@ import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.Timeout;
-import org.mockito.Mockito;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -583,11 +583,7 @@ public final class TestSecureOzoneCluster {
     omStore.initialize();
     OzoneManager.setTestSecureOmFlag(true);
     om = OzoneManager.createOm(null, config);
-    CertificateClient certClient = Mockito.mock(CertificateClient.class);
-    Mockito.when(certClient.getPrivateKey("om"))
-        .thenReturn(keyPair.getPrivate());
-    Mockito.when(certClient.getPublicKey("om"))
-        .thenReturn(keyPair.getPublic());
+    CertificateClient certClient = new CertificateClientTestImpl(config);
     om.setCertClient(certClient);
   }
 
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/CertificateClientTestImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/CertificateClientTestImpl.java
index ceb0200..bb5d44b 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/CertificateClientTestImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/CertificateClientTestImpl.java
@@ -69,17 +69,17 @@ public class CertificateClientTestImpl implements CertificateClient {
   }
 
   @Override
-  public PrivateKey getPrivateKey(String component) {
+  public PrivateKey getPrivateKey() {
     return keyPair.getPrivate();
   }
 
   @Override
-  public PublicKey getPublicKey(String component) {
+  public PublicKey getPublicKey() {
     return keyPair.getPublic();
   }
 
   @Override
-  public X509Certificate getCertificate(String component) {
+  public X509Certificate getCertificate() {
     return cert;
   }
 
@@ -89,20 +89,25 @@ public class CertificateClientTestImpl implements CertificateClient {
   }
 
   @Override
-  public byte[] signDataStream(InputStream stream, String component)
+  public byte[] signDataStream(InputStream stream)
       throws CertificateException {
     return new byte[0];
   }
 
   @Override
+  public byte[] signData(byte[] data) throws CertificateException {
+    return new byte[0];
+  }
+
+  @Override
   public boolean verifySignature(InputStream stream, byte[] signature,
-      X509Certificate x509Certificate) {
+      X509Certificate x509Certificate) throws CertificateException {
     return true;
   }
 
   @Override
   public boolean verifySignature(byte[] data, byte[] signature,
-      X509Certificate x509Certificate) {
+      X509Certificate x509Certificate) throws CertificateException {
     return true;
   }
 
@@ -117,32 +122,30 @@ public class CertificateClientTestImpl implements CertificateClient {
   }
 
   @Override
-  public void storePrivateKey(PrivateKey key, String component)
-      throws CertificateException {
-
-  }
-
-  @Override
-  public void storePublicKey(PublicKey key, String component)
+  public void storeCertificate(X509Certificate certificate)
       throws CertificateException {
 
   }
 
+  /**
+   * Stores the trusted chain of certificates for a specific component.
+   *
+   * @param keyStore - Cert Store.
+   * @throws CertificateException - on Error.
+   */
   @Override
-  public void storeCertificate(X509Certificate certificate, String component)
-      throws CertificateException {
+  public void storeTrustChain(CertStore keyStore) throws CertificateException {
 
   }
 
   @Override
-  public void storeTrustChain(CertStore certStore, String component)
+  public void storeTrustChain(List<X509Certificate> certificates)
       throws CertificateException {
 
   }
 
   @Override
-  public void storeTrustChain(List<X509Certificate> certificates,
-      String component) throws CertificateException {
-
+  public InitResponse init() throws CertificateException {
+    return null;
   }
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 5b0ff76..13bfd98 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -397,7 +397,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     }
     Objects.requireNonNull(certClient);
     return new OzoneBlockTokenSecretManager(secConfig, expiryTime,
-        certClient.getCertificate(OM_DAEMON).getSerialNumber().toString());
+        certClient.getCertificate().getSerialNumber().toString());
   }
 
   private void stopSecretManager() {
@@ -464,8 +464,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
    */
   private void readKeyPair() throws OzoneSecurityException {
     try {
-      keyPair = new KeyPair(certClient.getPublicKey(OM_DAEMON),
-          certClient.getPrivateKey(OM_DAEMON));
+      keyPair = new KeyPair(certClient.getPublicKey(),
+          certClient.getPrivateKey());
     } catch (Exception e) {
       throw new OzoneSecurityException("Error reading private file for "
           + "OzoneManager", e, OzoneSecurityException


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org