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/10/07 03:08:35 UTC

[GitHub] [ozone] neils-dev opened a new pull request #2721: HDDS-5870. Support s3 authentication on a per request basis

neils-dev opened a new pull request #2721:
URL: https://github.com/apache/ozone/pull/2721


   … basis.
   
   ## What changes were proposed in this pull request?
   
   Included in this PR is added functionality to s3 gateway Grpc feature to support s3 request user authentication on a per request basis.  Each s3 request through the s3 gateway is transported over gRPC through a persistent connection between the s3 gateway, s3g, and the Ozone Manager, om.  The s3g client creates an Ozone `S3AUTHINFO` token that is relayed over gRPC, `GrpcOmTransport`, through the `OzoneManagerProtocol` `OmRequest` to the om.  The om authenticates the user (s3 secret lookup) and either processes the request or rejects the request on authentication failure.
   
   Items of interest for request authentication:
   The client side s3g `GrpcOmTransport` sets the` OmRequest` `S3Authentication` structure with the contents of the `S3AUTHINFO` token within the thread context via `UserGroupInformation`.
   
   The server side `OzoneManagerServiceGrpc` uses the `OmRequest` `S3Authentication` to verifiy the s3 request `signature` thorugh a s3 secret lookup based on the `accessID` and generating the signature with the secret key and `stringToSign`.  This is done with the om `OzoneDelegationTokenSecretManager`. 
       
   
   ## What is the link to the Apache JIRA
   https://issues.apache.org/jira/browse/HDDS-5780
   
   ## How was this patch tested?
   Unit tests: TestGrpcOzoneManagerServer; 
   
   Manual Testing with secure ozone cluster:
    s3 request - ozone cluster processing s3 create bucket request
   
   `$ cd hadoop-ozone/dist/target/ozone-1.2.0-SNAPSHOT/compose/ozonesecure`
   **ENABLE** gRPC Ozone Manager through config.
   **DISABLE** acls through config.
   **_ozone.om.s3.grpc.server_enabled: "true"_**
   **_ozone.acl.enabled: "false"_**
   ADD to docker-compose.yaml or docker-config:
   $ vi docker-compose.yaml
   `OZONE-SITE.XML_ozone.om.s3.grpc.server_enabled: "true"`
   `OZONE-SITE.XML_ozone.acl.enabled: "false"`
   `$ docker-compose up -d --scale datanode=3`
   `$ docker-compose run scm bash`
   `bash-4.2$ kinit -kt /etc/security/keytabs/testuser.keytab testuser/scm`
   `bash-4.2$ ozone s3 getsecret`
   `$ export AWS_ACCESS_KEY=testuser/scm@EXAMPLE.COM AWS_SECRET_KEY=<key>`
   `$ aws s3api --endpoint http://localhost:9878 create-bucket --bucket=bucket1`
   _{
       "Location": "http://localhost:9878/bucket1"
   }_
   
   Now run unauthorized:
   `$ export AWS_ACCESS_KEY=` 
   `$ export AWS_SECRET_KEY=`
   `$ aws s3api --endpoint http://localhost:9878 create-bucket --bucket=bucket1`
    _An error occurred (500) when calling the CreateBucket operation (reached max retries: 2): Internal Server Error_


-- 
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] neils-dev commented on a change in pull request #2721: HDDS-5780. Support s3 authentication on a per request basis

