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 2022/01/29 00:45:21 UTC

[GitHub] [ozone] aswinshakil opened a new pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

aswinshakil opened a new pull request #3027:
URL: https://github.com/apache/ozone/pull/3027


   ## What changes were proposed in this pull request?
   
   In a secure cluster, Recon requires a Certificate in order to download SCM DB from SCM. The SCM acts as Certificate Authority that issues certificates for Recon.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6212
   
   ## How was this patch tested?
   
   The patch was tested manually.
   


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] aswinshakil commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/ReconCertificateClient.java
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+
+/**
+ * Certificate client for Recon.
+ */
+public class ReconCertificateClient  extends DefaultCertificateClient{

Review comment:
       I'll add the suggested changes. Thank you for the review. 




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] aswinshakil commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java
##########
@@ -129,6 +166,108 @@ public Void call() throws Exception {
     return null;
   }
 
+  /**
+   * Initializes secure Recon.
+   * */
+  private void initializeCertificateClient(OzoneConfiguration conf)
+      throws IOException {
+    LOG.info("Initializing secure Recon.");
+    certClient = new ReconCertificateClient(
+        new SecurityConfig(configuration),
+        reconStorage.getReconCertSerialId());
+
+    CertificateClient.InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful, case:{}.", response);
+      break;
+    case GETCERT:
+      getSCMSignedCert(conf);
+      LOG.info("Successfully stored SCM signed certificate, case:{}.",
+          response);
+      break;
+    case FAILURE:
+      LOG.error("Recon security initialization failed, case:{}.", response);
+      throw new RuntimeException("Recon security initialization failed.");
+    case RECOVER:
+      LOG.error("Recon security initialization failed. Recon certificate is " +
+          "missing.");
+      throw new RuntimeException("Recon security initialization failed.");
+    default:
+      LOG.error("Recon security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("Recon security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   * @param config
+   * */
+  private void getSCMSignedCert(OzoneConfiguration config) {
+    try {
+      PKCS10CertificationRequest csr = getCSR(config);
+      SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+          HddsServerUtil.getScmSecurityClientWithMaxRetry(config);
+      HddsProtos.ReconDetailsProto.Builder reconDetailsProtoBuilder =
+          HddsProtos.ReconDetailsProto.newBuilder()
+              .setHostName(InetAddress.getLocalHost().getHostName())
+              .setClusterId(reconStorage.getClusterID())
+              .setUuid(reconStorage.getReconId());
+
+      SCMSecurityProtocolProtos.SCMGetCertResponseProto response =
+          secureScmClient.getReconCertificateChain(
+              reconDetailsProtoBuilder.build(),
+              getEncodedString(csr));
+      // Persist certificates.
+      if(response.hasX509CACertificate()) {
+        String pemEncodedCert = response.getX509Certificate();
+        certClient.storeCertificate(pemEncodedCert, true);
+        certClient.storeCertificate(response.getX509CACertificate(), true,
+            true);
+
+        // Store Root CA certificate.
+        if (response.hasX509RootCACertificate()) {
+          certClient.storeRootCACertificate(
+              response.getX509RootCACertificate(), true);
+        }
+        String reconCertSerialId = getX509Certificate(pemEncodedCert).
+            getSerialNumber().toString();
+        reconStorage.setReconCertSerialId(reconCertSerialId);
+      } else {
+        throw new RuntimeException("Unable to retrieve recon certificate " +
+            "chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Creates CSR for Recon.
+   * @param config
+   * */
+  private PKCS10CertificationRequest getCSR(OzoneConfiguration config)

Review comment:
       Sure, I'll add this as a utility method.




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] avijayanhwx merged pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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


   


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] aswinshakil commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/interface-client/src/main/proto/hdds.proto
##########
@@ -84,6 +84,12 @@ message ScmNodeDetailsProto {
     required string hostName = 3;      // Hostname of SCM.
 }
 
+message ReconDetailsProto {
+    required string uuid = 1;
+    required string clusterId = 2;
+    required string hostName = 3;
+}

Review comment:
       Will update the PR with the suggested changes.




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] adoroszlai commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java
##########
@@ -151,6 +152,16 @@ public String getDataNodeCertificate(
     return getEncodedCertToString(certSignReq, NodeType.DATANODE);
   }
 
+  @Override
+  public String getReconCertificate(
+      NodeDetailsProto reconDetails,
+      String certSignReq) throws IOException {
+    LOGGER.info("Processing CSR for Recon {}, UUID: {}",
+        reconDetails.getHostName(), reconDetails.getUuid());
+    Objects.requireNonNull(reconDetails);
+    return getEncodedCertToString(certSignReq, NodeType.RECON);

Review comment:
       `NodeType` should be taken from the `NodeDetailsProto` parameter 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.

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] aswinshakil commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
##########
@@ -109,6 +109,7 @@ public ErrorCode getErrorCode() {
     MISSING_BLOCK_TOKEN,
     BLOCK_TOKEN_VERIFICATION_FAILED,
     GET_ROOT_CA_CERT_FAILED,
-    NOT_A_PRIMARY_SCM
+    NOT_A_PRIMARY_SCM,
+    GET_RECON_CERTIFICATE_FAILED

Review comment:
       Shouldn't Recon as a separate component in the environment have its own Certificate Failure code, instead of using a generic one?




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] aswinshakil commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMSecurityProtocolServer.java
##########
@@ -151,6 +152,16 @@ public String getDataNodeCertificate(
     return getEncodedCertToString(certSignReq, NodeType.DATANODE);
   }
 
+  @Override
+  public String getReconCertificate(
+      NodeDetailsProto reconDetails,
+      String certSignReq) throws IOException {
+    LOGGER.info("Processing CSR for Recon {}, UUID: {}",
+        reconDetails.getHostName(), reconDetails.getUuid());
+    Objects.requireNonNull(reconDetails);
+    return getEncodedCertToString(certSignReq, NodeType.RECON);

Review comment:
       Sure, I will resolve this and update it with the comments above.




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] adoroszlai commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
##########
@@ -109,6 +109,7 @@ public ErrorCode getErrorCode() {
     MISSING_BLOCK_TOKEN,
     BLOCK_TOKEN_VERIFICATION_FAILED,
     GET_ROOT_CA_CERT_FAILED,
-    NOT_A_PRIMARY_SCM
+    NOT_A_PRIMARY_SCM,
+    GET_RECON_CERTIFICATE_FAILED

Review comment:
       I don't think we handle any of these component-specific codes differently (they are all part of the same `if` condition).




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] adoroszlai commented on pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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


   @avijayanhwx would you like to take a look?


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] avijayanhwx commented on pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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


   Thank you for working on this @aswinshakil and @adoroszlai for the review.


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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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] adoroszlai commented on a change in pull request #3027: HDDS-6212. SCM Container DB bootstrap on Recon startup for secure cluster

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



##########
File path: hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/security/exception/SCMSecurityException.java
##########
@@ -109,6 +109,7 @@ public ErrorCode getErrorCode() {
     MISSING_BLOCK_TOKEN,
     BLOCK_TOKEN_VERIFICATION_FAILED,
     GET_ROOT_CA_CERT_FAILED,
-    NOT_A_PRIMARY_SCM
+    NOT_A_PRIMARY_SCM,
+    GET_RECON_CERTIFICATE_FAILED

Review comment:
       I think we can reuse `GET_CERTIFICATE_FAILED` instead of adding new error code.

##########
File path: hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/client/ReconCertificateClient.java
##########
@@ -0,0 +1,119 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdds.security.x509.certificate.client;
+
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
+import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.RECOVER;
+import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.SUCCESS;
+
+/**
+ * Certificate client for Recon.
+ */
+public class ReconCertificateClient  extends DefaultCertificateClient{

Review comment:
       This class has lot in common with OMCertificateClient.  Can you please extract a common parent?

##########
File path: hadoop-hdds/interface-client/src/main/proto/hdds.proto
##########
@@ -84,6 +84,12 @@ message ScmNodeDetailsProto {
     required string hostName = 3;      // Hostname of SCM.
 }
 
+message ReconDetailsProto {
+    required string uuid = 1;
+    required string clusterId = 2;
+    required string hostName = 3;
+}

Review comment:
       Can you please make it more generic by including a `NodeType` field (and renaming the message)?  This way it will be easier to add certificate for any other component (e.g. S3 Gateway) in the future.
   
   (Also please rename `SCMGetReconCertRequestProto` and other Recon-specific items.)

##########
File path: hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java
##########
@@ -129,6 +166,108 @@ public Void call() throws Exception {
     return null;
   }
 
+  /**
+   * Initializes secure Recon.
+   * */
+  private void initializeCertificateClient(OzoneConfiguration conf)
+      throws IOException {
+    LOG.info("Initializing secure Recon.");
+    certClient = new ReconCertificateClient(
+        new SecurityConfig(configuration),
+        reconStorage.getReconCertSerialId());
+
+    CertificateClient.InitResponse response = certClient.init();
+    LOG.info("Init response: {}", response);
+    switch (response) {
+    case SUCCESS:
+      LOG.info("Initialization successful, case:{}.", response);
+      break;
+    case GETCERT:
+      getSCMSignedCert(conf);
+      LOG.info("Successfully stored SCM signed certificate, case:{}.",
+          response);
+      break;
+    case FAILURE:
+      LOG.error("Recon security initialization failed, case:{}.", response);
+      throw new RuntimeException("Recon security initialization failed.");
+    case RECOVER:
+      LOG.error("Recon security initialization failed. Recon certificate is " +
+          "missing.");
+      throw new RuntimeException("Recon security initialization failed.");
+    default:
+      LOG.error("Recon security initialization failed. Init response: {}",
+          response);
+      throw new RuntimeException("Recon security initialization failed.");
+    }
+  }
+
+  /**
+   * Get SCM signed certificate and store it using certificate client.
+   * @param config
+   * */
+  private void getSCMSignedCert(OzoneConfiguration config) {
+    try {
+      PKCS10CertificationRequest csr = getCSR(config);
+      SCMSecurityProtocolClientSideTranslatorPB secureScmClient =
+          HddsServerUtil.getScmSecurityClientWithMaxRetry(config);
+      HddsProtos.ReconDetailsProto.Builder reconDetailsProtoBuilder =
+          HddsProtos.ReconDetailsProto.newBuilder()
+              .setHostName(InetAddress.getLocalHost().getHostName())
+              .setClusterId(reconStorage.getClusterID())
+              .setUuid(reconStorage.getReconId());
+
+      SCMSecurityProtocolProtos.SCMGetCertResponseProto response =
+          secureScmClient.getReconCertificateChain(
+              reconDetailsProtoBuilder.build(),
+              getEncodedString(csr));
+      // Persist certificates.
+      if(response.hasX509CACertificate()) {
+        String pemEncodedCert = response.getX509Certificate();
+        certClient.storeCertificate(pemEncodedCert, true);
+        certClient.storeCertificate(response.getX509CACertificate(), true,
+            true);
+
+        // Store Root CA certificate.
+        if (response.hasX509RootCACertificate()) {
+          certClient.storeRootCACertificate(
+              response.getX509RootCACertificate(), true);
+        }
+        String reconCertSerialId = getX509Certificate(pemEncodedCert).
+            getSerialNumber().toString();
+        reconStorage.setReconCertSerialId(reconCertSerialId);
+      } else {
+        throw new RuntimeException("Unable to retrieve recon certificate " +
+            "chain");
+      }
+    } catch (IOException | CertificateException e) {
+      LOG.error("Error while storing SCM signed certificate.", e);
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Creates CSR for Recon.
+   * @param config
+   * */
+  private PKCS10CertificationRequest getCSR(OzoneConfiguration config)

Review comment:
       This method is also very similar to existing code.  Can you please create a utility method building the CSR?




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

To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org

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