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/25 22:18:11 UTC

[GitHub] [ozone] smengcl opened a new pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

smengcl opened a new pull request #3018:
URL: https://github.com/apache/ozone/pull/3018


   https://issues.apache.org/jira/browse/HDDS-6084
   
   ```
   To allow upgrades and downgrades (from pre-finalized state) after multi-tenancy is introduced, the following items must be handled:
   ```
   
   - [x] Add OM layout feature for multi-tenancy
   - [x] Block requests to new tenancy APIs while pre-finalized.
   - [ ] Integration and acceptance tests to verify old S3 users are not affected.
     - [ ] Integration test addition in `TestMultiTenantVolume`.


-- 
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] smengcl merged pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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


   


-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
##########
@@ -111,4 +117,8 @@ public void beforeRequestApplyTxn(final JoinPoint joinPoint)
         om.getVersionManager(), lf.name());
   }
 
+  public static OMLayoutFeatureAspect aspectOf() {

Review comment:
       Is this method called anywhere?

##########
File path: hadoop-ozone/ozone-manager/src/main/resources/META-INF/aop.xml
##########
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureAspect"/>
+    <weaver options="-verbose -showWeaveInfo">
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler"/>
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB"/>

Review comment:
       Why is this class needed here?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +82,52 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
   @Test
   public void testS3TenantVolume() throws Exception {
+
     final String tenant = "tenant";
     final String principal = "username";
     final String bucketName = "bucket";
     final String accessID = UUID.randomUUID().toString();
 
     ObjectStore store = getStoreForAccessID(accessID);
+
+    // None of the tenant APIs is usable before the upgrade finalization step

Review comment:
       nit. Can we move the block from here to finalization to a helper method?

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
##########
@@ -29,8 +29,9 @@
  */
 public enum OMLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version");
-
+  INITIAL_VERSION(0, "Initial Layout Version"),
+  // TODO: Have a better name / description? Create table action on upgrade?
+  MULTITENANCY_SCHEMA(1, "Multi-tenancy Schema");

Review comment:
       I don't think we need an upgrade action, since the s3 secrets are staying in the same table. I think we can just call this feature `MULTITENANCY`.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
##########
@@ -54,11 +54,17 @@ public void checkLayoutFeature(JoinPoint joinPoint) throws IOException {
     String featureName = ((MethodSignature) joinPoint.getSignature())
         .getMethod().getAnnotation(DisallowedUntilLayoutVersion.class)
         .value().name();
-    LayoutVersionManager lvm = null;
+    LayoutVersionManager lvm;
+    final Object[] args = joinPoint.getArgs();
     if (joinPoint.getTarget() instanceof OzoneManagerRequestHandler) {
       OzoneManager ozoneManager = ((OzoneManagerRequestHandler)
           joinPoint.getTarget()).getOzoneManager();
       lvm = ozoneManager.getVersionManager();
+    } else if (args != null && args.length > 0 &&
+        args[0] instanceof OzoneManager) {
+      // Get OzoneManager instance from preExecute first arg

Review comment:
       Maybe check `joinPoint.getTarget() instanceof OMClientRequest && joinPoint.toShortString().equals(preExecute)`, either instead of or in addition to the argument checks? I think that is the only other place we would expect to put this annotation besides the request handler so that might help enforce and document it better.

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +82,52 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
   @Test
   public void testS3TenantVolume() throws Exception {
+
     final String tenant = "tenant";
     final String principal = "username";
     final String bucketName = "bucket";
     final String accessID = UUID.randomUUID().toString();
 
     ObjectStore store = getStoreForAccessID(accessID);
+
+    // None of the tenant APIs is usable before the upgrade finalization step
+    expectFailurePreFinalization(store::listTenant);
+    expectFailurePreFinalization(() -> store.listUsersInTenant(tenant, ""));
+    expectFailurePreFinalization(() -> store.tenantGetUserInfo(principal));
+    expectFailurePreFinalization(() -> store.createTenant(tenant));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignUserAccessId(principal, tenant, accessID));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignAdmin(principal, tenant, true));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeAdmin(accessID, tenant));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeUserAccessId(accessID));
+    expectFailurePreFinalization(() -> store.deleteTenant(tenant));
+
+    // S3 get/set/revoke secret APIs still work before finalization
+    final String accessId = "testUser1accessId1";
+    S3SecretValue s3SecretValue = store.getS3Secret(accessId);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    final String setSecret = "testsecret";
+    s3SecretValue = store.setS3Secret(accessId, setSecret);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    Assert.assertEquals(setSecret, s3SecretValue.getAwsSecret());
+    store.revokeS3Secret(accessId);
+
+
+    // Trigger OM finalization
+    cluster.getOzoneManager().finalizeUpgrade("clientId1");

Review comment:
       This should be submitted through the client so it goes through Ratis and updates the on disk state as well. Finalization is async so the test should wait on queryUpgradeFinalizationProgress to return `Status#ALREADY_FINALIZED`.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.

Review comment:
       Good point. I will use LICENSE.txt for now. Once we merge master we could use the new addition: https://github.com/apache/ozone/pull/3050/files#diff-472262337660b3a36c07cf07985181b7b4991652e06e5f9732ab770f79d66dc8R26-R28




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.

Review comment:
       Whelp. LICENSE.txt isn't guaranteed to work either: https://github.com/apache/ozone/commits/master/LICENSE.txt
   
   I'll just write a string to compare.




-- 
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] smengcl commented on pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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


   Triggered CI: https://github.com/smengcl/hadoop-ozone/commits/HDDS-6084-CI-2


