You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2020/09/09 13:52:51 UTC

[GitHub] [hadoop-ozone] captainzmc opened a new pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

captainzmc opened a new pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412


   ## What changes were proposed in this pull request?
   
   By HDDS-3725 Ozone currently supports Set volume quota. This PR refers to the implementation of HDDS-3725, and make  Ozone shell support set bucket quota.
   The current Quota setting does not take effect. HDDS-541 gives all the work needed to perfect Quota.
   This PR is a subtask of HDDS-541. 
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-3751
   
   ## How was this patch tested?
   
   UT added
   


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

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



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


[GitHub] [hadoop-ozone] amaliujia commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r486775970



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492518254



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
##########
@@ -150,6 +148,20 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
             .setIsVersionEnabled(dbBucketInfo.getIsVersionEnabled());
       }
 
+      //Check quotaInBytes and quotaInCounts to update
+      String volumeKey = omMetadataManager.getVolumeKey(volumeName);
+      OmVolumeArgs omVolumeArgs = omMetadataManager.getVolumeTable()
+          .get(volumeKey);
+      if (checkQuotaBytesValid(omVolumeArgs, omBucketArgs)) {

Review comment:
       same as 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.

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



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


[GitHub] [hadoop-ozone] adoroszlai commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -46,6 +47,14 @@
           "false/unspecified indicates otherwise")
   private Boolean isGdprEnforced;
 
+  @Option(names = {"--spaceQuota", "-sq"},

Review comment:
       Short options should be only one character to support option grouping (ie. `-sq` should be two separate options, same as `-s -q`).
   
   Long options should not be camel-case, rather lower-case using dash as separator.
   
   ```suggestion
     @Option(names = {"--space-quota", "-s"},
   ```

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",
+    description = "Updates parameter of the buckets")
+public class UpdateBucketHandler extends BucketHandler {
+
+  @Option(names = {"--spaceQuota", "-sq"},
+      description = "Quota in bytes of the newly created volume (eg. 1GB)")

Review comment:
       Please consider creating a [mixin](https://picocli.info/#_mixins) with the two quota options to ensure consistency and reduce duplication.  See [`ListOptions`](https://github.com/apache/hadoop-ozone/blob/079ee7fc2a223e1251b16b9c42004aa2a27bf0f4/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ListOptions.java) for example, and its usages in 
   
   https://github.com/apache/hadoop-ozone/blob/079ee7fc2a223e1251b16b9c42004aa2a27bf0f4/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/volume/ListVolumeHandler.java#L51-L52
   
   and
   
   https://github.com/apache/hadoop-ozone/blob/079ee7fc2a223e1251b16b9c42004aa2a27bf0f4/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/ListBucketHandler.java#L42-L43
   
   Option descriptions can be generic, without mentioning "newly created volume" etc., so they can be applied to create|update volume|bucket.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -192,6 +192,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         throw new OMException("Bucket already exist", BUCKET_ALREADY_EXISTS);
       }
 
+      //Check quotaInBytes and quotaInCounts to update
+      checkQuotaBytesValid(omVolumeArgs, omBucketInfo);
+      checkQuotaCountsValid(omVolumeArgs, omBucketInfo);

Review comment:
       Argument validity should be checked before acquiring lock, preferably in `preExecute`.
   
   Also, please verify that the bucket is not a link, if quota is set in the request.  Links cannot have actual content, so they should not have any quota defined, similar to encryption:
   
   https://github.com/apache/hadoop-ozone/blob/079ee7fc2a223e1251b16b9c42004aa2a27bf0f4/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java#L127-L130

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -46,6 +47,14 @@
           "false/unspecified indicates otherwise")
   private Boolean isGdprEnforced;
 
