You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/06/22 21:31:16 UTC

[GitHub] [ozone] aryangupta1998 opened a new pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

aryangupta1998 opened a new pull request #2357:
URL: https://github.com/apache/ozone/pull/2357


   ## What changes were proposed in this pull request?
   
   We are introducing 3 types of buckets as of now in Ozone.
   
   1. FSO (File System Optimized):- For this bucket, we need the configuration “ozone.om.enable.filesystem.paths” to be set to true.
      Behavior:
   
      - Supports atomic rename and delete operations.
      - Files written to this bucket can be read via S3.
      - Keys written via S3 with a delimiter “/” will create intermediate directories.
      - This bucket will allow interoperability between S3 and FS APIs.
   2. OBS (Object Store):- For this bucket, we need the configuration “ozone.om.enable.filesystem.paths” to be set to false.
      Behavior:
   
      - Keys written to this bucket will behave as a pure object-store.
      - Keys written via S3 with a delimiter “/” will not create intermediate dirs.
      - This bucket will not allow interoperability between S3 and FS APIs.
   3. LEGACY:- This bucket type has been introduced to handle the case of upgrading an existing cluster with pre-created buckets. Users can’t create a LEGACY bucket explicitly via the Ozone client.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5362
   
   ## How was this patch tested?
   
   Tested Manually
   


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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Thanks a lot @marton for the comments.
   
   > I am against removing the Map<String,String> as it can be used not only for storing bucket types but storing any other metadata. Also, it's an incompatible change if we remove it.
   
   To make it clear about the`Map<String,String>` entity - this patch will not remove `KeyValue` attribute from the BucketInfo and it will remain the same. Any upcoming feature or behaviour can still make use of this `KeyValue` attribute and build it.
   
   _OmClientProtocol.proto_
   ```
       message BucketInfo {
       .....
       repeated hadoop.hdds.KeyValue metadata = 7;
       .....
   + optional BucketLayoutProto bucketLayout = 18;
   ```
   
   This patch is moving the bucket layout information stored in the **Map to an enum attribute.**
   
   > Also, it's an incompatible change if we remove it.
   
   IIUC, since the FSO(prefix) feature is not released in any of the Ozone version we have still the opportunity to refactor/redesign the proto part. Please let me know if I missed anything.
   
   Following are the points while evaluating `enum` over `Map<String, String>`.
   
   - enum has options to limit the set of valid values and will validate the input efficiently. Please refer, https://picocli.info/#_enum_types
   - protobuf defaults to the enum constant at ordinal 0 and not worrying about null checks in a programatic way(easily avoid NPE cases). Not only from the commandline, while reading back the old data from DB, where the bucket layout will not exists in DB.
   - according to me enum makes code more maintainable and easy to read
   
   I'm completely open for a better approach. With map, we have to do all the above in a programatic way and its doable. Please let me know your feedback. Thanks 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.

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -201,6 +206,21 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.quotaInNamespace = quotaInNamespace;
   }
 
