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

[GitHub] [ozone] vivekratnavel opened a new pull request #1874: HDDS-4482. SCM should be able to persist CRL

vivekratnavel opened a new pull request #1874:
URL: https://github.com/apache/ozone/pull/1874


   ## What changes were proposed in this pull request?
   
   - Add a new table to persist CRL information
   - Add a new table to keep track of the last CRL sequence ID
   - Update the revoke certificates method to create a new CRL to revoke certificates
   - Add Unit tests
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-4482
   
   ## How was this patch tested?
   
   Unit tests
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
##########
@@ -101,26 +101,22 @@ X509Certificate getCertificate(String certSerialId)
   /**
    * Revokes a Certificate issued by this CertificateServer.
    *
-   * @param certificate - Certificate to revoke
-   * @param approver - Approval process to follow.
+   * @param certificates - List of Certificates to revoke.
+   * @param reason - Reason for revocation.
+   * @param securityConfig - Security Configuration.
    * @return Future that tells us what happened.
-   * @throws SCMSecurityException - on Error.
-   */
-  Future<Boolean> revokeCertificate(X509Certificate certificate,
-      ApprovalType approver) throws SCMSecurityException;
-
-  /**
-   * TODO : CRL, OCSP etc. Later. This is the start of a CertificateServer
-   * framework.
    */