+  @Option(names = {"--spaceQuota", "-sq"},
+      description = "Quota in bytes of the newly created bucket (eg. 1GB)")
+  private String quotaInBytes;
+
+  @Option(names = {"--quota", "-q"},

Review comment:
       I would suggest dropping the short option.
   
   ```suggestion
     @Option(names = {"--key-quota"},
   ```

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",
+    description = "Updates parameter of the buckets")
+public class UpdateBucketHandler extends BucketHandler {
+
+  @Option(names = {"--spaceQuota", "-sq"},
+      description = "Quota in bytes of the newly created volume (eg. 1GB)")
+  private String quotaInBytes;
+
+  @Option(names = {"--quota", "-q"},
+      description = "Key counts of the newly created bucket (eg. 5)")
+  private long quotaInCounts = OzoneConsts.QUOTA_RESET;
+
+  /**
+   * Executes create bucket.

Review comment:
       nit: leftover doc

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.

Review comment:
       nit: leftover doc




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-695220186


   Thanks for @ChenSammi @adoroszlai and @maobaolong 's review. I had fixed review issues. Could you help take another 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.

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



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


[GitHub] [hadoop-ozone] cxorm commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
cxorm commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-698094498


   Thanks @captainzmc for the work.
    
   Would you please rebase this PR ?
   I would take a look on it.


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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492461120



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -66,17 +69,23 @@
    * @param bucketEncryptionKey bucket encryption key name
    * @param sourceVolume
    * @param sourceBucket
+   * @param quotaInBytes Volume quota in bytes.

Review comment:
       Volume? 




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492461120



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -66,17 +69,23 @@
    * @param bucketEncryptionKey bucket encryption key name
    * @param sourceVolume
    * @param sourceBucket
+   * @param quotaInBytes Volume quota in bytes.

Review comment:
       parameter statement is incorrent.




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495775842



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -496,6 +491,28 @@ private static void verifyBucketName(String bucketName) throws OMException {
     }
   }
 
+  private static void verifyCountsQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {

Review comment:
       double ((




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-701139655


   Thanks @captainzmc for the contribution and @cxorm @adoroszlai @maobaolong 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.

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492517940



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume. We also need this check when update volume quato. 




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-698132943


    Hi all, status updates: rebase PR and resolve conflicts. This PR can be reviewed again.


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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-697133030


   Thanks @ChenSammi for the review. Review issues has been fixed.


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

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



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


[GitHub] [hadoop-ozone] captainzmc removed a comment on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc removed a comment on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-697133030


   Thanks @ChenSammi for the review. Review issues has been fixed.


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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-698132943


    Hi all, status updates: rebase PR and resolve conflicts. This PR can be reviewed again.


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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490231852



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -57,6 +57,9 @@
   private final String sourceVolume;
   private final String sourceBucket;
 
+  private long quotaInBytes;

Review comment:
       I will change the String in VolumeArgs to long




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r488534476



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Had added a verify method In RpcClient.




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

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



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


[GitHub] [hadoop-ozone] ChenSammi merged pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi merged pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412


   


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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490192669



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",

Review comment:
       update -> setquota




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490257305



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -439,6 +434,15 @@ public void createBucket(
     verifyVolumeName(volumeName);
     verifyBucketName(bucketName);
     Preconditions.checkNotNull(bucketArgs);
+    verifyCountsQuota(bucketArgs.getQuotaInCounts());
+    verifySpaceQuota(bucketArgs.getQuotaInBytes());
+
+    // When creating buckets using the API, if the user does not specify quota,
+    // 0 is passed in by default, which should be set to -1.

Review comment:
       In proto unsigned field can't have negative default value. The logic here is the same and I can encapsulate it as a method, which should be better




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492515786



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/volume/SetQuotaHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.ozone.shell.volume;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * Executes update volume calls.
+ */
+@Command(name = "setquota",
+    description = "Set quota of the volumes")
+public class SetQuotaHandler extends VolumeHandler {
+
+  @CommandLine.Mixin
+  private SetSpaceQuotaOptions quotaOptions;
+
+  @Option(names = {"--bucket-quota"},
+      description = "Bucket counts of the volume to set (eg. 5)")

Review comment:
       set -> create




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

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



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


[GitHub] [hadoop-ozone] amaliujia commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r486775970



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495775842



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -496,6 +491,28 @@ private static void verifyBucketName(String bucketName) throws OMException {
     }
   }
 
+  private static void verifyCountsQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {

Review comment:
       single ( is enough.




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492517940



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume. Better cover this case in UT.
   We also need this check when update volume quato. 




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

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



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


[GitHub] [hadoop-ozone] amaliujia commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
amaliujia commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r486775970



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Out of curiosity: what is the purpose of `@SuppressWarnings("parameternumber")`?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       Do you need to verify whether `quotaInBytes` and `quotaInCounts` are valid? e.g. >= 0?




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492511397



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ClearSpaceQuotaOptions.java
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.shell;
+
+import picocli.CommandLine;
+
+/**
+ * Common options for 'clrquota' comands.

Review comment:
       typo  comands 




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r488511539



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -174,6 +184,20 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.modificationTime = Instant.ofEpochMilli(modificationTime);
   }
 
+  @SuppressWarnings("parameternumber")

Review comment:
       Thanks for @amaliujia‘s review.
   Ozone's CheckStyle requires that the number of method variables must not exceed a certain value, or the check fails.  [So it borrows from other usage.](https://github.com/apache/hadoop-ozone/blob/d34ab29b622168c57bd2d3049d060c42735de7f4/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java#L207)




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-700512727


   LGTM,  +1. 
   
   Hi  @adoroszlai,I'm not sure if you are satified with the change,  would you like to take another look at this patch?  


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

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



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


[GitHub] [hadoop-ozone] maobaolong commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
maobaolong commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490169874



##########
File path: hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/TestOMRequestUtils.java
##########
@@ -283,7 +285,8 @@ public static void addVolumeToDB(String volumeName, String ownerName,
     OmVolumeArgs omVolumeArgs =
         OmVolumeArgs.newBuilder().setCreationTime(Time.now())
             .setVolume(volumeName).setAdminName(ownerName)
-            .setOwnerName(ownerName).build();
+            .setOwnerName(ownerName).setQuotaInBytes(1024 * GB)

Review comment:
       If you want to keep this `QuotaInBytes` and  `QuotaInCounts ` big enough, you can set it to Long.MAX_VALUE;

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
##########
@@ -269,6 +269,68 @@ public void testVolumeSetOwner() throws IOException {
     proxy.setVolumeOwner(volumeName, ownerName);
   }
 
+  @Test
+  public void testSetBucketQuota() throws IOException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    store.createVolume(volumeName);
+    store.getVolume(volumeName).setQuota(OzoneQuota.parseQuota(
+        "10GB", 10000L));
+    store.getVolume(volumeName).createBucket(bucketName);
+    OzoneBucket bucket = store.getVolume(volumeName).getBucket(bucketName);
+
+    Assert.assertEquals(OzoneConsts.QUOTA_RESET, bucket.getQuotaInBytes());
+    Assert.assertEquals(OzoneConsts.QUOTA_RESET, bucket.getQuotaInCounts());
+    store.getVolume(volumeName).getBucket(bucketName).setQuota(
+        OzoneQuota.parseQuota("1GB", 1000L));
+    OzoneBucket ozoneBucket = store.getVolume(volumeName).getBucket(bucketName);
+    Assert.assertEquals(1024 * 1024 * 1024,
+        ozoneBucket.getQuotaInBytes());
+    Assert.assertEquals(1000L, ozoneBucket.getQuotaInCounts());
+  }
+
+  @Test
+  public void testSetBucketQuotaIllegal() throws IOException {
+    String volumeName = UUID.randomUUID().toString();
+    String bucketName = UUID.randomUUID().toString();
+    store.createVolume(volumeName);
+    store.getVolume(volumeName).createBucket(bucketName);
+
+    try {
+      store.getVolume(volumeName).getBucket(bucketName).setQuota(
+          OzoneQuota.parseQuota("1GB", -100L));
+    } catch (IllegalArgumentException ex) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid values for quota", ex);
+    }
+    // The unit should be legal.
+    try {
+      store.getVolume(volumeName).getBucket(bucketName).setQuota(
+          OzoneQuota.parseQuota("1TEST", 100L));
+    } catch (IllegalArgumentException ex) {
+      GenericTestUtils.assertExceptionContains(
+          "Invalid values for quota", ex);
+    }
+
+    // The setting value cannot be greater than LONG.MAX_VALUE BYTES.
+    try {
+      store.getVolume(volumeName).getBucket(bucketName).setQuota(
+          OzoneQuota.parseQuota("9999999999999GB", 100L));

Review comment:
       You can use "9223372036854775808" here, it stand for the Long.MAX_VALUE + 1




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

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



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


[GitHub] [hadoop-ozone] captainzmc closed pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc closed pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412


   


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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492515197



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/volume/SetQuotaHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.ozone.shell.volume;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * Executes update volume calls.

Review comment:
       statement is stale. 




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495519238



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -314,6 +314,8 @@ enum Status {
 
     PARTIAL_RENAME = 65;
 
+    BUCKET_SPACE_QUOTA_NOT_RESET = 66;

Review comment:
       not used anymore.  

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
##########
@@ -229,7 +229,9 @@ public String toString() {
 
     NOT_SUPPORTED_OPERATION,
 
-    PARTIAL_RENAME
+    PARTIAL_RENAME,
+
+    BUCKET_SPACE_QUOTA_NOT_RESET

Review comment:
       same as other place. 




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490192669



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",

Review comment:
       update -> setquota,  could you please also change the volume quota update command to setquota? 
   
   We also need a remove quota CLI for both volume and bucket.  




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492461120



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -66,17 +69,23 @@
    * @param bucketEncryptionKey bucket encryption key name
    * @param sourceVolume
    * @param sourceBucket
+   * @param quotaInBytes Volume quota in bytes.

Review comment:
       Volume? 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -66,17 +69,23 @@
    * @param bucketEncryptionKey bucket encryption key name
    * @param sourceVolume
    * @param sourceBucket
+   * @param quotaInBytes Volume quota in bytes.

Review comment:
       parameter statement is incorrent.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -282,15 +285,48 @@ public boolean setOwner(String userName) throws IOException {
     return result;
   }
 
+  /**
+   * Clean the space quota of the volume.
+   *
+   * @throws IOException
+   */
+  public void clearSpaceQuota() throws IOException {
+    OzoneVolume ozoneVolume = proxy.getVolumeDetails(name);
+    Iterator bucketIter = ozoneVolume.listBuckets(null);
+    while (bucketIter.hasNext()) {
+      OzoneBucket nextBucket = (OzoneBucket) bucketIter.next();
+      if(nextBucket.getQuotaInBytes() != QUOTA_RESET) {

Review comment:
       Bucket has quota while Volume doesn't,this is a common case.  

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
##########
@@ -655,4 +655,15 @@ OzoneOutputStream createFile(String volumeName, String bucketName,
    * Getter for OzoneManagerClient.
    */
   OzoneManagerProtocol getOzoneManagerClient();
+
+  /**
+   * Set Bucket Quota.
+   * @param volumeName Name of the Volume.
+   * @param bucketName Name of the Bucket.
+   * @param quotaInBytes The maximum size this volume can be used.

Review comment:
       @param order doesn't match the real parmater order. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/ClearSpaceQuotaOptions.java
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.shell;
+
+import picocli.CommandLine;
+
+/**
+ * Common options for 'clrquota' comands.

Review comment:
       typo  comands 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/volume/SetQuotaHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.ozone.shell.volume;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * Executes update volume calls.

Review comment:
       statement is stale. 

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/volume/SetQuotaHandler.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.ozone.shell.volume;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * Executes update volume calls.
+ */
+@Command(name = "setquota",
+    description = "Set quota of the volumes")
+public class SetQuotaHandler extends VolumeHandler {
+
+  @CommandLine.Mixin
+  private SetSpaceQuotaOptions quotaOptions;
+
+  @Option(names = {"--bucket-quota"},
+      description = "Bucket counts of the volume to set (eg. 5)")

Review comment:
       set -> create

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume.

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketSetPropertyRequest.java
##########
@@ -150,6 +148,20 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
             .setIsVersionEnabled(dbBucketInfo.getIsVersionEnabled());
       }
 
+      //Check quotaInBytes and quotaInCounts to update
+      String volumeKey = omMetadataManager.getVolumeKey(volumeName);
+      OmVolumeArgs omVolumeArgs = omMetadataManager.getVolumeTable()
+          .get(volumeKey);
+      if (checkQuotaBytesValid(omVolumeArgs, omBucketArgs)) {

Review comment:
       same as above. 

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume. We also need this check when update volume quato. 

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume. Better cover this case in UT.
   We also need this check when update volume quato. 




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492500415



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneVolume.java
##########
@@ -282,15 +285,48 @@ public boolean setOwner(String userName) throws IOException {
     return result;
   }
 
+  /**
+   * Clean the space quota of the volume.
+   *
+   * @throws IOException
+   */
+  public void clearSpaceQuota() throws IOException {
+    OzoneVolume ozoneVolume = proxy.getVolumeDetails(name);
+    Iterator bucketIter = ozoneVolume.listBuckets(null);
+    while (bucketIter.hasNext()) {
+      OzoneBucket nextBucket = (OzoneBucket) bucketIter.next();
+      if(nextBucket.getQuotaInBytes() != QUOTA_RESET) {

Review comment:
       Bucket has quota while Volume doesn't,this is a common case.  




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-699594512


   Thanks @ChenSammi for the review. Review issues has been fixed.


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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r496451487



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -192,6 +192,10 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
         throw new OMException("Bucket already exist", BUCKET_ALREADY_EXISTS);
       }
 
+      //Check quotaInBytes and quotaInCounts to update
+      checkQuotaBytesValid(omVolumeArgs, omBucketInfo);
+      checkQuotaCountsValid(omVolumeArgs, omBucketInfo);

Review comment:
       Thanks @adoroszlai for the advice, the check action needs to get the volume or bucket in DB, so it better to do this action after acquiring the lock.  And other comments has been fixed.




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495777131



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -334,15 +329,11 @@ public boolean setVolumeOwner(String volumeName, String owner)
   }
 
   @Override
-  public void setVolumeQuota(String volumeName, long quotaInCounts,
-      long quotaInBytes) throws IOException {
+  public void setVolumeQuota(String volumeName, long quotaInBytes,
+      long quotaInCounts) throws IOException {

Review comment:
       Can we align the  quotaInBytes and quotaInCounts order in setVolumeQuota and setBucketQuota?  In another words, no need for the code change here. 




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492505555



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
##########
@@ -655,4 +655,15 @@ OzoneOutputStream createFile(String volumeName, String bucketName,
    * Getter for OzoneManagerClient.
    */
   OzoneManagerProtocol getOzoneManagerClient();
+
+  /**
+   * Set Bucket Quota.
+   * @param volumeName Name of the Volume.
+   * @param bucketName Name of the Bucket.
+   * @param quotaInBytes The maximum size this volume can be used.

Review comment:
       @param order doesn't match the real parmater 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.

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495775842



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -496,6 +491,28 @@ private static void verifyBucketName(String bucketName) throws OMException {
     }
   }
 
+  private static void verifyCountsQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {

Review comment:
       single (( is enough.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -496,6 +491,28 @@ private static void verifyBucketName(String bucketName) throws OMException {
     }
   }
 
+  private static void verifyCountsQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {
+      throw new IllegalArgumentException("Invalid values for quota : " +
+          "counts quota is :" + quota + ".");
+    }
+  }
+
+  private static void verifySpaceQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {

Review comment:
       same as 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.

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r496662068



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -441,6 +431,8 @@ public void createBucket(
     verifyVolumeName(volumeName);
     verifyBucketName(bucketName);
     Preconditions.checkNotNull(bucketArgs);
+    verifyCountsQuota(bucketArgs.getQuotaInCounts());
+    verifySpaceQuota(bucketArgs.getQuotaInBytes());

Review comment:
       The PR of createVolume did not add a checkmark before, I will add this in the createVolume.




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r492517940



##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +300,17 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+      OmBucketInfo omBucketInfo) {
+    long volumeQuotaInBytes = omVolumeArgs.getQuotaInBytes();
+    long quotaInBytes = omBucketInfo.getQuotaInBytes();
+    if(volumeQuotaInBytes < quotaInBytes) {

Review comment:
       Need check sum of all bucket quota under the Volume.




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r488534476



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -464,6 +469,8 @@ public void createBucket(
         .setStorageType(storageType)
         .setSourceVolume(bucketArgs.getSourceVolume())
         .setSourceBucket(bucketArgs.getSourceBucket())
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts)

Review comment:
       I had added a verify method In new 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.

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-690427621


   Hi @ChenSammi @cxorm, This PR is a bucket quota shell based on #1233 volume quota shell. Could you help review it?


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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490192669



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",

Review comment:
       update -> setQuota




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490270995



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -598,6 +628,20 @@ public void setBucketStorageType(
     ozoneManagerClient.setBucketProperty(builder.build());
   }
 
+  @Override
+  public void setBucketQuota(String volumeName, String bucketName,
+      long quotaInCounts, long quotaInBytes) throws IOException {
+    HddsClientUtils.verifyResourceName(bucketName);
+    verifyQuota(quotaInCounts, quotaInBytes);
+    OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts);
+    ozoneManagerClient.setBucketProperty(builder.build());

Review comment:
       Bucket provides the setBucketProperty method and the corresponding request to modify the bucket information. This method already exists and is not something I added later. So here I'm reusing the 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.

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r495797897



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/SetQuotaHandler.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.SetSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * set quota of the bucket.
+ */
+@Command(name = "setquota",
+    description = "Set quota of the buckets")
+public class SetQuotaHandler extends BucketHandler {
+
+  @CommandLine.Mixin
+  private SetSpaceQuotaOptions quotaOptions;
+
+  @Option(names = {"--key-quota"},
+      description = "Key counts of the newly created bucket (eg. 5)")
+  private long quotaInCounts = OzoneConsts.QUOTA_RESET;
+
+  /**
+   * Executes create bucket.

Review comment:
       leftover statement




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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490192669



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",

Review comment:
       update -> setquota,  could you please also change the volume quota update command to setquota




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

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



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


[GitHub] [hadoop-ozone] cxorm commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
cxorm commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-698094498


   Thanks @captainzmc for the work.
    
   Would you please rebase this PR ?
   I would take a look on it.


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

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



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


[GitHub] [hadoop-ozone] adoroszlai commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -441,6 +431,8 @@ public void createBucket(
     verifyVolumeName(volumeName);
     verifyBucketName(bucketName);
     Preconditions.checkNotNull(bucketArgs);
+    verifyCountsQuota(bucketArgs.getQuotaInCounts());
+    verifySpaceQuota(bucketArgs.getQuotaInBytes());

Review comment:
       `createBucket` verifies quota args, but `createVolume` does not.  Is this intentional?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -305,8 +300,8 @@ public void createVolume(String volumeName, VolumeArgs volArgs)
     builder.setVolume(volumeName);
     builder.setAdminName(admin);
     builder.setOwnerName(owner);
-    builder.setQuotaInBytes(quotaInBytes);
-    builder.setQuotaInCounts(quotaInCounts);
+    builder.setQuotaInBytes(getQuotaValue(volArgs.getQuotaInBytes()));
+    builder.setQuotaInCounts(getQuotaValue(volArgs.getQuotaInCounts()));

Review comment:
       Arguments are already checked, why change from use of the variables to `getQuotaValue`?
   
   ```suggestion
       builder.setQuotaInBytes(quotaInBytes);
       builder.setQuotaInCounts(quotaInCounts);
   ```

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/ClearQuotaHandler.java
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import org.apache.hadoop.ozone.shell.ClearSpaceQuotaOptions;
+import picocli.CommandLine;
+import picocli.CommandLine.Command;
+
+import java.io.IOException;
+
+/**
+ * clean quota of the bucket.
+ */
+@Command(name = "clrquota",
+    description = "clear quota of the bucket")
+public class ClearQuotaHandler extends BucketHandler {
+
+  @CommandLine.Mixin
+  private ClearSpaceQuotaOptions clrSpaceQuota;
+
+  @CommandLine.Option(names = {"--key-quota"},
+      description = "clear count quota")
+  private boolean clrKeyQuota;

Review comment:
       By using a bit more generic option name `--count-quota`, this could be moved into `ClearSpaceQuotaOptions` (and unified with volume's `--bucket-quota` option).

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -46,6 +49,13 @@
           "false/unspecified indicates otherwise")
   private Boolean isGdprEnforced;
 
+  @CommandLine.Mixin
+  private SetSpaceQuotaOptions quotaOptions;
+
+  @Option(names = {"--key-quota"},
+      description = "Key counts of the newly created bucket (eg. 5)")
+  private long quotaInCounts = OzoneConsts.QUOTA_RESET;

Review comment:
       Similarly to the option for `ClearQuotaHandler`:
   
   by using a bit more generic option name `--count-quota` (and description), this could be moved into `SetSpaceQuotaOptions` (and unified with volume's `--bucket-quota` option).




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

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



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


[GitHub] [hadoop-ozone] captainzmc commented on pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#issuecomment-700751283


   Thanks @adoroszlai’s review, I have already fixed the review issues, can you  help take another look? [CI is normal in my personal branch.](https://github.com/captainzmc/hadoop-ozone/runs/1182140007)


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

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



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


[GitHub] [hadoop-ozone] ChenSammi commented on a change in pull request #1412: HDDS-3751. Ozone sh client support bucket quota option.

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #1412:
URL: https://github.com/apache/hadoop-ozone/pull/1412#discussion_r490174497



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -494,6 +500,30 @@ private static void verifyBucketName(String bucketName) throws OMException {
     }
   }
 
+  private static void verifyCountsQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {
+      throw new IllegalArgumentException("Invalid values for quota : " +
+          "counts quota is :" + quota + ".");
+    }
+  }
+
+  private static void verifySpaceQuota(long quota) throws OMException {
+    if ((quota < OzoneConsts.QUOTA_RESET)) {
+      throw new IllegalArgumentException("Invalid values for quota : " +
+          "space quota is :" + quota + ".");
+    }
+  }
+
+  private static void verifyQuota(long quotaInCounts, long quotaInBytes)

Review comment:
       This verifyQuota has the same function as above two. Suggest choose one and remove the other. 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -439,6 +434,15 @@ public void createBucket(
     verifyVolumeName(volumeName);
     verifyBucketName(bucketName);
     Preconditions.checkNotNull(bucketArgs);
+    verifyCountsQuota(bucketArgs.getQuotaInCounts());
+    verifySpaceQuota(bucketArgs.getQuotaInBytes());
+
+    // When creating buckets using the API, if the user does not specify quota,
+    // 0 is passed in by default, which should be set to -1.

Review comment:
       Can we add default -1 value to newly added fields in proto file? 

##########
File path: hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
##########
@@ -297,4 +301,28 @@ private BucketEncryptionInfoProto getBeinfo(
             CipherSuite.convert(metadata.getCipher())));
     return bekb.build();
   }
+
+  public void checkQuotaBytesValid(OmVolumeArgs omVolumeArgs,
+                                      OmBucketInfo omBucketInfo) {

Review comment:
       indent

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -57,6 +57,9 @@
   private final String sourceVolume;
   private final String sourceBucket;
 
+  private long quotaInBytes;

Review comment:
       quotaInBytes in VolumeArgs has String as type. Can we unify them? 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -598,6 +628,20 @@ public void setBucketStorageType(
     ozoneManagerClient.setBucketProperty(builder.build());
   }
 
+  @Override
+  public void setBucketQuota(String volumeName, String bucketName,
+      long quotaInCounts, long quotaInBytes) throws IOException {
+    HddsClientUtils.verifyResourceName(bucketName);

Review comment:
       verify volume 

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -598,6 +628,20 @@ public void setBucketStorageType(
     ozoneManagerClient.setBucketProperty(builder.build());
   }
 
+  @Override
+  public void setBucketQuota(String volumeName, String bucketName,
+      long quotaInCounts, long quotaInBytes) throws IOException {
+    HddsClientUtils.verifyResourceName(bucketName);
+    verifyQuota(quotaInCounts, quotaInBytes);
+    OmBucketArgs.Builder builder = OmBucketArgs.newBuilder();
+    builder.setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setQuotaInBytes(quotaInBytes)
+        .setQuotaInCounts(quotaInCounts);
+    ozoneManagerClient.setBucketProperty(builder.build());

Review comment:
       setBucketQuota?  setBucketProperty?

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -46,6 +47,14 @@
           "false/unspecified indicates otherwise")
   private Boolean isGdprEnforced;
 
+  @Option(names = {"--spaceQuota", "-sq"},
+      description = "Quota in bytes of the newly created bucket (eg. 1GB)")
+  private String quotaInBytes;
+
+  @Option(names = {"--quota", "-q"},

Review comment:
       --quota -> --keyQuota   -q -> -kq

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/UpdateBucketHandler.java
##########
@@ -0,0 +1,70 @@
+/*
+ * 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.ozone.shell.bucket;
+
+import org.apache.hadoop.hdds.client.OzoneQuota;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.client.OzoneClient;
+import org.apache.hadoop.ozone.shell.OzoneAddress;
+import picocli.CommandLine.Command;
+import picocli.CommandLine.Option;
+
+import java.io.IOException;
+
+/**
+ * create bucket handler.
+ */
+@Command(name = "update",
+    description = "Updates parameter of the buckets")
+public class UpdateBucketHandler extends BucketHandler {
+
+  @Option(names = {"--spaceQuota", "-sq"},
+      description = "Quota in bytes of the newly created volume (eg. 1GB)")
+  private String quotaInBytes;
+
+  @Option(names = {"--quota", "-q"},

Review comment:
       same as 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.

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



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