-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
##########
@@ -111,4 +117,8 @@ public void beforeRequestApplyTxn(final JoinPoint joinPoint)
         om.getVersionManager(), lf.name());
   }
 
+  public static OMLayoutFeatureAspect aspectOf() {

Review comment:
       It is implicitly called by aspectj, if I leave this out it can throw `NoSuchMethodError`, but not always.




-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +89,81 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
+  /**
+   * Perform sanity checks before triggering upgrade finalization.
+   */
+  private void preFinalizationCheck(ObjectStore store, String tenant,

Review comment:
       I missed this the first time looking through, but since we are sharing a cluster to save time, we should make sure the upgrade happens before either of the two tests, because we want `testDefaultS3Volume` to run on the finalized version as well. Maybe we can make the upgrade steps their own test, and define an order for the tests. Since we are using junit4, I think this would have to be done with `@FixMethodOrder(MethodSorters.NAME_ASCENDING)`, and adding a prefix to the test names to indicate their order.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +82,52 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
   @Test
   public void testS3TenantVolume() throws Exception {
+
     final String tenant = "tenant";
     final String principal = "username";
     final String bucketName = "bucket";
     final String accessID = UUID.randomUUID().toString();
 
     ObjectStore store = getStoreForAccessID(accessID);
+
+    // None of the tenant APIs is usable before the upgrade finalization step
+    expectFailurePreFinalization(store::listTenant);
+    expectFailurePreFinalization(() -> store.listUsersInTenant(tenant, ""));
+    expectFailurePreFinalization(() -> store.tenantGetUserInfo(principal));
+    expectFailurePreFinalization(() -> store.createTenant(tenant));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignUserAccessId(principal, tenant, accessID));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignAdmin(principal, tenant, true));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeAdmin(accessID, tenant));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeUserAccessId(accessID));
+    expectFailurePreFinalization(() -> store.deleteTenant(tenant));
+
+    // S3 get/set/revoke secret APIs still work before finalization
+    final String accessId = "testUser1accessId1";
+    S3SecretValue s3SecretValue = store.getS3Secret(accessId);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    final String setSecret = "testsecret";
+    s3SecretValue = store.setS3Secret(accessId, setSecret);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    Assert.assertEquals(setSecret, s3SecretValue.getAwsSecret());
+    store.revokeS3Secret(accessId);
+
+
+    // Trigger OM finalization
+    cluster.getOzoneManager().finalizeUpgrade("clientId1");

Review comment:
       done.




-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +89,81 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
+  /**
+   * Perform sanity checks before triggering upgrade finalization.
+   */
+  private void preFinalizationCheck(ObjectStore store, String tenant,

Review comment:
       Or we could run the upgrade steps as part of the cluster setup, and if they fail it will show as a setup failure.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.
+#                        Compare files    /opt/hadoop/NOTICE.txt    /tmp/key1.result
+                        Execute and checkrc    rm /tmp/key1.result    0
+
+Create bucket using S3 API if the bucket doesn't exist

Review comment:
       Agreed. Done!




-- 
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] smengcl commented on pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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


   After 9 (!) retriggers on my CI-2 branch I finally got a green run.
   
   Thanks @avijayanhwx for the +1. I will merge this shortly.


-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/resources/META-INF/aop.xml
##########
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureAspect"/>
+    <weaver options="-verbose -showWeaveInfo">
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler"/>
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB"/>

Review comment:
       Why is OzoneManagerProtocolServerSideTranslatorPB needed 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] smengcl commented on pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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


   Thanks @errose28 and @avijayanhwx for the review and +1's.


-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -43,22 +55,108 @@
   private static MiniOzoneCluster cluster;
   private static String s3VolumeName;
 