+  @SuppressWarnings("parameternumber")
+  public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
+      String volumeName, String bucketName, StorageType storageType,
+      Boolean versioning, long creationTime, long modificationTime,
+      Map<String, String> metadata, String encryptionKeyName,
+      String sourceVolume, String sourceBucket, long usedBytes,
+      long usedNamespace, long quotaInBytes, long quotaInNamespace,
+      BucketType bucketType) {
+    this(conf, proxy, volumeName, bucketName, storageType, versioning,
+        creationTime, modificationTime, metadata, encryptionKeyName,
+        sourceVolume, sourceBucket, usedBytes, usedNamespace, quotaInBytes,
+        quotaInNamespace);
+    this.bucketType = bucketType;

Review comment:
       I'll initialize bucket type to avoid null value.
   `private BucketType bucketType = BucketType.DEFAULT;`




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

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



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -60,9 +67,18 @@
   public void execute(OzoneClient client, OzoneAddress address)
       throws IOException {
 
-    BucketArgs.Builder bb = new BucketArgs.Builder()
-        .setStorageType(StorageType.DEFAULT)
-        .setVersioning(false);
+    BucketArgs.Builder bb;
+    if (bucketType.equals(BucketType.OBJECT_STORE) || bucketType

Review comment:
       I think this check is not needed, we should specify an enum in the bucket args parameter




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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.

Review comment:
       Javadoc is wrong, please modify it.

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
##########
@@ -102,6 +107,7 @@
    * @param usedBytes - Bucket Quota Usage in bytes.
    * @param quotaInBytes Bucket quota in bytes.
    * @param quotaInNamespace Bucket quota in counts.
+   * @param bucketType Bucket Types.

Review comment:
       typo: `Bucket Types.` -> `Bucket Type.`

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -622,6 +621,40 @@ public void testRenameToAnExistingKey() throws Exception {
     }
   }
 
+  @Test
+  public void testCreateBucketWithBucketType() throws Exception {
+    String sampleVolumeName = UUID.randomUUID().toString();
+    String sampleBucketName = UUID.randomUUID().toString();
+    OzoneClient client = cluster.getClient();
+    ObjectStore store = client.getObjectStore();
+    store.createVolume(sampleVolumeName);
+    OzoneVolume volume = store.getVolume(sampleVolumeName);
+
+    // Case 1: Bucket Type: FSO
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setBucketType(BucketType.FSO);
+    volume.createBucket(sampleBucketName, builder.build());
+    OzoneBucket bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.FSO, bucket.getBucketType());
+
+    // Case 2: Bucket Type: OBJECT_STORE
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.OBJECT_STORE);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.OBJECT_STORE, bucket.getBucketType());
+
+    // Case 3: Bucket Type: Empty
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.DEFAULT);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.DEFAULT, bucket.getBucketType());

Review comment:
       Please point me to a test case where user passes an argument with LEGACY type. It shouldn't allow to create a LEGACY bucket and throws exception. If there is no test then please add it a 5th case like,
   
   // case 5: Bucket Type: LEGACY
       builder.setBucketType(BucketType.LEGACY);
       volume.createBucket(sampleBucketName, builder.build());

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -622,6 +621,40 @@ public void testRenameToAnExistingKey() throws Exception {
     }
   }
 
+  @Test
+  public void testCreateBucketWithBucketType() throws Exception {
+    String sampleVolumeName = UUID.randomUUID().toString();
+    String sampleBucketName = UUID.randomUUID().toString();
+    OzoneClient client = cluster.getClient();
+    ObjectStore store = client.getObjectStore();
+    store.createVolume(sampleVolumeName);
+    OzoneVolume volume = store.getVolume(sampleVolumeName);
+
+    // Case 1: Bucket Type: FSO
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setBucketType(BucketType.FSO);
+    volume.createBucket(sampleBucketName, builder.build());
+    OzoneBucket bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.FSO, bucket.getBucketType());
+
+    // Case 2: Bucket Type: OBJECT_STORE
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.OBJECT_STORE);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.OBJECT_STORE, bucket.getBucketType());
+
+    // Case 3: Bucket Type: Empty

