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 2020/12/14 23:58:23 UTC

[GitHub] [ozone] vivekratnavel commented on a change in pull request #1662: HDDS-4507. Add SCM CA CLI to query certificate.

vivekratnavel commented on a change in pull request #1662:
URL: https://github.com/apache/ozone/pull/1662#discussion_r542908082



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/protocol/SCMSecurityProtocol.java
##########
@@ -77,4 +80,16 @@ String getOMCertificate(OzoneManagerDetailsProto omDetails,
    */
   String getCACertificate() throws IOException;
 
+  /**
+   * Get list of certificates meet the query criteria.
+   *
+   * @param type            - node type: OM/SCM/DN.
+   * @param startSerialId   - start certificate serial id.
+   * @param count           - max number of certificates returned in a batch.
+   * @param isRevoked       - whether list for revoked certs only.
+   * @return list of PEM encoded certificate strings.
+   */
+  List<String> listCertificate(HddsProtos.NodeType type, long startSerialId,

Review comment:
       Nit: getCertificates may be a better name here
   ```suggestion
     List<String> getCertificates(HddsProtos.NodeType type, long startSerialId,
   ```

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/cert/InfoSubcommand.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.cli.cert;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.cli.GenericParentCommand;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Model.CommandSpec;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.Spec;
+
+/**
+ * This is the handler that process certificate info command.
+ */
+@Command(
+    name = "info",
+    description = "Show detail information of a specific certificate",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+class InfoSubcommand extends ScmCertSubcommand {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InfoSubcommand.class);
+
+  @Spec
+  private CommandSpec spec;
+
+  @Parameters(description = "Serial id of the certificate in decimal.")
+  private String serialId;
+
+  @Override
+  public void execute(SCMSecurityProtocol client) throws IOException {
+    final String certPemStr =
+        client.getCertificate(serialId);
+    Preconditions.checkNotNull(certPemStr,
+        "Certificate can't be found");
+
+    // Print container report info.
+    LOG.info("Certificate id: {}", serialId);
+    boolean verbose = spec.root().userObject() instanceof GenericParentCommand
+        && ((GenericParentCommand) spec.root().userObject()).isVerbose();
+    try {
+      X509Certificate cert = CertificateCodec.getX509Cert(certPemStr);
+      LOG.info(cert.toString());
+    } catch (CertificateException ex) {
+      LOG.error("Fail to get certificate id " + serialId);

Review comment:
       Nit
   ```suggestion
         LOG.error("Failed to get certificate id " + serialId);
   ```

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/cert/InfoSubcommand.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.cli.cert;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.cli.GenericParentCommand;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Model.CommandSpec;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.Spec;
+
+/**
+ * This is the handler that process certificate info command.
+ */
+@Command(
+    name = "info",
+    description = "Show detail information of a specific certificate",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+
+class InfoSubcommand extends ScmCertSubcommand {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(InfoSubcommand.class);
+
+  @Spec
+  private CommandSpec spec;
+
+  @Parameters(description = "Serial id of the certificate in decimal.")
+  private String serialId;
+
+  @Override
+  public void execute(SCMSecurityProtocol client) throws IOException {
+    final String certPemStr =
+        client.getCertificate(serialId);
+    Preconditions.checkNotNull(certPemStr,
+        "Certificate can't be found");
+
+    // Print container report info.
+    LOG.info("Certificate id: {}", serialId);
+    boolean verbose = spec.root().userObject() instanceof GenericParentCommand
+        && ((GenericParentCommand) spec.root().userObject()).isVerbose();
+    try {
+      X509Certificate cert = CertificateCodec.getX509Cert(certPemStr);
+      LOG.info(cert.toString());
+    } catch (CertificateException ex) {
+      LOG.error("Fail to get certificate id " + serialId);
+      throw new IOException("Fail to get certificate id " + serialId, ex);

Review comment:
       Nit
   ```suggestion
         throw new IOException("Failed to get certificate id " + serialId, ex);
   ```

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/cert/InfoSubcommand.java
##########
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.cli.cert;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.cli.GenericParentCommand;
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Model.CommandSpec;
+import picocli.CommandLine.Parameters;
+import picocli.CommandLine.Spec;
+
+/**
+ * This is the handler that process certificate info command.
+ */
+@Command(
+    name = "info",
+    description = "Show detail information of a specific certificate",

Review comment:
       Nit
   ```suggestion
       description = "Show detailed information for a specific certificate",
   ```

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateServer.java
##########
@@ -112,6 +114,16 @@ X509Certificate getCertificate(String certSerialId)
    * framework.
    */
 
+  /**
+   * List certificates.
+   * @param type            - node type: OM/SCM/DN
+   * @param startSerialId   - start certificate serial id
+   * @param count           - max number of certificates returned in a batch
+   * @return
+   * @throws IOException
+   */
+  List<X509Certificate> listCertificate(HddsProtos.NodeType type,

Review comment:
       Nit: getCertificates may be a better name here
   ```suggestion
     List<X509Certificate> getCertificates(HddsProtos.NodeType type,
   ```

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/cert/ListSubcommand.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.cli.cert;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.List;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Help.Visibility;
+import picocli.CommandLine.Option;
+
+/**
+ * This is the handler that process certificate list command.
+ */
+@Command(
+    name = "list",
+    description = "List certificates",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ListSubcommand extends ScmCertSubcommand {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ListSubcommand.class);
+
+  @Option(names = {"-s", "--start"},
+      description = "Certificate serial id to start the iteration",
+      defaultValue = "0", showDefaultValue = Visibility.ALWAYS)
+  private long startSerialId;
+
+  @Option(names = {"-c", "--count"},
+      description = "Maximum number of certificates to list",
+      defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
+  private int count;
+
+  @Option(names = {"-r", "--role"},
+      description = "Filter certificate by the role: om/datanode",
+      defaultValue = "datanode", showDefaultValue = Visibility.ALWAYS)
+  private String role;
+
+  @Option(names = {"-t", "--type"},
+      description = "Filter certificate by the type: valid or revoked",
+      defaultValue = "valid", showDefaultValue = Visibility.ALWAYS)
+  private String type;
+
+  private HddsProtos.NodeType parseCertRole(String r) {
+    if (r.equalsIgnoreCase("om")) {
+      return HddsProtos.NodeType.OM;
+    } else if (r.equalsIgnoreCase("scm")) {
+      return HddsProtos.NodeType.SCM;
+    } else {
+      return HddsProtos.NodeType.DATANODE;
+    }
+  }
+
+  private void printCert(X509Certificate cert) {
+    LOG.info("{}\t{}\t{}\t{}", cert.getSerialNumber(), cert.getNotBefore(),
+        cert.getNotAfter(), cert.getSubjectDN());
+  }
+
+  @Override
+  protected void execute(SCMSecurityProtocol client) throws IOException {
+    boolean isRevoked = type.equalsIgnoreCase("revoked");
+    List<String> certPemList = client.listCertificate(
+        parseCertRole(role), startSerialId, count, isRevoked);
+    LOG.info("Total {} {} certificates: ", certPemList.size(), type);
+    LOG.info("SerialNumber\t\tValid From\t\tValid To\t\tSubjectDN");
+    for (String certPemStr : certPemList) {
+      try {
+        X509Certificate cert = CertificateCodec.getX509Certificate(certPemStr);
+        printCert(cert);
+      } catch (CertificateException ex) {
+        LOG.error("Fail to parse certificate.");

Review comment:
       Nit:
   ```suggestion
           LOG.error("Failed to parse certificate.");
   ```

##########
File path: hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/cert/ListSubcommand.java
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.scm.cli.cert;
+
+import java.io.IOException;
+import java.security.cert.CertificateException;
+import java.security.cert.X509Certificate;
+import java.util.List;
+
+import org.apache.hadoop.hdds.cli.HddsVersionProvider;
+import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
+
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Help.Visibility;
+import picocli.CommandLine.Option;
+
+/**
+ * This is the handler that process certificate list command.
+ */
+@Command(
+    name = "list",
+    description = "List certificates",
+    mixinStandardHelpOptions = true,
+    versionProvider = HddsVersionProvider.class)
+public class ListSubcommand extends ScmCertSubcommand {
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ListSubcommand.class);
+
+  @Option(names = {"-s", "--start"},
+      description = "Certificate serial id to start the iteration",
+      defaultValue = "0", showDefaultValue = Visibility.ALWAYS)
+  private long startSerialId;
+
+  @Option(names = {"-c", "--count"},
+      description = "Maximum number of certificates to list",
+      defaultValue = "20", showDefaultValue = Visibility.ALWAYS)
+  private int count;
+
+  @Option(names = {"-r", "--role"},
+      description = "Filter certificate by the role: om/datanode",
+      defaultValue = "datanode", showDefaultValue = Visibility.ALWAYS)
+  private String role;
+
+  @Option(names = {"-t", "--type"},
+      description = "Filter certificate by the type: valid or revoked",
+      defaultValue = "valid", showDefaultValue = Visibility.ALWAYS)
+  private String type;
+
+  private HddsProtos.NodeType parseCertRole(String r) {
+    if (r.equalsIgnoreCase("om")) {
+      return HddsProtos.NodeType.OM;
+    } else if (r.equalsIgnoreCase("scm")) {
+      return HddsProtos.NodeType.SCM;
+    } else {
+      return HddsProtos.NodeType.DATANODE;
+    }
+  }
+
+  private void printCert(X509Certificate cert) {
+    LOG.info("{}\t{}\t{}\t{}", cert.getSerialNumber(), cert.getNotBefore(),
+        cert.getNotAfter(), cert.getSubjectDN());
+  }
+
+  @Override
+  protected void execute(SCMSecurityProtocol client) throws IOException {
+    boolean isRevoked = type.equalsIgnoreCase("revoked");
+    List<String> certPemList = client.listCertificate(
+        parseCertRole(role), startSerialId, count, isRevoked);
+    LOG.info("Total {} {} certificates: ", certPemList.size(), type);
+    LOG.info("SerialNumber\t\tValid From\t\tValid To\t\tSubjectDN");

Review comment:
       Nit: May be "Valid Until" or "Expiry" will suit better here. And instead of subjectDN, can we use subject?  
   ```suggestion
       LOG.info("SerialNumber\t\tValid From\t\tExpiry\t\tSubject");
   ```




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