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 2018/09/28 13:53:14 UTC

[1/2] hadoop git commit: HDDS-548. Create a Self-Signed Certificate. Contributed by Anu Engineer.

Repository: hadoop
Updated Branches:
  refs/heads/HDDS-4 fe85d5157 -> 6f5bc5a4c


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyPEMWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyPEMWriter.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyPEMWriter.java
new file mode 100644
index 0000000..db5d430
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyPEMWriter.java
@@ -0,0 +1,216 @@
+/*
+ * 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.keys;
+
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_METADATA_DIR_NAME;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+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.NoSuchProviderException;
+import java.security.PrivateKey;
+import java.security.PublicKey;
+import java.security.spec.InvalidKeySpecException;
+import java.security.spec.PKCS8EncodedKeySpec;
+import java.security.spec.X509EncodedKeySpec;
+import java.util.Set;
+import org.apache.commons.codec.binary.Base64;
+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.test.LambdaTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Test class for HDDS pem writer.
+ */
+public class TestHDDSKeyPEMWriter {
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+  private OzoneConfiguration configuration;
+  private HDDSKeyGenerator keyGenerator;
+  private String prefix;
+
+  @Before
+  public void init() throws IOException {
+    configuration = new OzoneConfiguration();
+    prefix = temporaryFolder.newFolder().toString();
+    configuration.set(HDDS_METADATA_DIR_NAME, prefix);
+    keyGenerator = new HDDSKeyGenerator(configuration);
+  }
+
+  /**
+   * Assert basic things like we are able to create a file, and the names are
+   * in expected format etc.
+   *
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   * @throws IOException - On I/O failure.
+   */
+  @Test
+  public void testWriteKey()
+      throws NoSuchProviderException, NoSuchAlgorithmException,
+      IOException, InvalidKeySpecException {
+    KeyPair keys = keyGenerator.generateKey();
+    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    pemWriter.writeKey(keys);
+
+    // Assert that locations have been created.
+    Path keyLocation = pemWriter.getSecurityConfig().getKeyLocation();
+    Assert.assertTrue(keyLocation.toFile().exists());
+
+    // Assert that locations are created in the locations that we specified
+    // using the Config.
+    Assert.assertTrue(keyLocation.toString().startsWith(prefix));
+    Path privateKeyPath = Paths.get(keyLocation.toString(),
+        pemWriter.getSecurityConfig().getPrivateKeyFileName());
+    Assert.assertTrue(privateKeyPath.toFile().exists());
+    Path publicKeyPath = Paths.get(keyLocation.toString(),
+        pemWriter.getSecurityConfig().getPublicKeyFileName());
+    Assert.assertTrue(publicKeyPath.toFile().exists());
+
+    // Read the private key and test if the expected String in the PEM file
+    // format exists.
+    byte[] privateKey = Files.readAllBytes(privateKeyPath);
+    String privateKeydata = new String(privateKey, StandardCharsets.UTF_8);
+    Assert.assertTrue(privateKeydata.contains("PRIVATE KEY"));
+
+    // Read the public key and test if the expected String in the PEM file
+    // format exists.
+    byte[] publicKey = Files.readAllBytes(publicKeyPath);
+    String publicKeydata = new String(publicKey, StandardCharsets.UTF_8);
+    Assert.assertTrue(publicKeydata.contains("PUBLIC KEY"));
+
+    // Let us decode the PEM file and parse it back into binary.
+    KeyFactory kf = KeyFactory.getInstance(
+        pemWriter.getSecurityConfig().getKeyAlgo());
+
+    // Replace the PEM Human readable guards.
+    privateKeydata =
+        privateKeydata.replace("-----BEGIN PRIVATE KEY-----\n", "");
+    privateKeydata =
+        privateKeydata.replace("-----END PRIVATE KEY-----", "");
+
+    // Decode the bas64 to binary format and then use an ASN.1 parser to
+    // parse the binary format.
+
+    byte[] keyBytes = Base64.decodeBase64(privateKeydata);
+    PKCS8EncodedKeySpec spec = new PKCS8EncodedKeySpec(keyBytes);
+    PrivateKey privateKeyDecoded = kf.generatePrivate(spec);
+    Assert.assertNotNull("Private Key should not be null",
+        privateKeyDecoded);
+
+    // Let us decode the public key and veriy that we can parse it back into
+    // binary.
+    publicKeydata =
+        publicKeydata.replace("-----BEGIN PUBLIC KEY-----\n", "");
+    publicKeydata =
+        publicKeydata.replace("-----END PUBLIC KEY-----", "");
+
+    keyBytes = Base64.decodeBase64(publicKeydata);
+    X509EncodedKeySpec pubKeyspec = new X509EncodedKeySpec(keyBytes);
+    PublicKey publicKeyDecoded = kf.generatePublic(pubKeyspec);
+    Assert.assertNotNull("Public Key should not be null",
+        publicKeyDecoded);
+
+    // Now let us assert the permissions on the Directories and files are as
+    // expected.
+    Set<PosixFilePermission> expectedSet = pemWriter.getPermissionSet();
+    Set<PosixFilePermission> currentSet =
+        Files.getPosixFilePermissions(privateKeyPath);
+    currentSet.removeAll(expectedSet);
+    Assert.assertEquals(0, currentSet.size());
+
+    currentSet =
+        Files.getPosixFilePermissions(publicKeyPath);
+    currentSet.removeAll(expectedSet);
+    Assert.assertEquals(0, currentSet.size());
+
+    currentSet =
+        Files.getPosixFilePermissions(keyLocation);
+    currentSet.removeAll(expectedSet);
+    Assert.assertEquals(0, currentSet.size());
+  }
+
+  /**
+   * Assert key rewrite fails without force option.
+   *
+   * @throws IOException - on I/O failure.
+   */
+  @Test
+  public void testReWriteKey()
+      throws Exception {
+    KeyPair kp = keyGenerator.generateKey();
+    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    SecurityConfig secConfig = pemWriter.getSecurityConfig();
+    pemWriter.writeKey(kp);
+
+    // Assert that rewriting of keys throws exception with valid messages.
+    LambdaTestUtils
+        .intercept(IOException.class, "Private Key file already exists.",
+            () -> pemWriter.writeKey(kp));
+    FileUtils.deleteQuietly(Paths.get(
+        secConfig.getKeyLocation().toString() + "/" + secConfig
+            .getPrivateKeyFileName()).toFile());
+    LambdaTestUtils
+        .intercept(IOException.class, "Public Key file already exists.",
+            () -> pemWriter.writeKey(kp));
+    FileUtils.deleteQuietly(Paths.get(
+        secConfig.getKeyLocation().toString() + "/" + secConfig
+            .getPublicKeyFileName()).toFile());
+
+    // Should succeed now as both public and private key are deleted.
+    pemWriter.writeKey(kp);
+    // Should succeed with overwrite flag as true.
+    pemWriter.writeKey(kp, true);
+
+  }
+
+  /**
+   * Assert key rewrite fails in non Posix file system.
+   *
+   * @throws IOException - on I/O failure.
+   */
+  @Test
+  public void testWriteKeyInNonPosixFS()
+      throws Exception {
+    KeyPair kp = keyGenerator.generateKey();
+    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
+    pemWriter.setIsPosixFileSystem(() -> false);
+
+    // Assert key rewrite fails in non Posix file system.
+    LambdaTestUtils
+        .intercept(IOException.class, "Unsupported File System for pem file.",
+            () -> pemWriter.writeKey(kp));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java
new file mode 100644
index 0000000..49e40b4
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/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.
+ *
+ */
+/**
+ * Test package for keys used in X.509 env.
+ */
+package org.apache.hadoop.hdds.security.x509.keys;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/package-info.java
new file mode 100644
index 0000000..f541468
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/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.
+ *
+ */
+/**
+ * X.509 Certificate and keys related tests.
+ */
+package org.apache.hadoop.hdds.security.x509;
\ No newline at end of file


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


[2/2] hadoop git commit: HDDS-548. Create a Self-Signed Certificate. Contributed by Anu Engineer.

Posted by aj...@apache.org.
HDDS-548. Create a Self-Signed Certificate. Contributed by Anu Engineer.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6f5bc5a4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6f5bc5a4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6f5bc5a4

Branch: refs/heads/HDDS-4
Commit: 6f5bc5a4cd868c04e0fecae22c00224293df407a
Parents: fe85d51
Author: Ajay Kumar <aj...@apache.com>
Authored: Fri Sep 28 06:52:56 2018 -0700
Committer: Ajay Kumar <aj...@apache.com>
Committed: Fri Sep 28 06:52:56 2018 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdds/HddsConfigKeys.java  |  16 ++
 .../hdds/security/x509/HDDSKeyGenerator.java    |  99 -------
 .../hdds/security/x509/HDDSKeyPEMWriter.java    | 254 ------------------
 .../hdds/security/x509/SecurityConfig.java      | 105 +++++---
 .../certificates/SelfSignedCertificate.java     | 212 +++++++++++++++
 .../x509/certificates/package-info.java         |  22 ++
 .../x509/exceptions/CertificateException.java   |  63 +++++
 .../x509/exceptions/SCMSecurityException.java   |  64 +++++
 .../security/x509/exceptions/package-info.java  |  23 ++
 .../security/x509/keys/HDDSKeyGenerator.java    | 106 ++++++++
 .../security/x509/keys/HDDSKeyPEMWriter.java    | 255 ++++++++++++++++++
 .../hdds/security/x509/keys/package-info.java   |  23 ++
 .../security/x509/TestHDDSKeyGenerator.java     |  81 ------
 .../security/x509/TestHDDSKeyPEMWriter.java     | 213 ---------------
 .../x509/certificates/TestRootCertificate.java  | 258 +++++++++++++++++++
 .../x509/certificates/package-info.java         |  22 ++
 .../x509/keys/TestHDDSKeyGenerator.java         |  87 +++++++
 .../x509/keys/TestHDDSKeyPEMWriter.java         | 216 ++++++++++++++++
 .../hdds/security/x509/keys/package-info.java   |  22 ++
 .../hadoop/hdds/security/x509/package-info.java |  22 ++
 20 files changed, 1484 insertions(+), 679 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 6524d4a..6fa8386 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -112,4 +112,20 @@ 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
+   * Default value is 5 years and written as P1865D.
+   */
+  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";
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyGenerator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyGenerator.java
deleted file mode 100644
index cb411b2..0000000
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyGenerator.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * 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;
-
-import org.apache.hadoop.conf.Configuration;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.security.KeyPair;
-import java.security.KeyPairGenerator;
-import java.security.NoSuchAlgorithmException;
-import java.security.NoSuchProviderException;
-
-/** A class to generate Key Pair for use with Certificates. */
-public class HDDSKeyGenerator {
-  private static final Logger LOG =
-      LoggerFactory.getLogger(HDDSKeyGenerator.class);
-  private final SecurityConfig securityConfig;
-
-  /**
-   * Constructor for HDDSKeyGenerator.
-   *
-   * @param configuration - config
-   */
-  public HDDSKeyGenerator(Configuration configuration) {
-    this.securityConfig = new SecurityConfig(configuration);
-  }
-
-  /**
-   * Returns the Security config used for this object.
-   * @return SecurityConfig
-   */
-  public SecurityConfig getSecurityConfig() {
-    return securityConfig;
-  }
-
-  /**
-   * Use Config to generate key.
-   *
-   * @return KeyPair
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   */
-  public KeyPair generateKey() throws NoSuchProviderException,
-      NoSuchAlgorithmException {
-    return generateKey(securityConfig.getSize(),
-        securityConfig.getAlgo(), securityConfig.getProvider());
-  }
-
-  /**
-   * Specify the size -- all other parameters are used from config.
-   *
-   * @param size - int, valid key sizes.
-   * @return KeyPair
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   */
-  public KeyPair generateKey(int size) throws
-      NoSuchProviderException, NoSuchAlgorithmException {
-    return generateKey(size,
-        securityConfig.getAlgo(), securityConfig.getProvider());
-  }
-
-  /**
-   * Custom Key Generation, all values are user provided.
-   *
-   * @param size - Key Size
-   * @param algorithm - Algorithm to use
-   * @param provider - Security provider.
-   * @return KeyPair.
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   */
-  public KeyPair generateKey(int size, String algorithm, String provider)
-      throws NoSuchProviderException, NoSuchAlgorithmException {
-    LOG.info("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
-        size, algorithm, provider);
-    KeyPairGenerator generator = KeyPairGenerator
-        .getInstance(algorithm, provider);
-    generator.initialize(size);
-    return generator.generateKeyPair();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyPEMWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyPEMWriter.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyPEMWriter.java
deleted file mode 100644
index 6ca7584..0000000
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/HDDSKeyPEMWriter.java
+++ /dev/null
@@ -1,254 +0,0 @@
-/*
- * 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;
-
-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.output.FileWriterWithEncoding;
-import org.apache.hadoop.conf.Configuration;
-import org.bouncycastle.util.io.pem.PemObject;
-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.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.KeyPair;
-import java.util.Set;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-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;
-
-/**
- * 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);
-  private final Path location;
-  private final SecurityConfig securityConfig;
-  private Set<PosixFilePermission> permissionSet =
-      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
-   */
-  public HDDSKeyPEMWriter(Configuration configuration) throws IOException {
-    Preconditions.checkNotNull(configuration, "Config cannot be null");
-    this.securityConfig = new SecurityConfig(configuration);
-    isPosixFileSystem = HDDSKeyPEMWriter::isPosix;
-    this.location = securityConfig.getKeyLocation();
-  }
-
-  /**
-   * Checks if File System supports posix style security permissions.
-   *
-   * @return True if it supports posix.
-   */
-  private static Boolean isPosix() {
-    return FileSystems.getDefault().supportedFileAttributeViews()
-        .contains("posix");
-  }
-
-  /**
-   * Returns the Permission set.
-   * @return Set
-   */
-  @VisibleForTesting
-  public Set<PosixFilePermission> getPermissionSet() {
-    return permissionSet;
-  }
-
-  /**
-   * Returns the Security config used for this object.
-   * @return SecurityConfig
-   */
-  public SecurityConfig getSecurityConfig() {
-    return securityConfig;
-  }
-
-  /**
-   * This function is used only for testing.
-   *
-   * @param isPosixFileSystem - Sets a boolean function for mimicking
-   * files systems that are not posix.
-   */
-  @VisibleForTesting
-  public void setIsPosixFileSystem(Supplier<Boolean> isPosixFileSystem) {
-    this.isPosixFileSystem = isPosixFileSystem;
-  }
-
-  /**
-   * Writes a given key using the default config options.
-   *
-   * @param keyPair - Key Pair to write to file.
-   * @throws IOException
-   */
-  public void writeKey(KeyPair keyPair) throws IOException {
-    writeKey(location, keyPair, securityConfig.getPrivateKeyName(),
-        securityConfig.getPublicKeyName(), false);
-  }
-
-  /**
-   * Writes a given key using default config options.
-   *
-   * @param keyPair - Key pair to write
-   * @param overwrite - Overwrites the keys if they already exist.
-   * @throws IOException
-   */
-  public void writeKey(KeyPair keyPair, boolean overwrite) throws IOException {
-    writeKey(location, keyPair, securityConfig.getPrivateKeyName(),
-        securityConfig.getPublicKeyName(), overwrite);
-  }
-
-  /**
-   * Writes a given key using default config options.
-   *
-   * @param basePath - The location to write to, override the config values.
-   * @param keyPair - Key pair to write
-   * @param overwrite - Overwrites the keys if they already exist.
-   * @throws IOException
-   */
-  public void writeKey(Path basePath, KeyPair keyPair, boolean overwrite)
-      throws IOException {
-    writeKey(basePath, keyPair, securityConfig.getPrivateKeyName(),
-        securityConfig.getPublicKeyName(), overwrite);
-  }
-
-  /**
-   * Helper function that actually writes data to the files.
-   *
-   * @param basePath - base path to write key
-   * @param keyPair - Key pair to write to file.
-   * @param privateKeyFileName - private key file name.
-   * @param publicKeyFileName - public key file name.
-   * @param force - forces overwriting the keys.
-   * @throws IOException
-   */
-  private synchronized void writeKey(Path basePath, KeyPair keyPair,
-      String privateKeyFileName, String publicKeyFileName, boolean force)
-      throws IOException {
-    checkPreconditions(basePath);
-
-    File privateKeyFile =
-        Paths.get(location.toString(), privateKeyFileName).toFile();
-    File publicKeyFile =
-        Paths.get(location.toString(), publicKeyFileName).toFile();
-    checkKeyFile(privateKeyFile, force, publicKeyFile);
-
-    try (PemWriter privateKeyWriter = new PemWriter(new
-        FileWriterWithEncoding(privateKeyFile, DEFAULT_CHARSET))) {
-      privateKeyWriter.writeObject(
-          new PemObject(PRIVATE_KEY, keyPair.getPrivate().getEncoded()));
-    }
-
-    try (PemWriter publicKeyWriter = new PemWriter(new
-        FileWriterWithEncoding(publicKeyFile, DEFAULT_CHARSET))) {
-      publicKeyWriter.writeObject(
-          new PemObject(PUBLIC_KEY, keyPair.getPublic().getEncoded()));
-    }
-    Files.setPosixFilePermissions(privateKeyFile.toPath(), permissionSet);
-    Files.setPosixFilePermissions(publicKeyFile.toPath(), permissionSet);
-  }
-
-  /**
-   * 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.
-   * @param publicKeyFile - public key file.
-   * @throws IOException
-   */
-  private void checkKeyFile(File privateKeyFile, boolean force,
-      File publicKeyFile) throws IOException {
-    if (privateKeyFile.exists() && force) {
-      if (!privateKeyFile.delete()) {
-        throw new IOException("Unable to delete private key file.");
-      }
-    }
-
-    if (publicKeyFile.exists() && force) {
-      if (!publicKeyFile.delete()) {
-        throw new IOException("Unable to delete public key file.");
-      }
-    }
-
-    if (privateKeyFile.exists()) {
-      throw new IOException("Private Key file already exists.");
-    }
-
-    if (publicKeyFile.exists()) {
-      throw new IOException("Public Key file already exists.");
-    }
-  }
-
-  /**
-   * Checks if base path exists and sets file permissions.
-   *
-   * @param basePath - base path to write key
-   * @throws IOException
-   */
-  private void checkPreconditions(Path basePath) throws IOException {
-    Preconditions.checkNotNull(basePath, "Base path cannot be null");
-    if (!isPosixFileSystem.get()) {
-      LOG.error("Keys cannot be stored securely without POSIX file system "
-          + "support for now.");
-      throw new IOException("Unsupported File System for pem file.");
-    }
-
-    if (Files.exists(basePath)) {
-      // Not the end of the world if we reset the permissions on an existing
-      // directory.
-      Files.setPosixFilePermissions(basePath, permissionSet);
-    } else {
-      boolean success = basePath.toFile().mkdirs();
-      if (!success) {
-        LOG.error("Unable to create the directory for the "
-            + "location. Location: {}", basePath);
-        throw new IOException("Unable to create the directory for the "
-            + "location. Location:" + basePath);
-      }
-      Files.setPosixFilePermissions(basePath, permissionSet);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
index 896a379..9231c8a 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/SecurityConfig.java
@@ -29,9 +29,10 @@ import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.security.Provider;
 import java.security.Security;
+import java.time.Duration;
 
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_LEN;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_ALGORITHM;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_LEN;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_SECURITY_PROVIDER;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_KEY_ALGORITHM;
 import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_KEY_DIR_NAME;
@@ -43,11 +44,17 @@ 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_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;
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_X509_SIGNATURE_ALGO_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
 
 /**
- * A class that deals with all Security related configs in HDDDS.
- * It is easier to have all Java code related to config in a single place.
+ * A class that deals with all Security related configs in HDDS.
+ *
+ * This class allows security configs to be read and used consistently across
+ * all of security related code base.
  */
 public class SecurityConfig {
   private static final Logger LOG =
@@ -55,15 +62,17 @@ public class SecurityConfig {
   private static volatile Provider provider;
   private final Configuration configuration;
   private final int size;
-  private final String algo;
+  private final String keyAlgo;
   private final String providerString;
   private final String metadatDir;
   private final String keyDir;
-  private final String privateKeyName;
-  private final String publicKeyName;
+  private final String privateKeyFileName;
+  private final String publicKeyFileName;
+  private final Duration certDuration;
+  private final String x509SignatureAlgo;
 
   /**
-   * Constructs a HDDSKeyGenerator.
+   * Constructs a SecurityConfig.
    *
    * @param configuration - HDDS Configuration
    */
@@ -71,10 +80,10 @@ public class SecurityConfig {
     Preconditions.checkNotNull(configuration, "Configuration cannot be null");
     this.configuration = configuration;
     this.size = this.configuration.getInt(HDDS_KEY_LEN, HDDS_DEFAULT_KEY_LEN);
-    this.algo = this.configuration.get(HDDS_KEY_ALGORITHM,
+    this.keyAlgo = this.configuration.get(HDDS_KEY_ALGORITHM,
         HDDS_DEFAULT_KEY_ALGORITHM);
     this.providerString = this.configuration.get(HDDS_SECURITY_PROVIDER,
-            HDDS_DEFAULT_SECURITY_PROVIDER);
+        HDDS_DEFAULT_SECURITY_PROVIDER);
 
     // Please Note: To make it easy for our customers we will attempt to read
     // HDDS metadata dir and if that is not set, we will use Ozone directory.
@@ -86,11 +95,17 @@ public class SecurityConfig {
         + " null. Please check configs.");
     this.keyDir = this.configuration.get(HDDS_KEY_DIR_NAME,
         HDDS_KEY_DIR_NAME_DEFAULT);
-    this.privateKeyName = this.configuration.get(HDDS_PRIVATE_KEY_FILE_NAME,
+    this.privateKeyFileName = this.configuration.get(HDDS_PRIVATE_KEY_FILE_NAME,
         HDDS_PRIVATE_KEY_FILE_NAME_DEFAULT);
-    this.publicKeyName =  this.configuration.get(HDDS_PUBLIC_KEY_FILE_NAME,
+    this.publicKeyFileName = this.configuration.get(HDDS_PUBLIC_KEY_FILE_NAME,
         HDDS_PUBLIC_KEY_FILE_NAME_DEFAULT);
 
+    String durationString = this.configuration.get(HDDS_X509_MAX_DURATION,
+        HDDS_X509_MAX_DURATION_DEFAULT);
+    this.certDuration = Duration.parse(durationString);
+    this.x509SignatureAlgo = this.configuration.get(HDDS_X509_SIGNATURE_ALGO,
+        HDDS_X509_SIGNATURE_ALGO_DEFAULT);
+
     // First Startup -- if the provider is null, check for the provider.
     if (SecurityConfig.provider == null) {
       synchronized (SecurityConfig.class) {
@@ -105,39 +120,38 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the Provider name.
-   * @return String Provider name.
-   */
-  public String getProviderString() {
-    return providerString;
-  }
-
-  /**
-   * Returns the public key file name.
+   * Returns the public key file name, This is used for storing the public
+   * keys on disk.
+   *
    * @return String, File name used for public keys.
    */
-  public String getPublicKeyName() {
-    return publicKeyName;
+  public String getPublicKeyFileName() {
+    return publicKeyFileName;
   }
 
   /**
-   * Returns the private key file name.
+   * Returns the private key file name.This is used for storing the private
+   * keys on disk.
+   *
    * @return String, File name used for private keys.
    */
-  public String getPrivateKeyName() {
-    return privateKeyName;
+  public String getPrivateKeyFileName() {
+    return privateKeyFileName;
   }
 
   /**
    * Returns the File path to where keys are stored.
-   * @return  String Key location.
+   *
+   * @return String Key location.
    */
   public Path getKeyLocation() {
     return Paths.get(metadatDir, keyDir);
   }
 
   /**
-   * Gets the Key Size.
+   * 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.
    *
    * @return key size.
    */
@@ -146,7 +160,8 @@ public class SecurityConfig {
   }
 
   /**
-   * Gets provider.
+   * Returns the Provider name. SCM defaults to using Bouncy Castle and will
+   * return "BC".
    *
    * @return String Provider name.
    */
@@ -155,22 +170,48 @@ public class SecurityConfig {
   }
 
   /**
-   * Returns the Key generation Algorithm used.
+   * Returns the Key generation Algorithm used.  User can change this by
+   * setting the "hdds.key.algo" in configuration.
    *
    * @return String Algo.
    */
-  public String getAlgo() {
-    return algo;
+  public String getKeyAlgo() {
+    return keyAlgo;
+  }
+
+  /**
+   * 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.
+   *
+   * @return String
+   */
+  public String getSignatureAlgo() {
+    return x509SignatureAlgo;
   }
 
   /**
    * Returns the Configuration used for initializing this SecurityConfig.
-   * @return  Configuration
+   *
+   * @return Configuration
    */
   public Configuration getConfiguration() {
     return configuration;
   }
 
+  /**
+   * 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
+   *
+   * Default value is 5 years and written as P1865D.
+   *
+   * @return Duration.
+   */
+  public Duration getMaxCertificateDuration() {
+    return this.certDuration;
+  }
 
   /**
    * Adds a security provider dynamically if it is not loaded already.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/SelfSignedCertificate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/SelfSignedCertificate.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/SelfSignedCertificate.java
new file mode 100644
index 0000000..fef7ac3
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/SelfSignedCertificate.java
@@ -0,0 +1,212 @@
+/*
+ * 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.certificates;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+import org.apache.hadoop.hdds.security.x509.exceptions.SCMSecurityException;
+import org.apache.hadoop.util.Time;
+import org.apache.logging.log4j.util.Strings;
+import org.bouncycastle.asn1.x500.X500Name;
+import org.bouncycastle.asn1.x509.BasicConstraints;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
+import org.bouncycastle.cert.CertIOException;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.X509v3CertificateBuilder;
+import org.bouncycastle.operator.ContentSigner;
+import org.bouncycastle.operator.OperatorCreationException;
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder;
+
+import java.math.BigInteger;
+import java.security.KeyPair;
+import java.time.Duration;
+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.
+ */
+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 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
+   * @param beginDate - NotBefore
+   * @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) {
+    this.subject = subject;
+    this.clusterID = clusterID;
+    this.scmID = scmID;
+    this.beginDate = beginDate;
+    this.endDate = endDate;
+    config = configuration;
+    this.key = keyPair;
+    this.isCA = ca;
+  }
+
+  @VisibleForTesting
+  public static String getNameFormat() {
+    return NAME_FORMAT;
+  }
+
+  public static Builder newBuilder() {
+    return new Builder();
+  }
+
+  private X509CertificateHolder generateCertificate()
+      throws OperatorCreationException, CertIOException {
+    // For the Root Certificate we form the name from Subject, SCM ID and
+    // Cluster ID.
+    String dnName = String.format(getNameFormat(), subject, scmID, clusterID);
+    X500Name name = new X500Name(dnName);
+    byte[] encoded = key.getPublic().getEncoded();
+    SubjectPublicKeyInfo publicKeyInfo =
+        SubjectPublicKeyInfo.getInstance(encoded);
+
+
+    ContentSigner contentSigner =
+        new JcaContentSignerBuilder(
+            config.getSignatureAlgo()).build(key.getPrivate());
+
+    // Please note: Since this is a root certificate we use "ONE" as the
+    // serial number. Also note that skip enforcing locale or UTC. We are
+    // trying to operate at the Days level, hence Time zone is also skipped for
+    // now.
+    BigInteger serial = BigInteger.ONE;
+    if (!isCA) {
+      serial = new BigInteger(Long.toString(Time.monotonicNow()));
+    }
+
+    X509v3CertificateBuilder builder = new X509v3CertificateBuilder(name,
+        serial, beginDate, endDate, name, publicKeyInfo);
+
+    if (isCA) {
+      builder.addExtension(Extension.basicConstraints, true,
+          new BasicConstraints(true));
+    }
+    return builder.build(contentSigner);
+  }
+
+  /**
+   * Builder class for Root Certificates.
+   */
+  public static class Builder {
+    private String subject;
+    private String clusterID;
+    private String scmID;
+    private Date beginDate;
+    private Date endDate;
+    private KeyPair key;
+    private SecurityConfig config;
+    private boolean isCA;
+
+    public Builder setConfiguration(Configuration configuration) {
+      this.config = new SecurityConfig(configuration);
+      return this;
+    }
+
+    public Builder setKey(KeyPair keyPair) {
+      this.key = keyPair;
+      return this;
+    }
+
+    public Builder setSubject(String subjectString) {
+      this.subject = subjectString;
+      return this;
+    }
+
+    public Builder setClusterID(String s) {
+      this.clusterID = s;
+      return this;
+    }
+
+    public Builder setScmID(String s) {
+      this.scmID = s;
+      return this;
+    }
+
+    public Builder setBeginDate(Date date) {
+      this.beginDate = new Date(date.toInstant().toEpochMilli());
+      return this;
+    }
+
+    public Builder setEndDate(Date date) {
+      this.endDate = new Date(date.toInstant().toEpochMilli());
+      return this;
+    }
+
+    public Builder makeCA() {
+      isCA = true;
+      return this;
+    }
+
+    public X509CertificateHolder build() throws SCMSecurityException {
+      Preconditions.checkNotNull(key, "Key cannot be null");
+      Preconditions.checkArgument(Strings.isNotBlank(subject), "Subject " +
+          "cannot be blank");
+      Preconditions.checkArgument(Strings.isNotBlank(clusterID), "Cluster ID " +
+          "cannot be blank");
+      Preconditions.checkArgument(Strings.isNotBlank(scmID), "SCM ID cannot " +
+          "be blank");
+
+      Preconditions.checkArgument(beginDate.before(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.");
+
+
+      SelfSignedCertificate rootCertificate =
+          new SelfSignedCertificate(this.subject,
+          this.scmID, this.clusterID, this.beginDate, this.endDate,
+          this.config, key, isCA);
+      try {
+        return rootCertificate.generateCertificate();
+      } catch (OperatorCreationException | CertIOException e) {
+        throw new CertificateException("Unable to create root certificate.",
+            e.getCause());
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java
new file mode 100644
index 0000000..e88737c
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/certificates/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.
+ *
+ */
+/**
+ * Utils for Certificates.
+ */
+package org.apache.hadoop.hdds.security.x509.certificates;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4d2dcfd
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/CertificateException.java
@@ -0,0 +1,63 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Certificate Exceptions from the SCM Security layer.
+ */
+public class CertificateException extends SCMSecurityException {
+
+  /**
+   * Ctor.
+   * @param message - Error Message.
+   */
+  public CertificateException(String message) {
+    super(message);
+  }
+
+  /**
+   * Ctor.
+   * @param message - Message.
+   * @param cause  - Actual cause.
+   */
+  public CertificateException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  /**
+   * Ctor.
+   * @param cause - Base Exception.
+   */
+  public CertificateException(Throwable cause) {
+    super(cause);
+  }
+
+  /**
+   * Ctor.
+   * @param message - Error Message
+   * @param cause  - Cause
+   * @param enableSuppression - Enable suppression.
+   * @param writableStackTrace - Writable stack trace.
+   */
+  public CertificateException(String message, Throwable cause,
+      boolean enableSuppression, boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/SCMSecurityException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/SCMSecurityException.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/SCMSecurityException.java
new file mode 100644
index 0000000..e1f6296
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/SCMSecurityException.java
@@ -0,0 +1,64 @@
+/*
+ * 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.exceptions;
+
+/**
+ * Root Security Exception call for all Certificate related Execptions.
+ */
+public class SCMSecurityException extends Exception {
+
+  /**
+   * Ctor.
+   * @param message - Error Message.
+   */
+  public SCMSecurityException(String message) {
+    super(message);
+  }
+
+  /**
+   * Ctor.
+   * @param message - Message.
+   * @param cause  - Actual cause.
+   */
+  public SCMSecurityException(String message, Throwable cause) {
+    super(message, cause);
+  }
+
+  /**
+   * Ctor.
+   * @param cause - Base Exception.
+   */
+  public SCMSecurityException(Throwable cause) {
+    super(cause);
+  }
+
+
+  /**
+   * Ctor.
+   * @param message - Error Message
+   * @param cause  - Cause
+   * @param enableSuppression - Enable suppression.
+   * @param writableStackTrace - Writable stack trace.
+   */
+  public SCMSecurityException(String message, Throwable cause,
+      boolean enableSuppression, boolean writableStackTrace) {
+    super(message, cause, enableSuppression, writableStackTrace);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/package-info.java
new file mode 100644
index 0000000..afcc474
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/exceptions/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * Exceptions thrown by X.509 security classes.
+ */
+package org.apache.hadoop.hdds.security.x509.exceptions;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java
new file mode 100644
index 0000000..99873cb
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyGenerator.java
@@ -0,0 +1,106 @@
+/*
+ * 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.keys;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.security.KeyPair;
+import java.security.KeyPairGenerator;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+
+/** A class to generate Key Pair for use with Certificates. */
+public class HDDSKeyGenerator {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(HDDSKeyGenerator.class);
+  private final SecurityConfig securityConfig;
+
+  /**
+   * Constructor for HDDSKeyGenerator.
+   *
+   * @param configuration - config
+   */
+  public HDDSKeyGenerator(Configuration configuration) {
+    this.securityConfig = new SecurityConfig(configuration);
+  }
+
+  /**
+   * Returns the Security config used for this object.
+   * @return SecurityConfig
+   */
+  public SecurityConfig getSecurityConfig() {
+    return securityConfig;
+  }
+
+  /**
+   * Use Config to generate key.
+   *
+   * @return KeyPair
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   */
+  public KeyPair generateKey() throws NoSuchProviderException,
+      NoSuchAlgorithmException {
+    return generateKey(securityConfig.getSize(),
+        securityConfig.getKeyAlgo(), securityConfig.getProvider());
+  }
+
+  /**
+   * Specify the size -- all other parameters are used from config.
+   *
+   * @param size - int, valid key sizes.
+   * @return KeyPair
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   */
+  public KeyPair generateKey(int size) throws
+      NoSuchProviderException, NoSuchAlgorithmException {
+    return generateKey(size,
+        securityConfig.getKeyAlgo(), securityConfig.getProvider());
+  }
+
+  /**
+   * Custom Key Generation, all values are user provided.
+   *
+   * @param size - Key Size
+   * @param algorithm - Algorithm to use
+   * @param provider - Security provider.
+   * @return KeyPair.
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   */
+  public KeyPair generateKey(int size, String algorithm, String provider)
+      throws NoSuchProviderException, NoSuchAlgorithmException {
+    LOG.info("Generating key pair using size:{}, Algorithm:{}, Provider:{}",
+        size, algorithm, provider);
+    KeyPairGenerator generator = KeyPairGenerator
+        .getInstance(algorithm, provider);
+    generator.initialize(size);
+    return generator.generateKeyPair();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyPEMWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyPEMWriter.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyPEMWriter.java
new file mode 100644
index 0000000..95be1c4
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/HDDSKeyPEMWriter.java
@@ -0,0 +1,255 @@
+/*
+ * 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.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.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.PemWriter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+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.KeyPair;
+import java.util.Set;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+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;
+
+/**
+ * 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);
+  private final Path location;
+  private final SecurityConfig securityConfig;
+  private Set<PosixFilePermission> permissionSet =
+      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
+   */
+  public HDDSKeyPEMWriter(Configuration configuration) throws IOException {
+    Preconditions.checkNotNull(configuration, "Config cannot be null");
+    this.securityConfig = new SecurityConfig(configuration);
+    isPosixFileSystem = HDDSKeyPEMWriter::isPosix;
+    this.location = securityConfig.getKeyLocation();
+  }
+
+  /**
+   * Checks if File System supports posix style security permissions.
+   *
+   * @return True if it supports posix.
+   */
+  private static Boolean isPosix() {
+    return FileSystems.getDefault().supportedFileAttributeViews()
+        .contains("posix");
+  }
+
+  /**
+   * Returns the Permission set.
+   * @return Set
+   */
+  @VisibleForTesting
+  public Set<PosixFilePermission> getPermissionSet() {
+    return permissionSet;
+  }
+
+  /**
+   * Returns the Security config used for this object.
+   * @return SecurityConfig
+   */
+  public SecurityConfig getSecurityConfig() {
+    return securityConfig;
+  }
+
+  /**
+   * This function is used only for testing.
+   *
+   * @param isPosixFileSystem - Sets a boolean function for mimicking
+   * files systems that are not posix.
+   */
+  @VisibleForTesting
+  public void setIsPosixFileSystem(Supplier<Boolean> isPosixFileSystem) {
+    this.isPosixFileSystem = isPosixFileSystem;
+  }
+
+  /**
+   * Writes a given key using the default config options.
+   *
+   * @param keyPair - Key Pair to write to file.
+   * @throws IOException - On I/O failure.
+   */
+  public void writeKey(KeyPair keyPair) throws IOException {
+    writeKey(location, keyPair, securityConfig.getPrivateKeyFileName(),
+        securityConfig.getPublicKeyFileName(), false);
+  }
+
+  /**
+   * Writes a given key using default config options.
+   *
+   * @param keyPair - Key pair to write
+   * @param overwrite - Overwrites the keys if they already exist.
+   * @throws IOException - On I/O failure.
+   */
+  public void writeKey(KeyPair keyPair, boolean overwrite) throws IOException {
+    writeKey(location, keyPair, securityConfig.getPrivateKeyFileName(),
+        securityConfig.getPublicKeyFileName(), overwrite);
+  }
+
+  /**
+   * Writes a given key using default config options.
+   *
+   * @param basePath - The location to write to, override the config values.
+   * @param keyPair - Key pair to write
+   * @param overwrite - Overwrites the keys if they already exist.
+   * @throws IOException - On I/O failure.
+   */
+  public void writeKey(Path basePath, KeyPair keyPair, boolean overwrite)
+      throws IOException {
+    writeKey(basePath, keyPair, securityConfig.getPrivateKeyFileName(),
+        securityConfig.getPublicKeyFileName(), overwrite);
+  }
+
+  /**
+   * Helper function that actually writes data to the files.
+   *
+   * @param basePath - base path to write key
+   * @param keyPair - Key pair to write to file.
+   * @param privateKeyFileName - private key file name.
+   * @param publicKeyFileName - public key file name.
+   * @param force - forces overwriting the keys.
+   * @throws IOException - On I/O failure.
+   */
+  private synchronized void writeKey(Path basePath, KeyPair keyPair,
+      String privateKeyFileName, String publicKeyFileName, boolean force)
+      throws IOException {
+    checkPreconditions(basePath);
+
+    File privateKeyFile =
+        Paths.get(location.toString(), privateKeyFileName).toFile();
+    File publicKeyFile =
+        Paths.get(location.toString(), publicKeyFileName).toFile();
+    checkKeyFile(privateKeyFile, force, publicKeyFile);
+
+    try (PemWriter privateKeyWriter = new PemWriter(new
+        FileWriterWithEncoding(privateKeyFile, DEFAULT_CHARSET))) {
+      privateKeyWriter.writeObject(
+          new PemObject(PRIVATE_KEY, keyPair.getPrivate().getEncoded()));
+    }
+
+    try (PemWriter publicKeyWriter = new PemWriter(new
+        FileWriterWithEncoding(publicKeyFile, DEFAULT_CHARSET))) {
+      publicKeyWriter.writeObject(
+          new PemObject(PUBLIC_KEY, keyPair.getPublic().getEncoded()));
+    }
+    Files.setPosixFilePermissions(privateKeyFile.toPath(), permissionSet);
+    Files.setPosixFilePermissions(publicKeyFile.toPath(), permissionSet);
+  }
+
+  /**
+   * 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.
+   * @param publicKeyFile - public key file.
+   * @throws IOException - On I/O failure.
+   */
+  private void checkKeyFile(File privateKeyFile, boolean force,
+      File publicKeyFile) throws IOException {
+    if (privateKeyFile.exists() && force) {
+      if (!privateKeyFile.delete()) {
+        throw new IOException("Unable to delete private key file.");
+      }
+    }
+
+    if (publicKeyFile.exists() && force) {
+      if (!publicKeyFile.delete()) {
+        throw new IOException("Unable to delete public key file.");
+      }
+    }
+
+    if (privateKeyFile.exists()) {
+      throw new IOException("Private Key file already exists.");
+    }
+
+    if (publicKeyFile.exists()) {
+      throw new IOException("Public Key file already exists.");
+    }
+  }
+
+  /**
+   * Checks if base path exists and sets file permissions.
+   *
+   * @param basePath - base path to write key
+   * @throws IOException - On I/O failure.
+   */
+  private void checkPreconditions(Path basePath) throws IOException {
+    Preconditions.checkNotNull(basePath, "Base path cannot be null");
+    if (!isPosixFileSystem.get()) {
+      LOG.error("Keys cannot be stored securely without POSIX file system "
+          + "support for now.");
+      throw new IOException("Unsupported File System for pem file.");
+    }
+
+    if (Files.exists(basePath)) {
+      // Not the end of the world if we reset the permissions on an existing
+      // directory.
+      Files.setPosixFilePermissions(basePath, permissionSet);
+    } else {
+      boolean success = basePath.toFile().mkdirs();
+      if (!success) {
+        LOG.error("Unable to create the directory for the "
+            + "location. Location: {}", basePath);
+        throw new IOException("Unable to create the directory for the "
+            + "location. Location:" + basePath);
+      }
+      Files.setPosixFilePermissions(basePath, permissionSet);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java
new file mode 100644
index 0000000..37a04d6
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/x509/keys/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ *
+ */
+
+/**
+ * Utils for private and public keys.
+ */
+package org.apache.hadoop.hdds.security.x509.keys;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyGenerator.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyGenerator.java
deleted file mode 100644
index 2ddf59c..0000000
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyGenerator.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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;
-
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
-import java.security.KeyPair;
-import java.security.NoSuchAlgorithmException;
-import java.security.NoSuchProviderException;
-import java.security.PublicKey;
-import java.security.interfaces.RSAPublicKey;
-import java.security.spec.PKCS8EncodedKeySpec;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test class for HDDS Key Generator.
- */
-public class TestHDDSKeyGenerator {
-  private static SecurityConfig config;
-
-  @Before
-  public void init() {
-    OzoneConfiguration conf = new OzoneConfiguration();
-    conf.set(OZONE_METADATA_DIRS,  GenericTestUtils.getTempPath("testpath"));
-    config = new SecurityConfig(conf);
-  }
-  /**
-   * In this test we verify that we are able to create a key pair, then get
-   * bytes of that and use ASN1. parser to parse it back to a private key.
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   */
-  @Test
-  public void testGenerateKey()
-      throws NoSuchProviderException, NoSuchAlgorithmException {
-    HDDSKeyGenerator keyGen = new HDDSKeyGenerator(config.getConfiguration());
-    KeyPair keyPair = keyGen.generateKey();
-    Assert.assertEquals(config.getAlgo(), keyPair.getPrivate().getAlgorithm());
-    PKCS8EncodedKeySpec keySpec =
-        new PKCS8EncodedKeySpec(keyPair.getPrivate().getEncoded());
-    Assert.assertEquals("PKCS#8", keySpec.getFormat());
-  }
-
-  /**
-   * In this test we assert that size that we specified is used for Key
-   * generation.
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   */
-  @Test
-  public void testGenerateKeyWithSize() throws NoSuchProviderException,
-      NoSuchAlgorithmException {
-    HDDSKeyGenerator keyGen = new HDDSKeyGenerator(config.getConfiguration());
-    KeyPair keyPair = keyGen.generateKey(4096);
-    PublicKey publicKey = keyPair.getPublic();
-    if(publicKey instanceof RSAPublicKey) {
-      Assert.assertEquals(4096,
-          ((RSAPublicKey)(publicKey)).getModulus().bitLength());
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyPEMWriter.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyPEMWriter.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyPEMWriter.java
deleted file mode 100644
index 68ff9e6..0000000
--- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/TestHDDSKeyPEMWriter.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * 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;
-
-import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_METADATA_DIR_NAME;
-
-import java.io.IOException;
-import java.nio.charset.StandardCharsets;
-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.NoSuchProviderException;
-import java.security.PrivateKey;
-import java.security.PublicKey;
-import java.security.spec.InvalidKeySpecException;
-import java.security.spec.PKCS8EncodedKeySpec;
-import java.security.spec.X509EncodedKeySpec;
-import java.util.Set;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-/**
- * Test class for HDDS pem writer.
- */
-public class TestHDDSKeyPEMWriter {
-
-  @Rule
-  public TemporaryFolder temporaryFolder = new TemporaryFolder();
-  private OzoneConfiguration configuration;
-  private HDDSKeyGenerator keyGenerator;
-  private String prefix;
-
-  @Before
-  public void init() throws IOException {
-    configuration = new OzoneConfiguration();
-    prefix = temporaryFolder.newFolder().toString();
-    configuration.set(HDDS_METADATA_DIR_NAME, prefix);
-    keyGenerator = new HDDSKeyGenerator(configuration);
-  }
-
-  /**
-   * Assert basic things like we are able to create a file, and the names are
-   * in expected format etc.
-   *
-   * @throws NoSuchProviderException
-   * @throws NoSuchAlgorithmException
-   * @throws IOException
-   */
-  @Test
-  public void testWriteKey()
-      throws NoSuchProviderException, NoSuchAlgorithmException,
-      IOException, InvalidKeySpecException {
-    KeyPair keys = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
-    pemWriter.writeKey(keys);
-
-    // Assert that locations have been created.
-    Path keyLocation = pemWriter.getSecurityConfig().getKeyLocation();
-    Assert.assertTrue(keyLocation.toFile().exists());
-
-    // Assert that locations are created in the locations that we specified
-    // using the Config.
-    Assert.assertTrue(keyLocation.toString().startsWith(prefix));
-    Path privateKeyPath = Paths.get(keyLocation.toString(),
-        pemWriter.getSecurityConfig().getPrivateKeyName());
-    Assert.assertTrue(privateKeyPath.toFile().exists());
-    Path publicKeyPath = Paths.get(keyLocation.toString(),
-        pemWriter.getSecurityConfig().getPublicKeyName());
-    Assert.assertTrue(publicKeyPath.toFile().exists());
-
-    // Read the private key and test if the expected String in the PEM file
-    // format exists.
-    byte[] privateKey = Files.readAllBytes(privateKeyPath);
-    String privateKeydata = new String(privateKey, StandardCharsets.UTF_8);
-    Assert.assertTrue(privateKeydata.contains("PRIVATE KEY"));
-
-    // Read the public key and test if the expected String in the PEM file
-    // format exists.
-    byte[] publicKey = Files.readAllBytes(publicKeyPath);
-    String publicKeydata = new String(publicKey, StandardCharsets.UTF_8);
-    Assert.assertTrue(publicKeydata.contains("PUBLIC KEY"));
-
-    // Let us decode the PEM file and parse it back into binary.
-    KeyFactory kf = KeyFactory.getInstance(
-        pemWriter.getSecurityConfig().getAlgo());
-
-    // Replace the PEM Human readable guards.
-    privateKeydata =
-        privateKeydata.replace("-----BEGIN PRIVATE KEY-----\n", "");
-    privateKeydata =
-        privateKeydata.replace("-----END PRIVATE KEY-----", "");
-
-    // Decode the bas64 to binary format and then use an ASN.1 parser to
-    // parse the binary format.
-
-    byte[] keyBytes = Base64.decodeBase64(privateKeydata);
-    PKCS8EncodedKeySpec spec = new PKCS8EncodedKeySpec(keyBytes);
-    PrivateKey privateKeyDecoded = kf.generatePrivate(spec);
-    Assert.assertNotNull("Private Key should not be null",
-        privateKeyDecoded);
-
-    // Let us decode the public key and veriy that we can parse it back into
-    // binary.
-    publicKeydata =
-        publicKeydata.replace("-----BEGIN PUBLIC KEY-----\n", "");
-    publicKeydata =
-        publicKeydata.replace("-----END PUBLIC KEY-----", "");
-
-    keyBytes = Base64.decodeBase64(publicKeydata);
-    X509EncodedKeySpec pubKeyspec = new X509EncodedKeySpec(keyBytes);
-    PublicKey publicKeyDecoded = kf.generatePublic(pubKeyspec);
-    Assert.assertNotNull("Public Key should not be null",
-        publicKeyDecoded);
-
-    // Now let us assert the permissions on the Directories and files are as
-    // expected.
-    Set<PosixFilePermission> expectedSet = pemWriter.getPermissionSet();
-    Set<PosixFilePermission> currentSet =
-        Files.getPosixFilePermissions(privateKeyPath);
-    currentSet.removeAll(expectedSet);
-    Assert.assertEquals(0, currentSet.size());
-
-    currentSet =
-        Files.getPosixFilePermissions(publicKeyPath);
-    currentSet.removeAll(expectedSet);
-    Assert.assertEquals(0, currentSet.size());
-
-    currentSet =
-        Files.getPosixFilePermissions(keyLocation);
-    currentSet.removeAll(expectedSet);
-    Assert.assertEquals(0, currentSet.size());
-  }
-
-  /**
-   * Assert key rewrite fails without force option.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testReWriteKey()
-      throws Exception {
-    KeyPair kp = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
-    SecurityConfig secConfig = pemWriter.getSecurityConfig();
-    pemWriter.writeKey(kp);
-
-    // Assert that rewriting of keys throws exception with valid messages.
-    LambdaTestUtils
-        .intercept(IOException.class, "Private Key file already exists.",
-            () -> pemWriter.writeKey(kp));
-    FileUtils.deleteQuietly(Paths.get(
-        secConfig.getKeyLocation().toString() + "/" + secConfig
-            .getPrivateKeyName()).toFile());
-    LambdaTestUtils
-        .intercept(IOException.class, "Public Key file already exists.",
-            () -> pemWriter.writeKey(kp));
-    FileUtils.deleteQuietly(Paths.get(
-        secConfig.getKeyLocation().toString() + "/" + secConfig
-            .getPublicKeyName()).toFile());
-
-    // Should succeed now as both public and private key are deleted.
-    pemWriter.writeKey(kp);
-    // Should succeed with overwrite flag as true.
-    pemWriter.writeKey(kp, true);
-
-  }
-
-  /**
-   * Assert key rewrite fails in non Posix file system.
-   *
-   * @throws IOException
-   */
-  @Test
-  public void testWriteKeyInNonPosixFS()
-      throws Exception {
-    KeyPair kp = keyGenerator.generateKey();
-    HDDSKeyPEMWriter pemWriter = new HDDSKeyPEMWriter(configuration);
-    pemWriter.setIsPosixFileSystem(() -> false);
-
-    // Assert key rewrite fails in non Posix file system.
-    LambdaTestUtils
-        .intercept(IOException.class, "Unsupported File System for pem file.",
-            () -> pemWriter.writeKey(kp));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java
new file mode 100644
index 0000000..3a00ca0
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/TestRootCertificate.java
@@ -0,0 +1,258 @@
+/*
+ * 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.certificates;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.exceptions.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
+import org.bouncycastle.asn1.x509.Extension;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.IOException;
+import java.math.BigInteger;
+import java.security.InvalidKeyException;
+import java.security.KeyPair;
+import java.security.NoSuchAlgorithmException;
+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.util.Date;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.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();
+
+  @Before
+  public void init() throws IOException {
+    conf.set(OZONE_METADATA_DIRS, temporaryFolder.newFolder().toString());
+    securityConfig = new SecurityConfig(conf);
+  }
+
+  @Test
+  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)));
+    String clusterID = UUID.randomUUID().toString();
+    String scmID = UUID.randomUUID().toString();
+    String subject = "testRootCert";
+    HDDSKeyGenerator keyGen =
+        new HDDSKeyGenerator(securityConfig.getConfiguration());
+    KeyPair keyPair = keyGen.generateKey();
+
+    SelfSignedCertificate.Builder builder =
+        SelfSignedCertificate.newBuilder()
+            .setBeginDate(notBefore)
+            .setEndDate(notAfter)
+            .setClusterID(clusterID)
+            .setScmID(scmID)
+            .setSubject(subject)
+            .setKey(keyPair)
+            .setConfiguration(conf);
+
+    X509CertificateHolder certificateHolder = builder.build();
+
+    //Assert that we indeed have a self signed certificate.
+    Assert.assertEquals(certificateHolder.getIssuer(),
+        certificateHolder.getSubject());
+
+
+    // Make sure that NotBefore is before the current Date
+    Date invalidDate = Date.from(now.minus(Duration.ofDays(1)));
+    Assert.assertFalse(
+        certificateHolder.getNotBefore()
+            .before(invalidDate));
+
+    //Make sure the end date is honored.
+    invalidDate = Date.from(now.plus(Duration.ofDays(366)));
+    Assert.assertFalse(
+        certificateHolder.getNotAfter()
+            .after(invalidDate));
+
+    // Check the Subject Name and Issuer Name is in the expected format.
+    String dnName = String.format(SelfSignedCertificate.getNameFormat(),
+        subject, scmID, clusterID);
+    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
+    // extension should be null.
+    Assert.assertNull(
+        certificateHolder.getExtension(Extension.basicConstraints));
+
+    // Extract the Certificate and verify that certificate matches the public
+    // key.
+    X509Certificate cert =
+        new JcaX509CertificateConverter().getCertificate(certificateHolder);
+    cert.verify(keyPair.getPublic());
+  }
+
+  @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)));
+    String clusterID = UUID.randomUUID().toString();
+    String scmID = UUID.randomUUID().toString();
+    String subject = "testRootCert";
+    HDDSKeyGenerator keyGen =
+        new HDDSKeyGenerator(securityConfig.getConfiguration());
+    KeyPair keyPair = keyGen.generateKey();
+
+    SelfSignedCertificate.Builder builder =
+        SelfSignedCertificate.newBuilder()
+            .setBeginDate(notBefore)
+            .setEndDate(notAfter)
+            .setClusterID(clusterID)
+            .setScmID(scmID)
+            .setSubject(subject)
+            .setKey(keyPair)
+            .setConfiguration(conf)
+            .makeCA();
+
+    X509CertificateHolder certificateHolder = builder.build();
+    // This time we asked for a CA Certificate, make sure that extension is
+    // present and valid.
+    Extension basicExt =
+        certificateHolder.getExtension(Extension.basicConstraints);
+
+    Assert.assertNotNull(basicExt);
+    Assert.assertTrue(basicExt.isCritical());
+
+    // Since this code assigns ONE for the root certificate, we check if the
+    // serial number is the expected number.
+    Assert.assertEquals(certificateHolder.getSerialNumber(), BigInteger.ONE);
+  }
+
+  @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)));
+    String clusterID = UUID.randomUUID().toString();
+    String scmID = UUID.randomUUID().toString();
+    String subject = "testRootCert";
+    HDDSKeyGenerator keyGen =
+        new HDDSKeyGenerator(securityConfig.getConfiguration());
+    KeyPair keyPair = keyGen.generateKey();
+
+    SelfSignedCertificate.Builder builder =
+        SelfSignedCertificate.newBuilder()
+            .setBeginDate(notBefore)
+            .setEndDate(notAfter)
+            .setClusterID(clusterID)
+            .setScmID(scmID)
+            .setSubject(subject)
+            .setConfiguration(conf)
+            .setKey(keyPair)
+            .makeCA();
+    try {
+      builder.setKey(null);
+      builder.build();
+      Assert.fail("Null Key should have failed.");
+    } catch (NullPointerException | IllegalArgumentException e) {
+      builder.setKey(keyPair);
+    }
+
+    // Now try with Blank Subject.
+    try {
+      builder.setSubject("");
+      builder.build();
+      Assert.fail("Null/Blank Subject should have thrown.");
+    } catch (IllegalArgumentException e) {
+      builder.setSubject(subject);
+    }
+
+    // Now try with blank/null SCM ID
+    try {
+      builder.setScmID(null);
+      builder.build();
+      Assert.fail("Null/Blank SCM ID should have thrown.");
+    } catch (IllegalArgumentException e) {
+      builder.setScmID(scmID);
+    }
+
+
+    // Now try with blank/null SCM ID
+    try {
+      builder.setClusterID(null);
+      builder.build();
+      Assert.fail("Null/Blank Cluster ID should have thrown.");
+    } catch (IllegalArgumentException e) {
+      builder.setClusterID(clusterID);
+    }
+
+
+    // Swap the Begin and End Date and verify that we cannot create a
+    // certificate like that.
+    try {
+      builder.setBeginDate(notAfter);
+      builder.setEndDate(notBefore);
+      builder.build();
+      Assert.fail("Illegal dates should have thrown.");
+    } catch (IllegalArgumentException e) {
+      builder.setBeginDate(notBefore);
+      builder.setEndDate(notAfter);
+    }
+
+    try {
+      KeyPair newKey = keyGen.generateKey();
+      KeyPair wrongKey = new KeyPair(newKey.getPublic(), keyPair.getPrivate());
+      builder.setKey(wrongKey);
+      X509CertificateHolder certificateHolder = builder.build();
+      X509Certificate cert =
+          new JcaX509CertificateConverter().getCertificate(certificateHolder);
+      cert.verify(wrongKey.getPublic());
+      Assert.fail("Invalid Key, should have thrown.");
+    } catch (SCMSecurityException | CertificateException
+        | SignatureException | InvalidKeyException e) {
+      builder.setKey(keyPair);
+    }
+    // Assert that we can create a certificate with all sane params.
+    Assert.assertNotNull(builder.build());
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/package-info.java
new file mode 100644
index 0000000..c8a31fc
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/certificates/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.
+ *
+ */
+/**
+ * Test classes for Certificate utilities.
+ */
+package org.apache.hadoop.hdds.security.x509.certificates;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6f5bc5a4/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyGenerator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyGenerator.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyGenerator.java
new file mode 100644
index 0000000..f9541a2
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/security/x509/keys/TestHDDSKeyGenerator.java
@@ -0,0 +1,87 @@
+/*
+ * 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.keys;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
+import java.security.KeyPair;
+import java.security.NoSuchAlgorithmException;
+import java.security.NoSuchProviderException;
+import java.security.PublicKey;
+import java.security.interfaces.RSAPublicKey;
+import java.security.spec.PKCS8EncodedKeySpec;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test class for HDDS Key Generator.
+ */
+public class TestHDDSKeyGenerator {
+  private SecurityConfig config;
+
+  @Before
+  public void init() {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    conf.set(OZONE_METADATA_DIRS,  GenericTestUtils.getTempPath("testpath"));
+    config = new SecurityConfig(conf);
+  }
+  /**
+   * In this test we verify that we are able to create a key pair, then get
+   * bytes of that and use ASN1. parser to parse it back to a private key.
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   */
+  @Test
+  public void testGenerateKey()
+      throws NoSuchProviderException, NoSuchAlgorithmException {
+    HDDSKeyGenerator keyGen = new HDDSKeyGenerator(config.getConfiguration());
+    KeyPair keyPair = keyGen.generateKey();
+    Assert.assertEquals(config.getKeyAlgo(),
+        keyPair.getPrivate().getAlgorithm());
+    PKCS8EncodedKeySpec keySpec =
+        new PKCS8EncodedKeySpec(keyPair.getPrivate().getEncoded());
+    Assert.assertEquals("PKCS#8", keySpec.getFormat());
+  }
+
+  /**
+   * In this test we assert that size that we specified is used for Key
+   * generation.
+   * @throws NoSuchProviderException - On Error, due to missing Java
+   * dependencies.
+   * @throws NoSuchAlgorithmException - On Error,  due to missing Java
+   * dependencies.
+   */
+  @Test
+  public void testGenerateKeyWithSize() throws NoSuchProviderException,
+      NoSuchAlgorithmException {
+    HDDSKeyGenerator keyGen = new HDDSKeyGenerator(config.getConfiguration());
+    KeyPair keyPair = keyGen.generateKey(4096);
+    PublicKey publicKey = keyPair.getPublic();
+    if(publicKey instanceof RSAPublicKey) {
+      Assert.assertEquals(4096,
+          ((RSAPublicKey)(publicKey)).getModulus().bitLength());
+    }
+  }
+}
\ No newline at end of file


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