Review comment:
       Here, it says `Empty` but it is passing `DEFAULT` in the builder `builder.setBucketType(BucketType.DEFAULT);`.
   Please remove `builder.setBucketType(BucketType.DEFAULT); `for verifying `Empty` case and you can add one more case for DEFAULT.
   
   `// Case 4: Bucket Type: DEFAULT`

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -60,9 +67,18 @@
   public void execute(OzoneClient client, OzoneAddress address)
       throws IOException {
 
-    BucketArgs.Builder bb = new BucketArgs.Builder()
-        .setStorageType(StorageType.DEFAULT)
-        .setVersioning(false);
+    BucketArgs.Builder bb;
+    if (bucketType.equals(BucketType.OBJECT_STORE) || bucketType
+        .equals(BucketType.FSO)) {
+      bb = new BucketArgs.Builder().setStorageType(StorageType.DEFAULT)
+          .setVersioning(false).setBucketType(bucketType);
+    } else {
+      throw new IllegalArgumentException(

Review comment:
       Hope we have test validating this exception. If not then please add it. 

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:
+      return OzoneManagerProtocolProtos.BucketTypeProto.LEGACY;
+    default:
+      throw new IllegalStateException(

Review comment:
       As this is not state, can we`throw new IllegalArgumentException("Error: BucketType not found, type=" + this)`?




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

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



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


[GitHub] [ozone] rakeshadr merged pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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


   


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

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -98,27 +100,24 @@
    *
    * @throws IOException
    */
-  @BeforeClass
-  public static void init() throws Exception {
+  @BeforeClass public static void init() throws Exception {
     conf = new OzoneConfiguration();
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
     omId = UUID.randomUUID().toString();
-    TestOMRequestUtils.configureFSOptimizedPaths(conf,
-            true, OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-            .setClusterId(clusterId)
-            .setScmId(scmId)
-            .setOmId(omId)
-            .build();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,

Review comment:
       Now we have the OM Metadata layout config and we need this config to start OM in FSO mode. I'll do the changes along with HDDS-5385.




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -60,9 +67,18 @@
   public void execute(OzoneClient client, OzoneAddress address)
       throws IOException {
 
-    BucketArgs.Builder bb = new BucketArgs.Builder()
-        .setStorageType(StorageType.DEFAULT)
-        .setVersioning(false);
+    BucketArgs.Builder bb;
+    if (bucketType.equals(BucketType.OBJECT_STORE) || bucketType
+        .equals(BucketType.FSO)) {
+      bb = new BucketArgs.Builder().setStorageType(StorageType.DEFAULT)
+          .setVersioning(false).setBucketType(bucketType);
+    } else {
+      throw new IllegalArgumentException(

Review comment:
       We have introduced an enum to check the required bucket types, now we don't need this exception hence, removed 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.

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

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



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Thanks for the reply @elek, the proposal is not to remove the Map<String,String>. That will remain as it is.




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

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
##########
@@ -378,6 +396,7 @@ public Builder() {
       this.metadata = new HashMap<>();
       this.quotaInBytes = OzoneConsts.QUOTA_RESET;
       this.quotaInNamespace = OzoneConsts.QUOTA_RESET;
+      this.bucketType = BucketType.DEFAULT;

Review comment:
       New users will only be able to create FILE_SYSTEM_OPTIMIZED and OBJECT_STORE buckets. Only existing buckets can be of type LEGACY. In BucketType.java we have mentioned that if the bucket type is null then the bucket type will be initialized to LEGACY. Also for unit tests, the buckets which don't have bucket type will be initialized to LEGACY, then we will check if the layout is of type prefix then we will change the bucket type to FILE_SYSTEM_OPTIMIZED. In the follow up jiras we will remove the layout 'PREFIX' concept. I hope this answers your question!




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

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

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



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


[GitHub] [ozone] bharatviswa504 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:

Review comment:
       Should not be LEGACY be categorized in to 2 types.
   LEGACY_FSO and LEGACY_OBJECT_STORE based on ozone.om.enable.filesystem.paths config




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -622,6 +621,40 @@ public void testRenameToAnExistingKey() throws Exception {
     }
   }
 
+  @Test
+  public void testCreateBucketWithBucketType() throws Exception {
+    String sampleVolumeName = UUID.randomUUID().toString();
+    String sampleBucketName = UUID.randomUUID().toString();
+    OzoneClient client = cluster.getClient();
+    ObjectStore store = client.getObjectStore();
+    store.createVolume(sampleVolumeName);
+    OzoneVolume volume = store.getVolume(sampleVolumeName);
+
+    // Case 1: Bucket Type: FSO
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setBucketType(BucketType.FSO);
+    volume.createBucket(sampleBucketName, builder.build());
+    OzoneBucket bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.FSO, bucket.getBucketType());
+
+    // Case 2: Bucket Type: OBJECT_STORE
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.OBJECT_STORE);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.OBJECT_STORE, bucket.getBucketType());
+
+    // Case 3: Bucket Type: Empty

Review comment:
       Removed `builder.setBucketType(BucketType.DEFAULT);` and added `// Case 4: Bucket Type: DEFAULT`




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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Thanks a lot @marton for the comments.
   
   > I am against removing the Map<String,String> as it can be used not only for storing bucket types but storing any other metadata. Also, it's an incompatible change if we remove it.
   
   To make it clear about the`Map<String,String>` entity - this patch will not remove `KeyValue` attribute from the BucketInfo and it will remain the same. Any upcoming feature or behaviour can still make use of this `KeyValue` attribute and build it.
   
   _OmClientProtocol.proto_
   ```
       message BucketInfo {
       .....
       repeated hadoop.hdds.KeyValue metadata = 7;
       .....
   + optional BucketLayoutProto bucketLayout = 18;
   ```
   
   This patch is moving the bucket layout information stored in the **Map to an enum attribute.**
   
   > Also, it's an incompatible change if we remove it.
   
   IIUC, since the FSO(prefix) feature is not released in any of the Ozone version we have still the opportunity to refactor/redesign the proto part. Please let me know if I missed anything.
   
   Following are the points considered while evaluating `enum` over `Map<String, String>`.
   
   - enum has options to limit the set of valid values and will validate the input efficiently. Please refer, https://picocli.info/#_enum_types
   - protobuf defaults to the enum constant at ordinal 0 and not worrying about null checks in a programatic way(easily avoid NPE cases). Not only from the commandline, while reading back the old data from DB, where the bucket layout will not exists in DB.
   - according to me enum makes code more maintainable and easy to read
   
   I'm completely open for a better approach. With map, we have to do all the above in a programatic way and its doable. Please let me know your feedback. Thanks 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.

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:

Review comment:
       If the flag is 'false' technically it will be OBS(object store) and I'll be mapping these buckets to OBS instead of LEGACY.
   If the flag is 'true' it will be a normalized path with intermediate directories and I'll be mapping these buckets to LEGACY. Now, with my above proposal, there will be only one LEGACY that represents an un-optimized filesystem.
   Does this make sense to you?
    




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:
+      return OzoneManagerProtocolProtos.BucketTypeProto.LEGACY;
+    default:
+      throw new IllegalStateException(

Review comment:
       Done




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

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

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



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


[GitHub] [ozone] rakeshadr commented on pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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


   Good Work. Thanks a lot @aryangupta1998 for the continuous efforts in refining the patch.
   
   Thank you @xiaoyuyao @mukul1987 @elek @bharatviswa504 for the useful comments and discussions.
   
   I've merged it to master branch.


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

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

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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
##########
@@ -378,6 +396,7 @@ public Builder() {
       this.metadata = new HashMap<>();
       this.quotaInBytes = OzoneConsts.QUOTA_RESET;
       this.quotaInNamespace = OzoneConsts.QUOTA_RESET;
+      this.bucketType = BucketType.DEFAULT;

Review comment:
       Can you elaborate on the BucketType.DEFAULT -> OBJECT_STORE? Should we chose LEGACY for backward compatibility? 




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -622,6 +621,40 @@ public void testRenameToAnExistingKey() throws Exception {
     }
   }
 
+  @Test
+  public void testCreateBucketWithBucketType() throws Exception {
+    String sampleVolumeName = UUID.randomUUID().toString();
+    String sampleBucketName = UUID.randomUUID().toString();
+    OzoneClient client = cluster.getClient();
+    ObjectStore store = client.getObjectStore();
+    store.createVolume(sampleVolumeName);
+    OzoneVolume volume = store.getVolume(sampleVolumeName);
+
+    // Case 1: Bucket Type: FSO
+    BucketArgs.Builder builder = BucketArgs.newBuilder();
+    builder.setBucketType(BucketType.FSO);
+    volume.createBucket(sampleBucketName, builder.build());
+    OzoneBucket bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.FSO, bucket.getBucketType());
+
+    // Case 2: Bucket Type: OBJECT_STORE
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.OBJECT_STORE);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.OBJECT_STORE, bucket.getBucketType());
+
+    // Case 3: Bucket Type: Empty
+    sampleBucketName = UUID.randomUUID().toString();
+    builder.setBucketType(BucketType.DEFAULT);
+    volume.createBucket(sampleBucketName, builder.build());
+    bucket = volume.getBucket(sampleBucketName);
+    Assert.assertEquals(sampleBucketName, bucket.getName());
+    Assert.assertEquals(BucketType.DEFAULT, bucket.getBucketType());

Review comment:
       Added `// case 5: Bucket Type: LEGACY`




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:

Review comment:
       Thanks, @bharatviswa504 for the comments, I'm thinking below changes:
   If the flag is 'false' technically it will be OBS(object store) and I'll be mapping these buckets to OBS instead of LEGACY.
   If the flag is 'true' it will be a normalized path with intermediate directories and I'll be mapping these buckets to LEGACY_FILESYSTEM. Now, with my above proposal, there will be only one LEGACY_FILESYSTEM that represents an un-optimized filesystem.
   Does this make sense to you?
    




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

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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -563,7 +583,7 @@ public long getUsedNamespace() {
   public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
       String prevKey) throws IOException {
 
-    if(OzoneFSUtils.isFSOptimizedBucket(getMetadata())){
+    if(OzoneFSUtils.isFSOptimizedBucket(bucketType)){

Review comment:
       can we have a KeyIteratorFactory that returns KeyIteratorWithFSO, KeyIterator or future KeyIterator types properly?




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

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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;

Review comment:
       NIT: Can we expand FSO to FILE_SYSTEM_OPTIMIZED and LEGACY to FILE_SYSTEM_LEGACY?




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
##########
@@ -102,6 +107,7 @@
    * @param usedBytes - Bucket Quota Usage in bytes.
    * @param quotaInBytes Bucket quota in bytes.
    * @param quotaInNamespace Bucket quota in counts.
+   * @param bucketType Bucket Types.

Review comment:
       Changed to Bucket Type.




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

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



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


[GitHub] [ozone] elek commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       > Also, the plan is not to have both the `Map<String,String>` as well as the enum together. The Map<String,String> will not be used once the enum has been introduced.
   
   I am against removing the `Map<String,String>` as it can be used not only for storing bucket types but storing any other metadata. Also, it's an incompatible change if we remove it.
   
   And if we need to keep it: wouldn't it make more sense to use it to store the bucket type (as we do it today?).
   
   Again: I fully support to use bucket type as a concept: talk about multiple bucket types, support bucket type argument in cli. I just try to understand the arguments to introduce it a new protobuf level concept instead of using existing metadata capabilities.




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

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -98,27 +100,24 @@
    *
    * @throws IOException
    */
-  @BeforeClass
-  public static void init() throws Exception {
+  @BeforeClass public static void init() throws Exception {
     conf = new OzoneConfiguration();
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
     omId = UUID.randomUUID().toString();
-    TestOMRequestUtils.configureFSOptimizedPaths(conf,
-            true, OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-            .setClusterId(clusterId)
-            .setScmId(scmId)
-            .setOmId(omId)
-            .build();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,
+        OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
     cluster.waitForClusterToBeReady();
     // create a volume and a bucket to be used by OzoneFileSystem
-    OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(cluster);
+    OzoneBucket bucket =
+        TestDataUtil.createVolumeAndBucket(cluster, BucketType.FSO);
     volumeName = bucket.getVolumeName();
     bucketName = bucket.getName();
 
-    String rootPath = String.format("%s://%s.%s/",
-            OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),
+    String rootPath = String
+        .format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),

Review comment:
       Yes, I'll revert the change.




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.

Review comment:
       Updated the Javadoc




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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Thanks a lot @marton for the comments.
   
   > I am against removing the Map<String,String> as it can be used not only for storing bucket types but storing any other metadata. Also, it's an incompatible change if we remove it.
   
   To make it clear about the`Map<String,String>` entity - this patch will not remove `KeyValue` attribute from the BucketInfo and it will remain the same. Any upcoming feature or behaviour can still make use of this `KeyValue` attribute and build it.
   
   _OmClientProtocol.proto_
   ```
       message BucketInfo {
       .....
       repeated hadoop.hdds.KeyValue metadata = 7;
       .....
   + optional BucketLayoutProto bucketLayout = 18;
   ```
   
   This patch is moving the bucket layout information stored in the **Map to an enum attribute.**
   
   > Also, it's an incompatible change if we remove it.
   
   IIUC, since the FSO(prefix) feature is not released in any of the Ozone version we have still the opportunity to refactor/redesign the proto part. Please let me know if I missed anything.
   
   Enum has following benefits:
   - enum has options to limit the set of valid values and will validate the input efficiently. Please refer, https://picocli.info/#_enum_types
   - protobuf defaults to the enum constant at ordinal 0 and not worrying about null checks in a programatic way(easily avoid NPE cases). Not only from the commandline, while reading back the old data from DB, where the bucket layout will not exists in DB.
   - according to me enum makes code more maintainable and easy to read
   
   I'm completely open for a better approach. With map, we have to do all the above in a programatic way and its doable. Please let me know your feedback. Thanks 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.

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -563,7 +583,7 @@ public long getUsedNamespace() {
   public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
       String prevKey) throws IOException {
 
-    if(OzoneFSUtils.isFSOptimizedBucket(getMetadata())){
+    if(OzoneFSUtils.isFSOptimizedBucket(bucketType)){

Review comment:
       @mukul1987, Now we checking using bucketType.equals(BucketType.FILE_SYSTEM_OPTIMIZED).
   @xiaoyuyao Implemented KeyIteratorFactory that returns KeyIteratorWithFSO, KeyIterator.




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

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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Closing the discussion thread, thanks everyone!




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

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

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



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       @elek , Can you please elaborate on this point. Enums are not inherently backward incompatible.  Using strings has no other advantages either.
   
   `using enum makes it harder to introduce new values without compatibility issues.`
   




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

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

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



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


[GitHub] [ozone] rakeshadr commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       @elek,
   If no more comments by EOD IST then I will commit this. If any more concerns, I can address them in later jira.
   
   I would like to get this in, as this is blocking further jira's progress.
   Thanks again for the reviews.




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

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

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



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


[GitHub] [ozone] xiaoyuyao commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -135,6 +136,10 @@
    * Quota of key count allocated for the bucket.
    */
   private long quotaInNamespace;
+  /**
+   * Bucket Type.
+   */
+  private BucketType bucketType;

Review comment:
       I think we need a default bucketType here otherwise Line586 can trigger NPE.




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

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



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -50,6 +51,12 @@
           "false/unspecified indicates otherwise")
   private Boolean isGdprEnforced;
 
+  @Option(names = { "--type", "-t" },

Review comment:
       This should be a ENUM based argument

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -98,27 +100,24 @@
    *
    * @throws IOException
    */
-  @BeforeClass
-  public static void init() throws Exception {
+  @BeforeClass public static void init() throws Exception {
     conf = new OzoneConfiguration();
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
     omId = UUID.randomUUID().toString();
-    TestOMRequestUtils.configureFSOptimizedPaths(conf,
-            true, OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-            .setClusterId(clusterId)
-            .setScmId(scmId)
-            .setOmId(omId)
-            .build();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,
+        OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
     cluster.waitForClusterToBeReady();
     // create a volume and a bucket to be used by OzoneFileSystem
-    OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(cluster);
+    OzoneBucket bucket =
+        TestDataUtil.createVolumeAndBucket(cluster, BucketType.FSO);
     volumeName = bucket.getVolumeName();
     bucketName = bucket.getName();
 
-    String rootPath = String.format("%s://%s.%s/",
-            OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),
+    String rootPath = String
+        .format("%s://%s.%s/", OzoneConsts.OZONE_URI_SCHEME, bucket.getName(),

Review comment:
       Is this change needed

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OmBucketInfo.java
##########
@@ -523,6 +547,9 @@ public BucketInfo getProtobuf() {
         .addAllMetadata(KeyValueUtil.toProtobuf(metadata))
         .setQuotaInBytes(quotaInBytes)
         .setQuotaInNamespace(quotaInNamespace);
+    if (bucketType != null) {
+      bib.setBucketType(bucketType.toProto());
+    }

Review comment:
       Again, what the bucket type if the bucketType is null ?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestObjectStoreWithFSO.java
##########
@@ -98,27 +100,24 @@
    *
    * @throws IOException
    */
-  @BeforeClass
-  public static void init() throws Exception {
+  @BeforeClass public static void init() throws Exception {
     conf = new OzoneConfiguration();
     clusterId = UUID.randomUUID().toString();
     scmId = UUID.randomUUID().toString();
     omId = UUID.randomUUID().toString();
-    TestOMRequestUtils.configureFSOptimizedPaths(conf,
-            true, OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX);
-    cluster = MiniOzoneCluster.newBuilder(conf)
-            .setClusterId(clusterId)
-            .setScmId(scmId)
-            .setOmId(omId)
-            .build();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,

Review comment:
       This line should not be needed

##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {
+    LEGACY = 1;

Review comment:
       Should legacy be legacy filesystem here ?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -61,6 +62,11 @@
   private long quotaInBytes;
   private long quotaInNamespace;
 
+  /**
+   * Bucket Type.
+   */
+  private BucketType bucketType;

Review comment:
       we should have a default value for the bucket type, or is it a required arg ?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -201,6 +206,21 @@ public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
     this.quotaInNamespace = quotaInNamespace;
   }
 
+  @SuppressWarnings("parameternumber")
+  public OzoneBucket(ConfigurationSource conf, ClientProtocol proxy,
+      String volumeName, String bucketName, StorageType storageType,
+      Boolean versioning, long creationTime, long modificationTime,
+      Map<String, String> metadata, String encryptionKeyName,
+      String sourceVolume, String sourceBucket, long usedBytes,
+      long usedNamespace, long quotaInBytes, long quotaInNamespace,
+      BucketType bucketType) {
+    this(conf, proxy, volumeName, bucketName, storageType, versioning,
+        creationTime, modificationTime, metadata, encryptionKeyName,
+        sourceVolume, sourceBucket, usedBytes, usedNamespace, quotaInBytes,
+        quotaInNamespace);
+    this.bucketType = bucketType;

Review comment:
       what will be the bucket type in the previous constructor ?

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -563,7 +583,7 @@ public long getUsedNamespace() {
   public Iterator<? extends OzoneKey> listKeys(String keyPrefix,
       String prevKey) throws IOException {
 
-    if(OzoneFSUtils.isFSOptimizedBucket(getMetadata())){
+    if(OzoneFSUtils.isFSOptimizedBucket(bucketType)){

Review comment:
       should we not check bucketTypre.equals(FSO) or something like that ?

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -60,9 +67,18 @@
   public void execute(OzoneClient client, OzoneAddress address)
       throws IOException {
 
-    BucketArgs.Builder bb = new BucketArgs.Builder()
-        .setStorageType(StorageType.DEFAULT)
-        .setVersioning(false);
+    BucketArgs.Builder bb;
+    if (bucketType.equals(BucketType.OBJECT_STORE) || bucketType

Review comment:
       https://picocli.info/#_enum_types




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -135,6 +136,10 @@
    * Quota of key count allocated for the bucket.
    */
   private long quotaInNamespace;
+  /**
+   * Bucket Type.
+   */
+  private BucketType bucketType;

Review comment:
       I'll add a default bucket type.




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;

Review comment:
       I'll change FSO to FILE_SYSTEM_OPTIMIZED and LEGACY to FILE_SYSTEM_LEGACY




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketType.java
##########
@@ -0,0 +1,56 @@
+/**
+ * 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.om.helpers;
+
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+
+/**
+ * The replication type to be used while writing key into ozone.
+ */
+public enum BucketType {
+  FSO, OBJECT_STORE, LEGACY;
+  public static final BucketType DEFAULT = OBJECT_STORE;
+  public static BucketType fromProto(
+      OzoneManagerProtocolProtos.BucketTypeProto bucketType) {
+    if (bucketType == null) {
+      return BucketType.LEGACY;
+    }
+    switch (bucketType) {
+    case FSO:
+      return BucketType.FSO;
+    case OBJECT_STORE:
+    default:
+      return DEFAULT;
+    }
+  }
+
+  public OzoneManagerProtocolProtos.BucketTypeProto toProto() {
+    switch (this) {
+    case FSO:
+      return OzoneManagerProtocolProtos.BucketTypeProto.FSO;
+    case OBJECT_STORE:
+      return OzoneManagerProtocolProtos.BucketTypeProto.OBJECT_STORE;
+    case LEGACY:

Review comment:
       Thanks, @bharatviswa504 for the comments, I'm thinking below changes:
   If the flag is 'false' technically it will be OBS(object store) and I'll be mapping these buckets to OBS instead of LEGACY.
   If the flag is 'true' it will be a normalized path with intermediate directories and I'll be mapping these buckets to LEGACY. Now, with my above proposal, there will be only one LEGACY that represents an un-optimized filesystem.
   Does this make sense to you?
    




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/bucket/CreateBucketHandler.java
##########
@@ -60,9 +67,18 @@
   public void execute(OzoneClient client, OzoneAddress address)
       throws IOException {
 
-    BucketArgs.Builder bb = new BucketArgs.Builder()
-        .setStorageType(StorageType.DEFAULT)
-        .setVersioning(false);
+    BucketArgs.Builder bb;
+    if (bucketType.equals(BucketType.OBJECT_STORE) || bucketType

Review comment:
       Done




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

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

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



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


[GitHub] [ozone] mukul1987 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Also, the plan is not to have both the `Map<String,String>` as well as the enum together. The `Map<String,String>` will not be used once the enum has been introduced. 




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

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

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



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


[GitHub] [ozone] elek commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket layouts in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {

Review comment:
       Thanks for the suggestion and for the patch @aryangupta1998 . I already shared my concern offline, but let me repeat m y arguments here in public:
   
   While I really like the concept of having different type of buckets (or more preciously: different type of behavior for different buckets), I am not sure if the best representation is introducing a new bucket "type".
   
   We already have a custom `Map<String,String>` field, which (today) is used to store the flags about FSO or non-FSO behavior.
   
   Having both the custom fields (aka. bucket properties/behaviors) and the bucket type seems to be confusing for me from usability point of view.
   
   I have no problem with using bucket type on CLI but wouldn't be enough to implement it only on the client side and use the good old `Map<String,String>` properties? 
   
   
   It would be more backward and forward compatible (using enum makes it harder to introduce new values without compatibility issues.)
   
   What do you think? 
    




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

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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/BucketArgs.java
##########
@@ -61,6 +62,11 @@
   private long quotaInBytes;
   private long quotaInNamespace;
 
+  /**
+   * Bucket Type.
+   */
+  private BucketType bucketType;

Review comment:
       Yes, I'll add a default value to bucketType.




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

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



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


[GitHub] [ozone] aryangupta1998 commented on a change in pull request #2357: HDDS-5362. [FSO] Support bucket types in OM

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



##########
File path: hadoop-ozone/interface-client/src/main/proto/OmClientProtocol.proto
##########
@@ -528,6 +529,12 @@ enum StorageTypeProto {
     RAM_DISK = 4;
 }
 
+enum BucketTypeProto {
+    LEGACY = 1;

Review comment:
       Yes it represents legacy filesystem, I'll rename to LEGACY_FILESYSTEM.




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

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



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