Posted by GitBox <gi...@apache.org>.
neils-dev commented on a change in pull request #2721:
URL: https://github.com/apache/ozone/pull/2721#discussion_r725732054



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -60,22 +79,66 @@ public void submitRequest(OMRequest request,
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
     OMResponse omResponse;
+
+    if (secConfig.isSecurityEnabled()) {
+      if (request.hasS3Authentication()) {
+        S3Authentication auth = request.getS3Authentication();
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(auth.getStringToSign());
+        identifier.setSignature(auth.getSignature());
+        identifier.setAwsAccessId(auth.getAccessId());
+        identifier.setOwner(new Text(auth.getAccessId()));
+        try {
+          // authenticate user with signature verification through
+          // delegationTokenMgr validateToken via retrievePassword
+          delegationTokenMgr.retrievePassword(identifier);
+        } catch (Throwable e) {
+          LOG.error("signatures do NOT match for S3 identifier:{}",
+              identifier, e);
+          responseObserver.onNext(
+              createErrorResponse(request,
+                  new OMException("User " + request.getUserInfo()
+                      .getUserName() +
+                      " request authorization failure: " +
+                      "signatures do NOT match",
+                      OMException.ResultCodes.S3_SECRET_NOT_FOUND)));
+          responseObserver.onCompleted();
+          return;
+        }
+      }
+    }
+
+    org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
+        callCount.incrementAndGet(),
+        null,
+        null,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientId.getClientId()));
+    // TODO: currently require setting the Server class for each request
+    // with thread context (Server.Call()) that includes retries
+    // and importantly random ClientId.  This is currently necessary for
+    // Om Ratis Server to create createWriteRaftClientRequest.
+    // Look to remove Server class requirement for issuing ratis transactions
+    // for OMRequests.  Test through successful ratis-enabled OMRequest
+    // handling without dependency on hadoop IPC based Server.
     try {
-      org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
-          callCount.incrementAndGet(),
-          null,
-          null,
-          RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-          ClientId.getClientId()));
-      // TODO: currently require setting the Server class for each request
-      // with thread context (Server.Call()) that includes retries
-      // and importantly random ClientId.  This is currently necessary for
-      // Om Ratis Server to create createWriteRaftClientRequest.
-      // Look to remove Server class requirement for issuing ratis transactions
-      // for OMRequests.  Test through successful ratis-enabled OMRequest 
-      // handling without dependency on hadoop IPC based Server.
-      omResponse = this.omTranslator.
-          submitRequest(NULL_RPC_CONTROLLER, request);
+      omResponse =
+          UserGroupInformation.getCurrentUser().doAs(
+              (PrivilegedExceptionAction<OMResponse>) () -> {
+                try {
+                  return this.omTranslator.
+                      submitRequest(NULL_RPC_CONTROLLER, request);
+                } catch (Throwable se) {
+                  Throwable e = se.getCause();
+                  if (se == null) {

Review comment:
       Thanks, updated exception block handling in recent commit.




-- 
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 #2721: HDDS-5780. Support s3 authentication on a per request basis

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -60,22 +79,66 @@ public void submitRequest(OMRequest request,
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
     OMResponse omResponse;
+
+    if (secConfig.isSecurityEnabled()) {
+      if (request.hasS3Authentication()) {
+        S3Authentication auth = request.getS3Authentication();
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(auth.getStringToSign());
+        identifier.setSignature(auth.getSignature());
+        identifier.setAwsAccessId(auth.getAccessId());
+        identifier.setOwner(new Text(auth.getAccessId()));
+        try {
+          // authenticate user with signature verification through
+          // delegationTokenMgr validateToken via retrievePassword
+          delegationTokenMgr.retrievePassword(identifier);
+        } catch (Throwable e) {
+          LOG.error("signatures do NOT match for S3 identifier:{}",
+              identifier, e);
+          responseObserver.onNext(
+              createErrorResponse(request,
+                  new OMException("User " + request.getUserInfo()
+                      .getUserName() +
+                      " request authorization failure: " +
+                      "signatures do NOT match",
+                      OMException.ResultCodes.S3_SECRET_NOT_FOUND)));
+          responseObserver.onCompleted();
+          return;
+        }
+      }
+    }
+
+    org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
+        callCount.incrementAndGet(),
+        null,
+        null,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientId.getClientId()));
+    // TODO: currently require setting the Server class for each request
+    // with thread context (Server.Call()) that includes retries
+    // and importantly random ClientId.  This is currently necessary for
+    // Om Ratis Server to create createWriteRaftClientRequest.
+    // Look to remove Server class requirement for issuing ratis transactions
+    // for OMRequests.  Test through successful ratis-enabled OMRequest
+    // handling without dependency on hadoop IPC based Server.
     try {
-      org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
-          callCount.incrementAndGet(),
-          null,
-          null,
-          RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-          ClientId.getClientId()));
-      // TODO: currently require setting the Server class for each request
-      // with thread context (Server.Call()) that includes retries
-      // and importantly random ClientId.  This is currently necessary for
-      // Om Ratis Server to create createWriteRaftClientRequest.
-      // Look to remove Server class requirement for issuing ratis transactions
-      // for OMRequests.  Test through successful ratis-enabled OMRequest 
-      // handling without dependency on hadoop IPC based Server.
-      omResponse = this.omTranslator.
-          submitRequest(NULL_RPC_CONTROLLER, request);
+      omResponse =
+          UserGroupInformation.getCurrentUser().doAs(
+              (PrivilegedExceptionAction<OMResponse>) () -> {
+                try {
+                  return this.omTranslator.
+                      submitRequest(NULL_RPC_CONTROLLER, request);
+                } catch (Throwable se) {
+                  Throwable e = se.getCause();
+                  if (se == null) {
+                    throw new IOException(se);
+                  } else {
+                    throw e instanceof IOException ?
+                        (IOException) e : new IOException(se);

Review comment:
       Also typo?
   
   ```suggestion
                           (IOException) e : new IOException(e);
   ```

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -60,22 +79,66 @@ public void submitRequest(OMRequest request,
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
     OMResponse omResponse;
+
+    if (secConfig.isSecurityEnabled()) {
+      if (request.hasS3Authentication()) {
+        S3Authentication auth = request.getS3Authentication();
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(auth.getStringToSign());
+        identifier.setSignature(auth.getSignature());
+        identifier.setAwsAccessId(auth.getAccessId());
+        identifier.setOwner(new Text(auth.getAccessId()));
+        try {
+          // authenticate user with signature verification through
+          // delegationTokenMgr validateToken via retrievePassword
+          delegationTokenMgr.retrievePassword(identifier);
+        } catch (Throwable e) {
+          LOG.error("signatures do NOT match for S3 identifier:{}",
+              identifier, e);
+          responseObserver.onNext(
+              createErrorResponse(request,
+                  new OMException("User " + request.getUserInfo()
+                      .getUserName() +
+                      " request authorization failure: " +
+                      "signatures do NOT match",
+                      OMException.ResultCodes.S3_SECRET_NOT_FOUND)));
+          responseObserver.onCompleted();
+          return;
+        }
+      }
+    }
+
+    org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
+        callCount.incrementAndGet(),
+        null,
+        null,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientId.getClientId()));
+    // TODO: currently require setting the Server class for each request
+    // with thread context (Server.Call()) that includes retries
+    // and importantly random ClientId.  This is currently necessary for
+    // Om Ratis Server to create createWriteRaftClientRequest.
+    // Look to remove Server class requirement for issuing ratis transactions
+    // for OMRequests.  Test through successful ratis-enabled OMRequest
+    // handling without dependency on hadoop IPC based Server.
     try {
-      org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
-          callCount.incrementAndGet(),
-          null,
-          null,
-          RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-          ClientId.getClientId()));
-      // TODO: currently require setting the Server class for each request
-      // with thread context (Server.Call()) that includes retries
-      // and importantly random ClientId.  This is currently necessary for
-      // Om Ratis Server to create createWriteRaftClientRequest.
-      // Look to remove Server class requirement for issuing ratis transactions
-      // for OMRequests.  Test through successful ratis-enabled OMRequest 
-      // handling without dependency on hadoop IPC based Server.
-      omResponse = this.omTranslator.
-          submitRequest(NULL_RPC_CONTROLLER, request);
+      omResponse =
+          UserGroupInformation.getCurrentUser().doAs(
+              (PrivilegedExceptionAction<OMResponse>) () -> {
+                try {
+                  return this.omTranslator.
+                      submitRequest(NULL_RPC_CONTROLLER, request);
+                } catch (Throwable se) {
+                  Throwable e = se.getCause();
+                  if (se == null) {

Review comment:
       `se` cannot be `null` here.  Intended `e == null`?




-- 
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 #2721: HDDS-5780. Support s3 authentication on a per request basis

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -60,22 +79,66 @@ public void submitRequest(OMRequest request,
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
     OMResponse omResponse;
+
+    if (secConfig.isSecurityEnabled()) {
+      if (request.hasS3Authentication()) {
+        S3Authentication auth = request.getS3Authentication();
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(auth.getStringToSign());
+        identifier.setSignature(auth.getSignature());
+        identifier.setAwsAccessId(auth.getAccessId());
+        identifier.setOwner(new Text(auth.getAccessId()));
+        try {
+          // authenticate user with signature verification through
+          // delegationTokenMgr validateToken via retrievePassword
+          delegationTokenMgr.retrievePassword(identifier);
+        } catch (Throwable e) {
+          LOG.error("signatures do NOT match for S3 identifier:{}",
+              identifier, e);
+          responseObserver.onNext(
+              createErrorResponse(request,
+                  new OMException("User " + request.getUserInfo()
+                      .getUserName() +
+                      " request authorization failure: " +
+                      "signatures do NOT match",
+                      OMException.ResultCodes.S3_SECRET_NOT_FOUND)));
+          responseObserver.onCompleted();
+          return;
+        }
+      }
+    }
+
+    org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
+        callCount.incrementAndGet(),
+        null,
+        null,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientId.getClientId()));
+    // TODO: currently require setting the Server class for each request
+    // with thread context (Server.Call()) that includes retries
+    // and importantly random ClientId.  This is currently necessary for
+    // Om Ratis Server to create createWriteRaftClientRequest.
+    // Look to remove Server class requirement for issuing ratis transactions
+    // for OMRequests.  Test through successful ratis-enabled OMRequest
+    // handling without dependency on hadoop IPC based Server.
     try {
-      org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
-          callCount.incrementAndGet(),
-          null,
-          null,
-          RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-          ClientId.getClientId()));
-      // TODO: currently require setting the Server class for each request
-      // with thread context (Server.Call()) that includes retries
-      // and importantly random ClientId.  This is currently necessary for
-      // Om Ratis Server to create createWriteRaftClientRequest.
-      // Look to remove Server class requirement for issuing ratis transactions
-      // for OMRequests.  Test through successful ratis-enabled OMRequest 
-      // handling without dependency on hadoop IPC based Server.
-      omResponse = this.omTranslator.
-          submitRequest(NULL_RPC_CONTROLLER, request);
+      omResponse =
+          UserGroupInformation.getCurrentUser().doAs(
+              (PrivilegedExceptionAction<OMResponse>) () -> {
+                try {
+                  return this.omTranslator.
+                      submitRequest(NULL_RPC_CONTROLLER, request);
+                } catch (Throwable se) {
+                  Throwable e = se.getCause();
+                  if (se == null) {
+                    throw new IOException(se);
+                  } else {
+                    throw e instanceof IOException ?
+                        (IOException) e : new IOException(se);

Review comment:
       OK, I found that this logic comes from `ProtobufHelper`.




-- 
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 merged pull request #2721: HDDS-5780. Support s3 authentication on a per request basis

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


   


-- 
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] neils-dev commented on a change in pull request #2721: HDDS-5780. Support s3 authentication on a per request basis

Posted by GitBox <gi...@apache.org>.
neils-dev commented on a change in pull request #2721:
URL: https://github.com/apache/ozone/pull/2721#discussion_r725731281



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerServiceGrpc.java
##########
@@ -60,22 +79,66 @@ public void submitRequest(OMRequest request,
         request.getCmdType().name());
     AtomicInteger callCount = new AtomicInteger(0);
     OMResponse omResponse;
+
+    if (secConfig.isSecurityEnabled()) {
+      if (request.hasS3Authentication()) {
+        S3Authentication auth = request.getS3Authentication();
+        OzoneTokenIdentifier identifier = new OzoneTokenIdentifier();
+        identifier.setTokenType(S3AUTHINFO);
+        identifier.setStrToSign(auth.getStringToSign());
+        identifier.setSignature(auth.getSignature());
+        identifier.setAwsAccessId(auth.getAccessId());
+        identifier.setOwner(new Text(auth.getAccessId()));
+        try {
+          // authenticate user with signature verification through
+          // delegationTokenMgr validateToken via retrievePassword
+          delegationTokenMgr.retrievePassword(identifier);
+        } catch (Throwable e) {
+          LOG.error("signatures do NOT match for S3 identifier:{}",
+              identifier, e);
+          responseObserver.onNext(
+              createErrorResponse(request,
+                  new OMException("User " + request.getUserInfo()
+                      .getUserName() +
+                      " request authorization failure: " +
+                      "signatures do NOT match",
+                      OMException.ResultCodes.S3_SECRET_NOT_FOUND)));
+          responseObserver.onCompleted();
+          return;
+        }
+      }
+    }
+
+    org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
+        callCount.incrementAndGet(),
+        null,
+        null,
+        RPC.RpcKind.RPC_PROTOCOL_BUFFER,
+        ClientId.getClientId()));
+    // TODO: currently require setting the Server class for each request
+    // with thread context (Server.Call()) that includes retries
+    // and importantly random ClientId.  This is currently necessary for
+    // Om Ratis Server to create createWriteRaftClientRequest.
+    // Look to remove Server class requirement for issuing ratis transactions
+    // for OMRequests.  Test through successful ratis-enabled OMRequest
+    // handling without dependency on hadoop IPC based Server.
     try {
-      org.apache.hadoop.ipc.Server.getCurCall().set(new Server.Call(1,
-          callCount.incrementAndGet(),
-          null,
-          null,
-          RPC.RpcKind.RPC_PROTOCOL_BUFFER,
-          ClientId.getClientId()));
-      // TODO: currently require setting the Server class for each request
-      // with thread context (Server.Call()) that includes retries
-      // and importantly random ClientId.  This is currently necessary for
-      // Om Ratis Server to create createWriteRaftClientRequest.
-      // Look to remove Server class requirement for issuing ratis transactions
-      // for OMRequests.  Test through successful ratis-enabled OMRequest 
-      // handling without dependency on hadoop IPC based Server.
-      omResponse = this.omTranslator.
-          submitRequest(NULL_RPC_CONTROLLER, request);
+      omResponse =
+          UserGroupInformation.getCurrentUser().doAs(
+              (PrivilegedExceptionAction<OMResponse>) () -> {
+                try {
+                  return this.omTranslator.
+                      submitRequest(NULL_RPC_CONTROLLER, request);
+                } catch (Throwable se) {
+                  Throwable e = se.getCause();
+                  if (se == null) {
+                    throw new IOException(se);
+                  } else {
+                    throw e instanceof IOException ?
+                        (IOException) e : new IOException(se);

Review comment:
       Thanks @adoroszlai.  I'm updating the exception handling block to check if `se.getCause() `is `null` ( `if e == null` ) like you suggested.
   Here though in this block checking if `e` (se.getCause() ) is `instanceof IOException` keep it **as is** with if check _is_ true, throwing e (IOException) and if e _is not_ (IOException) then throwing an new IOException constructed with the throwable (se).  Let me know your thoughts on keeping it as is.




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