+  private static final String TENANT_NAME = "tenant";
+  private static final String USER_PRINCIPAL = "username";
+  private static final String BUCKET_NAME = "bucket";
+  private static final String ACCESS_ID = UUID.randomUUID().toString();
+
   @BeforeClass
   public static void initClusterProvider() throws Exception {
     OzoneConfiguration conf = new OzoneConfiguration();
     conf.setBoolean(
         OMMultiTenantManagerImpl.OZONE_OM_TENANT_DEV_SKIP_RANGER, true);
     MiniOzoneCluster.Builder builder = MiniOzoneCluster.newBuilder(conf)
-        .withoutDatanodes();
+        .withoutDatanodes()
+        .setOmLayoutVersion(OMLayoutFeature.INITIAL_VERSION.layoutVersion());
     cluster = builder.build();
     s3VolumeName = HddsClientUtils.getDefaultS3VolumeName(conf);
+
+    preFinalizationChecks(getStoreForAccessID(ACCESS_ID));
+    finalizeOMUpgrade();
   }
 
   @AfterClass
   public static void shutdownClusterProvider() {
     cluster.shutdown();
   }
 
+  private static void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
+  /**
+   * Perform sanity checks before triggering upgrade finalization.
+   */
+  private static void preFinalizationChecks(ObjectStore store)
+      throws Exception {
+
+    // None of the tenant APIs is usable before the upgrade finalization step
+    expectFailurePreFinalization(
+        store::listTenant);
+    expectFailurePreFinalization(() ->
+        store.listUsersInTenant(TENANT_NAME, ""));
+    expectFailurePreFinalization(() ->
+        store.tenantGetUserInfo(USER_PRINCIPAL));
+    expectFailurePreFinalization(() ->
+        store.createTenant(TENANT_NAME));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignUserAccessId(USER_PRINCIPAL, TENANT_NAME, ACCESS_ID));
+    expectFailurePreFinalization(() ->
+        store.tenantAssignAdmin(USER_PRINCIPAL, TENANT_NAME, true));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeAdmin(ACCESS_ID, TENANT_NAME));
+    expectFailurePreFinalization(() ->
+        store.tenantRevokeUserAccessId(ACCESS_ID));
+    expectFailurePreFinalization(() ->
+        store.deleteTenant(TENANT_NAME));
+
+    // S3 get/set/revoke secret APIs still work before finalization
+    final String accessId = "testUser1accessId1";
+    S3SecretValue s3SecretValue = store.getS3Secret(accessId);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    final String setSecret = "testsecret";
+    s3SecretValue = store.setS3Secret(accessId, setSecret);
+    Assert.assertEquals(accessId, s3SecretValue.getAwsAccessKey());
+    Assert.assertEquals(setSecret, s3SecretValue.getAwsSecret());
+    store.revokeS3Secret(accessId);
+  }
+
+  /**
+   * Trigger OM upgrade finalization from the client and block until completion
+   * (status FINALIZATION_DONE).
+   */
+  private static void finalizeOMUpgrade()
+      throws IOException, InterruptedException, TimeoutException {
+
+    // Trigger OM upgrade finalization. Ref: FinalizeUpgradeSubCommand#call
+    final OzoneManagerProtocol client = cluster.getRpcClient().getObjectStore()
+        .getClientProxy().getOzoneManagerClient();
+    final String upgradeClientID = "Test-Upgrade-Client-" + UUID.randomUUID();
+    UpgradeFinalizer.StatusAndMessages finalizationResponse =
+        client.finalizeUpgrade(upgradeClientID);
+
+    // Not sure if the status transitions as soon as client call returns.
+    // Can remove if this causes issue.
+    Assert.assertTrue(isStarting(finalizationResponse.status()));

Review comment:
       isStarting is the expected return value here, so it shouldn't hurt to leave it in. If something else is returned it would be an error since the cluster should not already be finalized. The finalization required status should only returned if the query upgrade progress command was given here instead of a finalize command.

##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.

Review comment:
       How about the license file? Should be more stable. We could update the generate/validate for normal keys to use this as well.

##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.
+#                        Compare files    /opt/hadoop/NOTICE.txt    /tmp/key1.result
+                        Execute and checkrc    rm /tmp/key1.result    0
+
+Create bucket using S3 API if the bucket doesn't exist

Review comment:
       Can we move the aws client writes to generate.robot and leave the aws client reads in validate.robot? It seems a little confusing now having validate.robot do generate and validate for S3.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/dist/src/main/smoketest/upgrade/validate.robot
##########
@@ -28,3 +29,25 @@ Read data from previously created key
     ${random} =         Generate Random String  5  [NUMBERS]
     ${output} =         Execute          ozone sh key get /${PREFIX}-volume/${PREFIX}-bucket/${PREFIX}-key /tmp/key-${random}
                         Should not contain  ${output}       Failed
+
+Setup credentials for S3
+# TODO: Run "Setup secure v4 headers" instead when security is enabled
+    Run Keyword         Setup dummy credentials for S3
+
+Read data from previously created key using S3 API
+    ${result} =         Execute AWSS3APICli and checkrc    get-object --bucket ${PREFIX}-bucket --key key1 /tmp/key1.result    0
+    # Note: "Compare files" doesn't work on NOTICE.txt as it is updated in new Ozone versions.

Review comment:
       Whelp. LICENSE.txt isn't guaranteed to work either: https://github.com/apache/ozone/commits/master/LICENSE.txt
   
   I'll just writing a string to compare.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
##########
@@ -29,8 +29,9 @@
  */
 public enum OMLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version");
-
+  INITIAL_VERSION(0, "Initial Layout Version"),
+  // TODO: Have a better name / description? Create table action on upgrade?
+  MULTITENANCY_SCHEMA(1, "Multi-tenancy Schema");

Review comment:
       Thanks for confirming this.
   
   Does this mean the new multi-tenancy tables that didn't exist in older OM would be created automatically upon OM upgrade (or during table writes) ?




-- 
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] smengcl commented on pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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


   Note: the `Acceptance (MR)` test failure is resolved by HDDS-6239. That fix will be backported during the next merge of master branch into this feature branch.
   
   I have triggered a CI in my fork on a branch that has HDDS-6239 applied:
   https://github.com/smengcl/hadoop-ozone/commits/HDDS-6084-CI