+  Future<Boolean> revokeCertificates(List<X509Certificate> certificates,
+                                     int reason,

Review comment:
       Can the reason be a Enum?

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
##########
@@ -101,26 +101,22 @@ X509Certificate getCertificate(String certSerialId)
   /**
    * Revokes a Certificate issued by this CertificateServer.
    *
-   * @param certificate - Certificate to revoke
-   * @param approver - Approval process to follow.
+   * @param certificates - List of Certificates to revoke.
+   * @param reason - Reason for revocation.
+   * @param securityConfig - Security Configuration.
    * @return Future that tells us what happened.
-   * @throws SCMSecurityException - on Error.
-   */
-  Future<Boolean> revokeCertificate(X509Certificate certificate,
-      ApprovalType approver) throws SCMSecurityException;
-
-  /**
-   * TODO : CRL, OCSP etc. Later. This is the start of a CertificateServer
-   * framework.

Review comment:
       NIT: javadoc?

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
##########
@@ -272,20 +272,22 @@ private X509CertificateHolder signAndStoreCertificate(LocalDate beginDate,
   }
 
   @Override
-  public Future<Boolean> revokeCertificate(X509Certificate certificate,
-      CertificateApprover.ApprovalType approverType)
-      throws SCMSecurityException {
+  public Future<Boolean> revokeCertificates(List<X509Certificate> certificates,
+                                            int reason,
+                                            SecurityConfig securityConfig) {
     CompletableFuture<Boolean> revoked = new CompletableFuture<>();
-    if (certificate == null) {
+    if (certificates == null || certificates.isEmpty()) {

Review comment:
       NIT: we can use Collection#isEmptyOrNull here.

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java
##########
@@ -47,12 +50,21 @@ void storeValidCertificate(BigInteger serialID,
                              X509Certificate certificate) throws IOException;
 
   /**
-   * Moves a certificate in a transactional manner from valid certificate to
+   * Adds the certificates to be revoked to a new CRL and moves all the
+   * certificates in a transactional manner from valid certificate to
    * revoked certificate state.
-   * @param serialID - Serial ID of the certificate.
+   * @param certificates - List of X509 Certificates to be revoked.
+   * @param caCertificateHolder - X509 Certificate Holder of the CA.
+   * @param reason - CRLReason for revocation.
+   * @param securityConfig - Security Configuration.
+   * @param keyPair - Public and Private key of the CA.
    * @throws IOException
    */
-  void revokeCertificate(BigInteger serialID) throws IOException;
+  void revokeCertificates(List<X509Certificate> certificates,
+                          X509CertificateHolder caCertificateHolder,
+                          int reason, SecurityConfig securityConfig,
+                          KeyPair keyPair)

Review comment:
       can we abstract the CRL handling into a separate class (like Approver for CSR) without bringing CA keypair into the certificate store interface and class?

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/crl/CRLInfo.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.crl;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CRLCodec;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.IOException;
+import java.security.cert.CRLException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509CRL;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Class that wraps Certificate Revocation List Info.
+ */
+public class CRLInfo implements Comparator<CRLInfo>,
+    Comparable<CRLInfo> {
+
+  private X509CRL x509CRL;
+  private long creationTimestamp;
+
+  private CRLInfo(X509CRL x509CRL, long creationTimestamp) {
+    this.x509CRL = x509CRL;
+    this.creationTimestamp = creationTimestamp;
+  }
+
+  /**
+   * Constructor for CRLInfo. Needed for serialization findbugs.
+   */
+  public CRLInfo() {
+  }
+
+  public static CRLInfo fromProtobuf(HddsProtos.CRLInfoProto info)
+      throws IOException, CRLException, CertificateException {
+    CRLInfo.Builder builder = new CRLInfo.Builder();
+    return builder
+        .setX509CRL(CRLCodec.getX509CRL(info.getX509CRL()))
+        .setCreationTimestamp(info.getCreationTimestamp())
+        .build();
+  }
+
+  public HddsProtos.CRLInfoProto getProtobuf() throws SCMSecurityException {
+    HddsProtos.CRLInfoProto.Builder builder =
+        HddsProtos.CRLInfoProto.newBuilder();
+
+    return builder.setX509CRL(CRLCodec.getPEMEncodedString(getX509CRL()))
+        .setCreationTimestamp(getCreationTimestamp())
+        .build();
+  }
+
+  public X509CRL getX509CRL() {
+    return x509CRL;
+  }
+
+  public long getCreationTimestamp() {
+    return creationTimestamp;
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less
+   * than, equal to, or greater than the specified object.
+   *
+   * @param o the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object
+   * is less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(@NotNull CRLInfo o) {
+    return this.compare(this, o);
+  }
+
+  /**
+   * Compares its two arguments for order.  Returns a negative integer,
+   * zero, or a positive integer as the first argument is less than, equal
+   * to, or greater than the second.<p>
+   * <p>
+   *
+   * @param o1 the first object to be compared.
+   * @param o2 the second object to be compared.
+   * @return a negative integer, zero, or a positive integer as the
+   * first argument is less than, equal to, or greater than the
+   * second.
+   * @throws NullPointerException if an argument is null and this
+   *                              comparator does not permit null arguments
+   * @throws ClassCastException   if the arguments' types prevent them from
+   *                              being compared by this comparator.
+   */
+  @Override
+  public int compare(CRLInfo o1, CRLInfo o2) {
+    return 0;

Review comment:
       This does not seem to be implemented yet?  

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/crl/CRLInfo.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.crl;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CRLCodec;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.IOException;
+import java.security.cert.CRLException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509CRL;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Class that wraps Certificate Revocation List Info.
+ */
+public class CRLInfo implements Comparator<CRLInfo>,
+    Comparable<CRLInfo> {
+
+  private X509CRL x509CRL;
+  private long creationTimestamp;
+
+  private CRLInfo(X509CRL x509CRL, long creationTimestamp) {
+    this.x509CRL = x509CRL;
+    this.creationTimestamp = creationTimestamp;
+  }
+
+  /**
+   * Constructor for CRLInfo. Needed for serialization findbugs.
+   */
+  public CRLInfo() {
+  }
+
+  public static CRLInfo fromProtobuf(HddsProtos.CRLInfoProto info)
+      throws IOException, CRLException, CertificateException {
+    CRLInfo.Builder builder = new CRLInfo.Builder();
+    return builder
+        .setX509CRL(CRLCodec.getX509CRL(info.getX509CRL()))
+        .setCreationTimestamp(info.getCreationTimestamp())
+        .build();
+  }
+
+  public HddsProtos.CRLInfoProto getProtobuf() throws SCMSecurityException {
+    HddsProtos.CRLInfoProto.Builder builder =
+        HddsProtos.CRLInfoProto.newBuilder();
+
+    return builder.setX509CRL(CRLCodec.getPEMEncodedString(getX509CRL()))
+        .setCreationTimestamp(getCreationTimestamp())
+        .build();
+  }
+
+  public X509CRL getX509CRL() {
+    return x509CRL;
+  }
+
+  public long getCreationTimestamp() {
+    return creationTimestamp;
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less
+   * than, equal to, or greater than the specified object.
+   *
+   * @param o the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object
+   * is less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(@NotNull CRLInfo o) {
+    return this.compare(this, o);
+  }
+
+  /**
+   * Compares its two arguments for order.  Returns a negative integer,
+   * zero, or a positive integer as the first argument is less than, equal
+   * to, or greater than the second.<p>
+   * <p>
+   *
+   * @param o1 the first object to be compared.
+   * @param o2 the second object to be compared.
+   * @return a negative integer, zero, or a positive integer as the
+   * first argument is less than, equal to, or greater than the
+   * second.
+   * @throws NullPointerException if an argument is null and this
+   *                              comparator does not permit null arguments
+   * @throws ClassCastException   if the arguments' types prevent them from
+   *                              being compared by this comparator.
+   */
+  @Override
+  public int compare(CRLInfo o1, CRLInfo o2) {
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    CRLInfo that = (CRLInfo) o;
+
+    return this.getX509CRL().equals(that.x509CRL) &&
+        this.creationTimestamp == that.creationTimestamp;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(x509CRL);
+  }
+

Review comment:
       Can we add a toString override for debugging output, etc?
   

##########
File path: hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java
##########
@@ -213,6 +224,55 @@ public void testRequestCertificateWithInvalidSubject() throws IOException,
     assertNotNull(holder.get());
   }
 
+  @Test
+  public void testRevokeCertificates() throws Exception {
+    String scmId =  RandomStringUtils.randomAlphabetic(4);
+    String clusterId =  RandomStringUtils.randomAlphabetic(4);
+
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        clusterId, scmId, caStore);
+    testCA.init(new SecurityConfig(conf),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    KeyPair keyPair =
+        new HDDSKeyGenerator(conf).generateKey();
+    PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
+        .addDnsName("hadoop.apache.org")
+        .addIpAddress("8.8.8.8")
+        .setCA(false)
+        .setSubject("testCA")
+        .setConfiguration(conf)
+        .setKey(keyPair)
+        .build();
+
+    // Let us convert this to a string to mimic the common use case.
+    String csrString = CertificateSignRequest.getEncodedString(csr);
+
+    Future<X509CertificateHolder> holder = testCA.requestCertificate(csrString,
+        CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
+
+    X509Certificate certificate =
+        new JcaX509CertificateConverter().getCertificate(holder.get());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(certificate);
+    Future<Boolean> revoked = testCA.revokeCertificates(certs,
+        CRLReason.keyCompromise, new SecurityConfig(conf));
+
+    // Revoking a valid certificate should return true.
+    assertTrue(revoked.get());
+

Review comment:
       can we verify that after revoke, getCertificateByID from ca with valid  and invalid type, respectively?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =
+      new DBColumnFamilyDefinition<>(
+          "crlInfo",
+          Long.class,
+          new LongCodec(),
+          CRLInfo.class,
+          new CRLInfoCodec());
+
+  public static final DBColumnFamilyDefinition<String, Long>
+      CRL_SEQUENCE_ID =

Review comment:
       It's a bit overkill to use a table to just save the crl sequence id. Any chance to put it into the SCM version file?

##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =

Review comment:
       NIT: CRL_INFO=>CRLS to be consistent with other table definition.

##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMCertStore.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.scm.server;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateStore;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.crl.CRLInfo;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.bouncycastle.asn1.x509.CRLReason;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+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.nio.file.Files;
+import java.security.KeyPair;
+import java.security.cert.X509CRLEntry;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CRL_SEQUENCE_ID_KEY;
+
+/**
+ * Test class for @{@link SCMCertStore}.
+ */
+public class TestSCMCertStore {
+
+  private static final String COMPONENT_NAME = "scm";
+  private static final Long INITIAL_SEQUENCE_ID = 1L;
+
+  private OzoneConfiguration config;
+  private SCMMetadataStore scmMetadataStore;
+  private SCMCertStore scmCertStore;
+  private SecurityConfig securityConfig;
+  private X509Certificate x509Certificate;
+  private KeyPair keyPair;
+
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    config.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempDir.newFolder().getAbsolutePath());
+
+    securityConfig = new SecurityConfig(config);
+  }
+
+  @Before
+  public void initDbStore() throws IOException {
+    scmMetadataStore = new SCMMetadataStoreImpl(config);
+    scmCertStore = new SCMCertStore(scmMetadataStore, INITIAL_SEQUENCE_ID);
+  }
+
+  @Before
+  public void generateCertificate() throws Exception {
+    Files.createDirectories(securityConfig.getKeyLocation(COMPONENT_NAME));
+    x509Certificate = generateX509Cert(null);
+  }
+
+  @After
+  public void destroyDbStore() throws Exception {
+    if (scmMetadataStore.getStore() != null) {
+      scmMetadataStore.getStore().close();
+    }
+  }
+
+  @Test
+  public void testRevokeCertificates() throws Exception {
+
+    BigInteger serialID = x509Certificate.getSerialNumber();
+    scmCertStore.storeValidCertificate(serialID, x509Certificate);
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+        CertificateStore.CertType.VALID_CERTS));
+
+    X509CertificateHolder caCertificateHolder =
+        new X509CertificateHolder(generateX509Cert(keyPair).getEncoded());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(x509Certificate);
+    scmCertStore.revokeCertificates(certs,
+        caCertificateHolder,
+        CRLReason.unspecified, securityConfig,
+        keyPair);
+
+    Assert.assertNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.VALID_CERTS));
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.REVOKED_CERTS));
+
+    // CRL Info table should have a CRL with sequence id
+    Assert.assertEquals(
+        INITIAL_SEQUENCE_ID + 1L,
+        (long) scmMetadataStore.getCRLInfoTable().iterator().next().getKey());
+
+    // Check the sequence ID table for latest sequence id
+    Assert.assertEquals(INITIAL_SEQUENCE_ID + 1L, (long)
+        scmMetadataStore.getCRLSequenceIdTable().get(CRL_SEQUENCE_ID_KEY));
+
+    CRLInfo crlInfo =
+        scmMetadataStore.getCRLInfoTable().iterator().next().getValue();
+
+    Set<? extends X509CRLEntry> revokedCertificates =
+        crlInfo.getX509CRL().getRevokedCertificates();
+    Assert.assertEquals(1L, revokedCertificates.size());
+    Assert.assertEquals(x509Certificate.getSerialNumber(),
+        revokedCertificates.iterator().next().getSerialNumber());
+
+    // Now trying to revoke the already revoked certificate should result in
+    // a warning message and no-op. It should not create a new CRL.
+    scmCertStore.revokeCertificates(certs,

Review comment:
       How does the caller know that has already been revoked? a warning message won't work at API level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/crl/CRLInfo.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.crl;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CRLCodec;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.IOException;
+import java.security.cert.CRLException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509CRL;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Class that wraps Certificate Revocation List Info.
+ */
+public class CRLInfo implements Comparator<CRLInfo>,
+    Comparable<CRLInfo> {
+
+  private X509CRL x509CRL;
+  private long creationTimestamp;
+
+  private CRLInfo(X509CRL x509CRL, long creationTimestamp) {
+    this.x509CRL = x509CRL;
+    this.creationTimestamp = creationTimestamp;
+  }
+
+  /**
+   * Constructor for CRLInfo. Needed for serialization findbugs.
+   */
+  public CRLInfo() {
+  }
+
+  public static CRLInfo fromProtobuf(HddsProtos.CRLInfoProto info)
+      throws IOException, CRLException, CertificateException {
+    CRLInfo.Builder builder = new CRLInfo.Builder();
+    return builder
+        .setX509CRL(CRLCodec.getX509CRL(info.getX509CRL()))
+        .setCreationTimestamp(info.getCreationTimestamp())
+        .build();
+  }
+
+  public HddsProtos.CRLInfoProto getProtobuf() throws SCMSecurityException {
+    HddsProtos.CRLInfoProto.Builder builder =
+        HddsProtos.CRLInfoProto.newBuilder();
+
+    return builder.setX509CRL(CRLCodec.getPEMEncodedString(getX509CRL()))
+        .setCreationTimestamp(getCreationTimestamp())
+        .build();
+  }
+
+  public X509CRL getX509CRL() {
+    return x509CRL;
+  }
+
+  public long getCreationTimestamp() {
+    return creationTimestamp;
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less
+   * than, equal to, or greater than the specified object.
+   *
+   * @param o the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object
+   * is less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(@NotNull CRLInfo o) {
+    return this.compare(this, o);
+  }
+
+  /**
+   * Compares its two arguments for order.  Returns a negative integer,
+   * zero, or a positive integer as the first argument is less than, equal
+   * to, or greater than the second.<p>
+   * <p>
+   *
+   * @param o1 the first object to be compared.
+   * @param o2 the second object to be compared.
+   * @return a negative integer, zero, or a positive integer as the
+   * first argument is less than, equal to, or greater than the
+   * second.
+   * @throws NullPointerException if an argument is null and this
+   *                              comparator does not permit null arguments
+   * @throws ClassCastException   if the arguments' types prevent them from
+   *                              being compared by this comparator.
+   */
+  @Override
+  public int compare(CRLInfo o1, CRLInfo o2) {
+    return 0;

Review comment:
       This does not seem to be implemented yet?  




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =
+      new DBColumnFamilyDefinition<>(
+          "crlInfo",
+          Long.class,
+          new LongCodec(),
+          CRLInfo.class,
+          new CRLInfoCodec());
+
+  public static final DBColumnFamilyDefinition<String, Long>
+      CRL_SEQUENCE_ID =

Review comment:
       Good point. I'm OK with CRL sequence id in the table for atomic CRL operations using rocksDB. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java
##########
@@ -70,35 +85,69 @@ public void storeValidCertificate(BigInteger serialID,
   }
 
   @Override
-  public void revokeCertificate(BigInteger serialID) throws IOException {
+  public Optional<Long> revokeCertificates(
+      List<BigInteger> serialIDs,
+      X509CertificateHolder caCertificateHolder,
+      CRLReason reason,

Review comment:
       Can we add a revoke time parameter for this interface method so that we don't need to modify the interface later to support scheduled revoke? The caller can pass now() until the scheduled revoke is implemented. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao merged pull request #1874: HDDS-4482. SCM should be able to persist CRL

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


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =

Review comment:
       NIT: CRL_INFO=>CRLS to be consistent with other table definition.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMCertStore.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.scm.server;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateStore;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.crl.CRLInfo;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.bouncycastle.asn1.x509.CRLReason;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+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.nio.file.Files;
+import java.security.KeyPair;
+import java.security.cert.X509CRLEntry;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CRL_SEQUENCE_ID_KEY;
+
+/**
+ * Test class for @{@link SCMCertStore}.
+ */
+public class TestSCMCertStore {
+
+  private static final String COMPONENT_NAME = "scm";
+  private static final Long INITIAL_SEQUENCE_ID = 1L;
+
+  private OzoneConfiguration config;
+  private SCMMetadataStore scmMetadataStore;
+  private SCMCertStore scmCertStore;
+  private SecurityConfig securityConfig;
+  private X509Certificate x509Certificate;
+  private KeyPair keyPair;
+
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    config.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempDir.newFolder().getAbsolutePath());
+
+    securityConfig = new SecurityConfig(config);
+  }
+
+  @Before
+  public void initDbStore() throws IOException {
+    scmMetadataStore = new SCMMetadataStoreImpl(config);
+    scmCertStore = new SCMCertStore(scmMetadataStore, INITIAL_SEQUENCE_ID);
+  }
+
+  @Before
+  public void generateCertificate() throws Exception {
+    Files.createDirectories(securityConfig.getKeyLocation(COMPONENT_NAME));
+    x509Certificate = generateX509Cert(null);
+  }
+
+  @After
+  public void destroyDbStore() throws Exception {
+    if (scmMetadataStore.getStore() != null) {
+      scmMetadataStore.getStore().close();
+    }
+  }
+
+  @Test
+  public void testRevokeCertificates() throws Exception {
+
+    BigInteger serialID = x509Certificate.getSerialNumber();
+    scmCertStore.storeValidCertificate(serialID, x509Certificate);
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+        CertificateStore.CertType.VALID_CERTS));
+
+    X509CertificateHolder caCertificateHolder =
+        new X509CertificateHolder(generateX509Cert(keyPair).getEncoded());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(x509Certificate);
+    scmCertStore.revokeCertificates(certs,
+        caCertificateHolder,
+        CRLReason.unspecified, securityConfig,
+        keyPair);
+
+    Assert.assertNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.VALID_CERTS));
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.REVOKED_CERTS));
+
+    // CRL Info table should have a CRL with sequence id
+    Assert.assertEquals(
+        INITIAL_SEQUENCE_ID + 1L,
+        (long) scmMetadataStore.getCRLInfoTable().iterator().next().getKey());
+
+    // Check the sequence ID table for latest sequence id
+    Assert.assertEquals(INITIAL_SEQUENCE_ID + 1L, (long)
+        scmMetadataStore.getCRLSequenceIdTable().get(CRL_SEQUENCE_ID_KEY));
+
+    CRLInfo crlInfo =
+        scmMetadataStore.getCRLInfoTable().iterator().next().getValue();
+
+    Set<? extends X509CRLEntry> revokedCertificates =
+        crlInfo.getX509CRL().getRevokedCertificates();
+    Assert.assertEquals(1L, revokedCertificates.size());
+    Assert.assertEquals(x509Certificate.getSerialNumber(),
+        revokedCertificates.iterator().next().getSerialNumber());
+
+    // Now trying to revoke the already revoked certificate should result in
+    // a warning message and no-op. It should not create a new CRL.
+    scmCertStore.revokeCertificates(certs,

Review comment:
       Good catch @xiaoyuyao. I have now changed the signature of this method to return an optional CRL sequence id. It will return empty if no new CRL was generated. Otherwise, it will return the sequence id which can be returned to the users to keep track of the status of revocations in the future.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMCertStore.java
##########
@@ -0,0 +1,235 @@
+/*
+ * 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.scm.server;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateStore;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.crl.CRLInfo;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.bouncycastle.asn1.x509.CRLReason;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+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.nio.file.Files;
+import java.security.KeyPair;
+import java.security.cert.X509CRLEntry;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.hadoop.ozone.OzoneConsts.CRL_SEQUENCE_ID_KEY;
+
+/**
+ * Test class for @{@link SCMCertStore}.
+ */
+public class TestSCMCertStore {
+
+  private static final String COMPONENT_NAME = "scm";
+  private static final Long INITIAL_SEQUENCE_ID = 1L;
+
+  private OzoneConfiguration config;
+  private SCMMetadataStore scmMetadataStore;
+  private SCMCertStore scmCertStore;
+  private SecurityConfig securityConfig;
+  private X509Certificate x509Certificate;
+  private KeyPair keyPair;
+
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    config.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempDir.newFolder().getAbsolutePath());
+
+    securityConfig = new SecurityConfig(config);
+  }
+
+  @Before
+  public void initDbStore() throws IOException {
+    scmMetadataStore = new SCMMetadataStoreImpl(config);
+    scmCertStore = new SCMCertStore(scmMetadataStore, INITIAL_SEQUENCE_ID);
+  }
+
+  @Before
+  public void generateCertificate() throws Exception {
+    Files.createDirectories(securityConfig.getKeyLocation(COMPONENT_NAME));
+    x509Certificate = generateX509Cert(null);
+  }
+
+  @After
+  public void destroyDbStore() throws Exception {
+    if (scmMetadataStore.getStore() != null) {
+      scmMetadataStore.getStore().close();
+    }
+  }
+
+  @Test
+  public void testRevokeCertificates() throws Exception {
+
+    BigInteger serialID = x509Certificate.getSerialNumber();
+    scmCertStore.storeValidCertificate(serialID, x509Certificate);
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+        CertificateStore.CertType.VALID_CERTS));
+
+    X509CertificateHolder caCertificateHolder =
+        new X509CertificateHolder(generateX509Cert(keyPair).getEncoded());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(x509Certificate);
+    scmCertStore.revokeCertificates(certs,
+        caCertificateHolder,
+        CRLReason.unspecified, securityConfig,
+        keyPair);
+
+    Assert.assertNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.VALID_CERTS));
+
+    Assert.assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.REVOKED_CERTS));
+
+    // CRL Info table should have a CRL with sequence id
+    Assert.assertEquals(
+        INITIAL_SEQUENCE_ID + 1L,
+        (long) scmMetadataStore.getCRLInfoTable().iterator().next().getKey());
+
+    // Check the sequence ID table for latest sequence id
+    Assert.assertEquals(INITIAL_SEQUENCE_ID + 1L, (long)
+        scmMetadataStore.getCRLSequenceIdTable().get(CRL_SEQUENCE_ID_KEY));
+
+    CRLInfo crlInfo =
+        scmMetadataStore.getCRLInfoTable().iterator().next().getValue();
+
+    Set<? extends X509CRLEntry> revokedCertificates =
+        crlInfo.getX509CRL().getRevokedCertificates();
+    Assert.assertEquals(1L, revokedCertificates.size());
+    Assert.assertEquals(x509Certificate.getSerialNumber(),
+        revokedCertificates.iterator().next().getSerialNumber());
+
+    // Now trying to revoke the already revoked certificate should result in
+    // a warning message and no-op. It should not create a new CRL.
+    scmCertStore.revokeCertificates(certs,

Review comment:
       How does the caller know that has already been revoked? a warning message won't work at API level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on pull request #1874: HDDS-4482. SCM should be able to persist CRL

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


   @xiaoyuyao I have fixed all the concerns you had. Can you please take another look? Thank you!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on pull request #1874: HDDS-4482. SCM should be able to persist CRL

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


   @xiaoyuyao Thanks for the detailed review! I have posted a patch to resolve most of your comments. Please review again and let me know of any more changes needed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateStore.java
##########
@@ -47,12 +50,21 @@ void storeValidCertificate(BigInteger serialID,
                              X509Certificate certificate) throws IOException;
 
   /**
-   * Moves a certificate in a transactional manner from valid certificate to
+   * Adds the certificates to be revoked to a new CRL and moves all the
+   * certificates in a transactional manner from valid certificate to
    * revoked certificate state.
-   * @param serialID - Serial ID of the certificate.
+   * @param certificates - List of X509 Certificates to be revoked.
+   * @param caCertificateHolder - X509 Certificate Holder of the CA.
+   * @param reason - CRLReason for revocation.
+   * @param securityConfig - Security Configuration.
+   * @param keyPair - Public and Private key of the CA.
    * @throws IOException
    */
-  void revokeCertificate(BigInteger serialID) throws IOException;
+  void revokeCertificates(List<X509Certificate> certificates,
+                          X509CertificateHolder caCertificateHolder,
+                          int reason, SecurityConfig securityConfig,
+                          KeyPair keyPair)

Review comment:
       can we abstract the CRL handling into a separate class (like Approver for CSR) without bringing CA keypair into the certificate store interface and class?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =
+      new DBColumnFamilyDefinition<>(
+          "crlInfo",
+          Long.class,
+          new LongCodec(),
+          CRLInfo.class,
+          new CRLInfoCodec());
+
+  public static final DBColumnFamilyDefinition<String, Long>
+      CRL_SEQUENCE_ID =

Review comment:
       It's a bit overkill to use a table to just save the crl sequence id. Any chance to put it into the SCM version file?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMCertStore.java
##########
@@ -70,35 +85,69 @@ public void storeValidCertificate(BigInteger serialID,
   }
 
   @Override
-  public void revokeCertificate(BigInteger serialID) throws IOException {
+  public Optional<Long> revokeCertificates(
+      List<BigInteger> serialIDs,
+      X509CertificateHolder caCertificateHolder,
+      CRLReason reason,
+      CRLApprover crlApprover)
+      throws IOException {
+    Date now = new Date();
+    X509v2CRLBuilder builder =
+        new X509v2CRLBuilder(caCertificateHolder.getIssuer(), now);
+    List<X509Certificate> certsToRevoke = new ArrayList<>();
+    X509CRL crl;
+    Optional<Long> sequenceId = Optional.empty();

Review comment:
       Can you add some javadoc on the interface for the case where non of the certs are revoked due to either not found or already revoked? 
   In this case, the caller need to check the returned Optional for empty. 

##########
File path: hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMCertStore.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.scm.server;
+
+import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
+import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStoreImpl;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CRLApprover;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.CertificateStore;
+import org.apache.hadoop.hdds.security.x509.certificate.authority.DefaultCRLApprover;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.apache.hadoop.hdds.security.x509.crl.CRLInfo;
+import org.apache.hadoop.hdds.utils.db.Table;
+import org.apache.hadoop.hdds.utils.db.TableIterator;
+import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
+import org.bouncycastle.asn1.x509.CRLReason;
+import org.bouncycastle.cert.X509CertificateHolder;
+import org.junit.After;
+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.nio.file.Files;
+import java.security.KeyPair;
+import java.security.cert.X509CRLEntry;
+import java.security.cert.X509Certificate;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.apache.hadoop.ozone.OzoneConsts.CRL_SEQUENCE_ID_KEY;
+
+/**
+ * Test class for @{@link SCMCertStore}.
+ */
+public class TestSCMCertStore {
+
+  private static final String COMPONENT_NAME = "scm";
+  private static final Long INITIAL_SEQUENCE_ID = 1L;
+
+  private OzoneConfiguration config;
+  private SCMMetadataStore scmMetadataStore;
+  private SCMCertStore scmCertStore;
+  private SecurityConfig securityConfig;
+  private X509Certificate x509Certificate;
+  private KeyPair keyPair;
+  private CRLApprover crlApprover;
+
+  @Rule
+  public final TemporaryFolder tempDir = new TemporaryFolder();
+
+  @Before
+  public void setUp() throws Exception {
+    config = new OzoneConfiguration();
+
+    config.set(HddsConfigKeys.OZONE_METADATA_DIRS,
+        tempDir.newFolder().getAbsolutePath());
+
+    securityConfig = new SecurityConfig(config);
+    keyPair = KeyStoreTestUtil.generateKeyPair("RSA");
+  }
+
+  @Before
+  public void initDbStore() throws IOException {
+    scmMetadataStore = new SCMMetadataStoreImpl(config);
+    scmCertStore = new SCMCertStore(scmMetadataStore, INITIAL_SEQUENCE_ID);
+  }
+
+  @Before
+  public void generateCertificate() throws Exception {
+    Files.createDirectories(securityConfig.getKeyLocation(COMPONENT_NAME));
+    x509Certificate = generateX509Cert();
+  }
+
+  @Before
+  public void initCRLApprover() {
+    crlApprover = new DefaultCRLApprover(securityConfig,
+        keyPair.getPrivate());
+  }
+
+  @After
+  public void destroyDbStore() throws Exception {
+    if (scmMetadataStore.getStore() != null) {
+      scmMetadataStore.getStore().close();
+    }
+  }
+
+  @Test
+  public void testRevokeCertificates() throws Exception {
+
+    BigInteger serialID = x509Certificate.getSerialNumber();
+    scmCertStore.storeValidCertificate(serialID, x509Certificate);
+
+    assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+        CertificateStore.CertType.VALID_CERTS));
+
+    X509CertificateHolder caCertificateHolder =
+        new X509CertificateHolder(generateX509Cert().getEncoded());
+    List<BigInteger> certs = new ArrayList<>();
+    certs.add(x509Certificate.getSerialNumber());
+    Optional<Long> sequenceId = scmCertStore.revokeCertificates(certs,
+        caCertificateHolder,
+        CRLReason.lookup(CRLReason.keyCompromise), crlApprover);
+
+    assertTrue(sequenceId.isPresent());
+    assertEquals(INITIAL_SEQUENCE_ID + 1L, (long) sequenceId.get());
+
+    assertNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.VALID_CERTS));
+
+    assertNotNull(
+        scmCertStore.getCertificateByID(serialID,
+            CertificateStore.CertType.REVOKED_CERTS));
+
+    // CRL Info table should have a CRL with sequence id
+    assertNotNull(scmMetadataStore.getCRLInfoTable()
+        .get(sequenceId.get()));
+
+    // Check the sequence ID table for latest sequence id
+    assertEquals(INITIAL_SEQUENCE_ID + 1L, (long)
+        scmMetadataStore.getCRLSequenceIdTable().get(CRL_SEQUENCE_ID_KEY));
+
+    CRLInfo crlInfo =
+        scmMetadataStore.getCRLInfoTable().get(sequenceId.get());
+
+    Set<? extends X509CRLEntry> revokedCertificates =
+        crlInfo.getX509CRL().getRevokedCertificates();
+    assertEquals(1L, revokedCertificates.size());
+    assertEquals(x509Certificate.getSerialNumber(),
+        revokedCertificates.iterator().next().getSerialNumber());
+
+    // Now trying to revoke the already revoked certificate should result in
+    // a warning message and no-op. It should not create a new CRL.
+    sequenceId = scmCertStore.revokeCertificates(certs,
+        caCertificateHolder,
+        CRLReason.lookup(CRLReason.unspecified), crlApprover);
+
+    assertFalse(sequenceId.isPresent());
+
+    int size = 0;
+    TableIterator<Long, ? extends Table.KeyValue<Long, CRLInfo>> iter =
+        scmMetadataStore.getCRLInfoTable().iterator();
+
+    while(iter.hasNext()) {
+      size++;
+      iter.next();
+    }
+
+    assertEquals(1, size);
+
+    // Generate 3 more certificates and revoke 2 of them
+    List<BigInteger> newSerialIDs = new ArrayList<>();
+    for (int i = 0; i<3; i++) {
+      X509Certificate cert = generateX509Cert();
+      scmCertStore.storeValidCertificate(cert.getSerialNumber(), cert);
+      newSerialIDs.add(cert.getSerialNumber());
+    }
+
+    // Add the first 2 certificates to the revocation list
+    sequenceId = scmCertStore.revokeCertificates(newSerialIDs.subList(0, 2),
+        caCertificateHolder,
+        CRLReason.lookup(CRLReason.aACompromise), crlApprover);
+
+    // This should create a CRL with sequence id INITIAL_SEQUENCE_ID + 2
+    // And contain 2 certificates in it
+    assertTrue(sequenceId.isPresent());
+    assertEquals(INITIAL_SEQUENCE_ID + 2L, (long) sequenceId.get());
+
+    // Check the sequence ID table for latest sequence id
+    assertEquals(INITIAL_SEQUENCE_ID + 2L, (long)
+        scmMetadataStore.getCRLSequenceIdTable().get(CRL_SEQUENCE_ID_KEY));
+
+    CRLInfo newCrlInfo = scmMetadataStore.getCRLInfoTable()
+        .get(sequenceId.get());
+    revokedCertificates = newCrlInfo.getX509CRL().getRevokedCertificates();
+    assertEquals(2L, revokedCertificates.size());
+    assertNotNull(
+        revokedCertificates.stream().filter(c ->
+            c.getSerialNumber().equals(newSerialIDs.get(0)))
+            .findAny());
+
+    assertNotNull(
+        revokedCertificates.stream().filter(c ->
+            c.getSerialNumber().equals(newSerialIDs.get(1)))
+            .findAny());
+
+    // Valid certs table should have 1 cert
+    size = 0;
+    TableIterator<BigInteger,

Review comment:
       NIT: this can be a helper function like getTableSize() to avoid duplicated code. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultCAServer.java
##########
@@ -272,20 +272,22 @@ private X509CertificateHolder signAndStoreCertificate(LocalDate beginDate,
   }
 
   @Override
-  public Future<Boolean> revokeCertificate(X509Certificate certificate,
-      CertificateApprover.ApprovalType approverType)
-      throws SCMSecurityException {
+  public Future<Boolean> revokeCertificates(List<X509Certificate> certificates,
+                                            int reason,
+                                            SecurityConfig securityConfig) {
     CompletableFuture<Boolean> revoked = new CompletableFuture<>();
-    if (certificate == null) {
+    if (certificates == null || certificates.isEmpty()) {

Review comment:
       NIT: we can use Collection#isEmptyOrNull here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java
##########
@@ -213,6 +224,55 @@ public void testRequestCertificateWithInvalidSubject() throws IOException,
     assertNotNull(holder.get());
   }
 
+  @Test
+  public void testRevokeCertificates() throws Exception {
+    String scmId =  RandomStringUtils.randomAlphabetic(4);
+    String clusterId =  RandomStringUtils.randomAlphabetic(4);
+
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        clusterId, scmId, caStore);
+    testCA.init(new SecurityConfig(conf),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    KeyPair keyPair =
+        new HDDSKeyGenerator(conf).generateKey();
+    PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
+        .addDnsName("hadoop.apache.org")
+        .addIpAddress("8.8.8.8")
+        .setCA(false)
+        .setSubject("testCA")
+        .setConfiguration(conf)
+        .setKey(keyPair)
+        .build();
+
+    // Let us convert this to a string to mimic the common use case.
+    String csrString = CertificateSignRequest.getEncodedString(csr);
+
+    Future<X509CertificateHolder> holder = testCA.requestCertificate(csrString,
+        CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
+
+    X509Certificate certificate =
+        new JcaX509CertificateConverter().getCertificate(holder.get());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(certificate);
+    Future<Boolean> revoked = testCA.revokeCertificates(certs,
+        CRLReason.keyCompromise, new SecurityConfig(conf));
+
+    // Revoking a valid certificate should return true.
+    assertTrue(revoked.get());
+

Review comment:
       can we verify that after revoke, getCertificateByID from ca with valid  and invalid type, respectively?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
##########
@@ -101,26 +101,22 @@ X509Certificate getCertificate(String certSerialId)
   /**
    * Revokes a Certificate issued by this CertificateServer.
    *
-   * @param certificate - Certificate to revoke
-   * @param approver - Approval process to follow.
+   * @param certificates - List of Certificates to revoke.
+   * @param reason - Reason for revocation.
+   * @param securityConfig - Security Configuration.
    * @return Future that tells us what happened.
-   * @throws SCMSecurityException - on Error.
-   */
-  Future<Boolean> revokeCertificate(X509Certificate certificate,
-      ApprovalType approver) throws SCMSecurityException;
-
-  /**
-   * TODO : CRL, OCSP etc. Later. This is the start of a CertificateServer
-   * framework.

Review comment:
       NIT: javadoc?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
##########
@@ -101,26 +101,22 @@ X509Certificate getCertificate(String certSerialId)
   /**
    * Revokes a Certificate issued by this CertificateServer.
    *
-   * @param certificate - Certificate to revoke
-   * @param approver - Approval process to follow.
+   * @param certificates - List of Certificates to revoke.
+   * @param reason - Reason for revocation.
+   * @param securityConfig - Security Configuration.
    * @return Future that tells us what happened.
-   * @throws SCMSecurityException - on Error.
-   */
-  Future<Boolean> revokeCertificate(X509Certificate certificate,
-      ApprovalType approver) throws SCMSecurityException;
-
-  /**
-   * TODO : CRL, OCSP etc. Later. This is the start of a CertificateServer
-   * framework.
    */
+  Future<Boolean> revokeCertificates(List<X509Certificate> certificates,
+                                     int reason,

Review comment:
       Can the reason be a Enum?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/metadata/SCMDBDefinition.java
##########
@@ -73,13 +75,30 @@
 
   public static final DBColumnFamilyDefinition<ContainerID, ContainerInfo>
       CONTAINERS =
-      new DBColumnFamilyDefinition<ContainerID, ContainerInfo>(
+      new DBColumnFamilyDefinition<>(
           "containers",
           ContainerID.class,
           new ContainerIDCodec(),
           ContainerInfo.class,
           new ContainerInfoCodec());
 
+  public static final DBColumnFamilyDefinition<Long, CRLInfo> CRL_INFO =
+      new DBColumnFamilyDefinition<>(
+          "crlInfo",
+          Long.class,
+          new LongCodec(),
+          CRLInfo.class,
+          new CRLInfoCodec());
+
+  public static final DBColumnFamilyDefinition<String, Long>
+      CRL_SEQUENCE_ID =

Review comment:
       Good suggestion @xiaoyuyao. But, having this sequence id stored in RocksDB gives us the ability to batch transactions when revoke certificates operation is executed. If we use the version file to keep track of this id, we will lose this ability and might cause conflicts with the CRL sequence id. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/crl/CRLInfo.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.crl;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CRLCodec;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.IOException;
+import java.security.cert.CRLException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509CRL;
+import java.util.Comparator;
+import java.util.Objects;
+
+/**
+ * Class that wraps Certificate Revocation List Info.
+ */
+public class CRLInfo implements Comparator<CRLInfo>,
+    Comparable<CRLInfo> {
+
+  private X509CRL x509CRL;
+  private long creationTimestamp;
+
+  private CRLInfo(X509CRL x509CRL, long creationTimestamp) {
+    this.x509CRL = x509CRL;
+    this.creationTimestamp = creationTimestamp;
+  }
+
+  /**
+   * Constructor for CRLInfo. Needed for serialization findbugs.
+   */
+  public CRLInfo() {
+  }
+
+  public static CRLInfo fromProtobuf(HddsProtos.CRLInfoProto info)
+      throws IOException, CRLException, CertificateException {
+    CRLInfo.Builder builder = new CRLInfo.Builder();
+    return builder
+        .setX509CRL(CRLCodec.getX509CRL(info.getX509CRL()))
+        .setCreationTimestamp(info.getCreationTimestamp())
+        .build();
+  }
+
+  public HddsProtos.CRLInfoProto getProtobuf() throws SCMSecurityException {
+    HddsProtos.CRLInfoProto.Builder builder =
+        HddsProtos.CRLInfoProto.newBuilder();
+
+    return builder.setX509CRL(CRLCodec.getPEMEncodedString(getX509CRL()))
+        .setCreationTimestamp(getCreationTimestamp())
+        .build();
+  }
+
+  public X509CRL getX509CRL() {
+    return x509CRL;
+  }
+
+  public long getCreationTimestamp() {
+    return creationTimestamp;
+  }
+
+  /**
+   * Compares this object with the specified object for order.  Returns a
+   * negative integer, zero, or a positive integer as this object is less
+   * than, equal to, or greater than the specified object.
+   *
+   * @param o the object to be compared.
+   * @return a negative integer, zero, or a positive integer as this object
+   * is less than, equal to, or greater than the specified object.
+   * @throws NullPointerException if the specified object is null
+   * @throws ClassCastException   if the specified object's type prevents it
+   *                              from being compared to this object.
+   */
+  @Override
+  public int compareTo(@NotNull CRLInfo o) {
+    return this.compare(this, o);
+  }
+
+  /**
+   * Compares its two arguments for order.  Returns a negative integer,
+   * zero, or a positive integer as the first argument is less than, equal
+   * to, or greater than the second.<p>
+   * <p>
+   *
+   * @param o1 the first object to be compared.
+   * @param o2 the second object to be compared.
+   * @return a negative integer, zero, or a positive integer as the
+   * first argument is less than, equal to, or greater than the
+   * second.
+   * @throws NullPointerException if an argument is null and this
+   *                              comparator does not permit null arguments
+   * @throws ClassCastException   if the arguments' types prevent them from
+   *                              being compared by this comparator.
+   */
+  @Override
+  public int compare(CRLInfo o1, CRLInfo o2) {
+    return 0;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    CRLInfo that = (CRLInfo) o;
+
+    return this.getX509CRL().equals(that.x509CRL) &&
+        this.creationTimestamp == that.creationTimestamp;
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(x509CRL);
+  }
+

Review comment:
       Can we add a toString override for debugging output, etc?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [ozone] vivekratnavel commented on a change in pull request #1874: HDDS-4482. SCM should be able to persist CRL

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



##########
File path: hadoop-hdds/framework/src/test/java/org/apache/hadoop/hdds/security/x509/certificate/authority/TestDefaultCAServer.java
##########
@@ -213,6 +224,55 @@ public void testRequestCertificateWithInvalidSubject() throws IOException,
     assertNotNull(holder.get());
   }
 
+  @Test
+  public void testRevokeCertificates() throws Exception {
+    String scmId =  RandomStringUtils.randomAlphabetic(4);
+    String clusterId =  RandomStringUtils.randomAlphabetic(4);
+
+    CertificateServer testCA = new DefaultCAServer("testCA",
+        clusterId, scmId, caStore);
+    testCA.init(new SecurityConfig(conf),
+        CertificateServer.CAType.SELF_SIGNED_CA);
+
+    KeyPair keyPair =
+        new HDDSKeyGenerator(conf).generateKey();
+    PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
+        .addDnsName("hadoop.apache.org")
+        .addIpAddress("8.8.8.8")
+        .setCA(false)
+        .setSubject("testCA")
+        .setConfiguration(conf)
+        .setKey(keyPair)
+        .build();
+
+    // Let us convert this to a string to mimic the common use case.
+    String csrString = CertificateSignRequest.getEncodedString(csr);
+
+    Future<X509CertificateHolder> holder = testCA.requestCertificate(csrString,
+        CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
+
+    X509Certificate certificate =
+        new JcaX509CertificateConverter().getCertificate(holder.get());
+    List<X509Certificate> certs = new ArrayList<>();
+    certs.add(certificate);
+    Future<Boolean> revoked = testCA.revokeCertificates(certs,
+        CRLReason.keyCompromise, new SecurityConfig(conf));
+
+    // Revoking a valid certificate should return true.
+    assertTrue(revoked.get());
+

Review comment:
       I have added unit tests to test all those scenarios in TestSCMCertStore.java. Those kinds of tests are not possible for TestDefaultCAServer because we use a MockCAServer instance to unit test the CA server. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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