-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/resources/META-INF/aop.xml
##########
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureAspect"/>
+    <weaver options="-verbose -showWeaveInfo">
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler"/>
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB"/>

Review comment:
       `OzoneManagerProtocolServerSideTranslatorPB` is added here as it calls the `preExecute`s with the annotation.
   
   Without this line the `DisallowedUntilLayoutVersion` annotation on e.g. `OMTenantCreateRequest#preExecute` becomes ineffective.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/resources/META-INF/aop.xml
##########
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureAspect"/>
+    <weaver options="-verbose -showWeaveInfo">
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler"/>
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB"/>

Review comment:
       `OzoneManagerProtocolServerSideTranslatorPB` is added here as it calls the `preExecute`s with the annotation.




-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
##########
@@ -111,4 +117,8 @@ public void beforeRequestApplyTxn(final JoinPoint joinPoint)
         om.getVersionManager(), lf.name());
   }
 
+  public static OMLayoutFeatureAspect aspectOf() {

Review comment:
       Ok, interesting. Maybe we can add a comment explaining that so it doesn't look like dead code.

##########
File path: hadoop-ozone/ozone-manager/src/main/resources/META-INF/aop.xml
##########
@@ -0,0 +1,23 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  Licensed 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. See accompanying LICENSE file.
+-->
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureAspect"/>
+    <weaver options="-verbose -showWeaveInfo">
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerRequestHandler"/>
+      <include within="org.apache.hadoop.ozone.protocolPB.OzoneManagerProtocolServerSideTranslatorPB"/>

Review comment:
       Got it, thanks




-- 
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] errose28 commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
##########
@@ -29,8 +29,9 @@
  */
 public enum OMLayoutFeature implements LayoutFeature {
   //////////////////////////////  //////////////////////////////
-  INITIAL_VERSION(0, "Initial Layout Version");
-
+  INITIAL_VERSION(0, "Initial Layout Version"),
+  // TODO: Have a better name / description? Create table action on upgrade?
+  MULTITENANCY_SCHEMA(1, "Multi-tenancy Schema");

Review comment:
       The new tables will actually be created in pre-finalize, I believe when the DB is opened. However, the old Ozone version will disregard tables it does not know about. So as long as no information is put in these tables during pre-finalize, it is not an incompatible change. That is why the upgrade/downgrade acceptance tests pass on our branch.




-- 
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] smengcl commented on a change in pull request #3018: HDDS-6084. [Multi-Tenant] Handle upgrades to version supporting S3 multi-tenancy

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/multitenant/TestMultiTenantVolume.java
##########
@@ -77,14 +89,81 @@ public void testDefaultS3Volume() throws Exception {
     assertS3BucketNotFound(store, bucketName);
   }
 
+  private void expectFailurePreFinalization(VoidCallable eval)
+      throws Exception {
+    LambdaTestUtils.intercept(OMException.class,
+        "cannot be invoked before finalization", eval);
+  }
+
+  /**
+   * Perform sanity checks before triggering upgrade finalization.
+   */
+  private void preFinalizationCheck(ObjectStore store, String tenant,

Review comment:
       Good point.
   
   I like the idea of moving the pre-finalization checks to the cluster setup stage. Will do.




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