You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2022/06/17 12:06:49 UTC

[GitHub] [ozone] JyotinderSingh opened a new pull request, #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

JyotinderSingh opened a new pull request, #3528:
URL: https://github.com/apache/ozone/pull/3528

   ## What changes were proposed in this pull request?
   
   Buckets created with OFS should support File System semantics. This functionality was accidentally removed as a part of #3296.
   This patch adds it back and also adds validation logic for the same.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-6907
   
   ## How was this patch tested?
   
   Existing tests.
   


-- 
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] JyotinderSingh commented on a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r902267396


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java:
##########
@@ -0,0 +1,224 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.TrashPolicyOzone;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TrashPolicy;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Ozone file system tests to validate default bucket layout configuration
+ * and behaviour.
+ * TODO: Refactor this and TestOzoneFileSystem to reduce duplication.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneFSBucketLayout {
+
+  private static BucketLayout defaultBucketLayout;
+
+  private static MiniOzoneCluster cluster = null;
+  private static FileSystem fs;
+  private static ObjectStore objectStore;
+  private static BasicRootedOzoneClientAdapterImpl adapter;
+
+  private String volumeName;
+  private Path volumePath;
+
+  @Parameterized.Parameters
+  public static Collection<BucketLayout> data() {
+    return Arrays.asList(
+        null,
+        BucketLayout.FILE_SYSTEM_OPTIMIZED,
+        BucketLayout.LEGACY,
+        BucketLayout.OBJECT_STORE
+    );
+  }
+
+  public TestOzoneFSBucketLayout(BucketLayout bucketLayout) {
+    // Ignored. Actual init done in initParam().
+    // This empty constructor is still required to avoid argument exception.
+  }
+
+  @Parameterized.BeforeParam
+  public static void initParam(BucketLayout bucketLayout)
+      throws IOException, InterruptedException, TimeoutException {
+    // Initialize the cluster before EACH set of parameters
+    defaultBucketLayout = bucketLayout;
+
+    initClusterAndEnv();
+  }
+
+  @Parameterized.AfterParam
+  public static void teardownParam() {
+    // Tear down the cluster after EACH set of parameters
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    IOUtils.closeQuietly(fs);
+  }
+
+  @Before
+  public void createVolumeAndBucket() throws IOException {
+    // create a volume and a bucket to be used by RootedOzoneFileSystem (OFS)
+    volumeName =
+        TestDataUtil.createVolumeAndBucket(cluster)
+            .getVolumeName();
+    volumePath = new Path(OZONE_URI_DELIMITER, volumeName);
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    if (defaultBucketLayout != null &&
+        defaultBucketLayout.equals(BucketLayout.OBJECT_STORE)) {
+      return;
+    }
+    fs.delete(volumePath, true);
+  }
+
+  public static FileSystem getFs() {

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 a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r900291520


##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -190,6 +192,12 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort,
       this.configuredDnPort = conf.getInt(
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+
+      // Fetches the bucket layout to be used by OFS.
+      this.defaultOFSBucketLayout = BucketLayout.fromString(
+          conf.get(OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT,
+              OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT_DEFAULT));

Review Comment:
   Add a client side validation and throw exception if its configured a value other than FSO/LEGACY buck layouts. Exception can be IllegalArgumentException or OMException or a better one.



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java:
##########
@@ -476,6 +476,15 @@ public final class OzoneConfigKeys {
       OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS =
       300 * 1000;
 
+  public static final String OZONE_FS_DEFAULT_BUCKET_LAYOUT =

Review Comment:
   @JyotinderSingh Can we add this config in OzoneClientConfig. I could see, this class encapsulates all the ozone client configs. 
   
   ```
   /**
    * Configuration values for Ozone Client.
    */
   @ConfigGroup(prefix = "ozone.client")
   public class OzoneClientConfig {
   ```
   
   With the prefix, the config name will become  `ozone.client.fs.default.bucket.layout`
   



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java:
##########
@@ -224,6 +224,8 @@ public static void initClusterAndEnv() throws IOException,
           bucketLayout.name());
     } else {
       bucketLayout = BucketLayout.LEGACY;
+      conf.set(OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT,

Review Comment:
   Can we add specific test case safe guarding ofs bucket creation behavior. Probably you can add a new test class itself to verify different cases:
   
   Pre-requisite) ozone.default.bucket.layout Server defaults to OBJECT_STORE. As this should be ignored at any point of time.
   
   case-1) client side configured to empty, use default FSO. Should create buck as FSO.
   case-2) client side configured to OBS, throws Exception
   case-3) client side configured to LEGACY, ofs should create buck as LEGACY.
   case-4) client side configured to FSO, ofs should create buck as FSO.



-- 
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 merged pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

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


-- 
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] JyotinderSingh commented on pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on PR #3528:
URL: https://github.com/apache/ozone/pull/3528#issuecomment-1159789227

   Thank you for the reviews @rakeshadr. I have addressed them in my latest commit.


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

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

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


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


[GitHub] [ozone] JyotinderSingh commented on a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r901142242


##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java:
##########
@@ -476,6 +476,15 @@ public final class OzoneConfigKeys {
       OZONE_CLIENT_BUCKET_REPLICATION_CONFIG_REFRESH_PERIOD_DEFAULT_MS =
       300 * 1000;
 
+  public static final String OZONE_FS_DEFAULT_BUCKET_LAYOUT =

Review Comment:
   Done



##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestRootedOzoneFileSystem.java:
##########
@@ -224,6 +224,8 @@ public static void initClusterAndEnv() throws IOException,
           bucketLayout.name());
     } else {
       bucketLayout = BucketLayout.LEGACY;
+      conf.set(OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT,

Review Comment:
   Added the tests.



-- 
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] JyotinderSingh commented on a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r901142262


##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -190,6 +192,12 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort,
       this.configuredDnPort = conf.getInt(
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+
+      // Fetches the bucket layout to be used by OFS.
+      this.defaultOFSBucketLayout = BucketLayout.fromString(
+          conf.get(OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT,
+              OzoneConfigKeys.OZONE_FS_DEFAULT_BUCKET_LAYOUT_DEFAULT));

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 a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r902090784


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java:
##########
@@ -0,0 +1,224 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.TrashPolicyOzone;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TrashPolicy;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Ozone file system tests to validate default bucket layout configuration
+ * and behaviour.
+ * TODO: Refactor this and TestOzoneFileSystem to reduce duplication.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneFSBucketLayout {
+
+  private static BucketLayout defaultBucketLayout;
+
+  private static MiniOzoneCluster cluster = null;
+  private static FileSystem fs;
+  private static ObjectStore objectStore;
+  private static BasicRootedOzoneClientAdapterImpl adapter;
+
+  private String volumeName;
+  private Path volumePath;
+
+  @Parameterized.Parameters
+  public static Collection<BucketLayout> data() {
+    return Arrays.asList(
+        null,

Review Comment:
   Can we add a case where the config sets to INVALID.
   
   A case where `clientConfig.setFsDefaultBucketLayout("INVALID");`



-- 
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 diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r902109558


##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -190,6 +192,30 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort,
       this.configuredDnPort = conf.getInt(
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+
+      // Fetches the bucket layout to be used by OFS.

Review Comment:
   Please extract this entire bucket layout init logic to a method, like,
   
   `initDefaultBucketLayout(conf);`



-- 
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] JyotinderSingh commented on a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
JyotinderSingh commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r902267688


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java:
##########
@@ -0,0 +1,224 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.TrashPolicyOzone;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TrashPolicy;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Ozone file system tests to validate default bucket layout configuration
+ * and behaviour.
+ * TODO: Refactor this and TestOzoneFileSystem to reduce duplication.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneFSBucketLayout {
+
+  private static BucketLayout defaultBucketLayout;
+
+  private static MiniOzoneCluster cluster = null;
+  private static FileSystem fs;
+  private static ObjectStore objectStore;
+  private static BasicRootedOzoneClientAdapterImpl adapter;
+
+  private String volumeName;
+  private Path volumePath;
+
+  @Parameterized.Parameters
+  public static Collection<BucketLayout> data() {
+    return Arrays.asList(
+        null,

Review Comment:
   Done.



##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/BasicRootedOzoneClientAdapterImpl.java:
##########
@@ -190,6 +192,30 @@ public BasicRootedOzoneClientAdapterImpl(String omHost, int omPort,
       this.configuredDnPort = conf.getInt(
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
           OzoneConfigKeys.DFS_CONTAINER_IPC_PORT_DEFAULT);
+
+      // Fetches the bucket layout to be used by OFS.

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 a diff in pull request #3528: HDDS-6907. OFS should create buckets with FILE_SYSTEM_OPTIMIZED layout.

Posted by GitBox <gi...@apache.org>.
rakeshadr commented on code in PR #3528:
URL: https://github.com/apache/ozone/pull/3528#discussion_r902118253


##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFSBucketLayout.java:
##########
@@ -0,0 +1,224 @@
+/**
+ * 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.fs.ozone;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.TrashPolicyOzone;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.io.IOException;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_FS_ITERATE_BATCH_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.junit.Assert.fail;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.TrashPolicy;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.TestDataUtil;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+/**
+ * Ozone file system tests to validate default bucket layout configuration
+ * and behaviour.
+ * TODO: Refactor this and TestOzoneFileSystem to reduce duplication.
+ */
+@RunWith(Parameterized.class)
+public class TestOzoneFSBucketLayout {
+
+  private static BucketLayout defaultBucketLayout;
+
+  private static MiniOzoneCluster cluster = null;
+  private static FileSystem fs;
+  private static ObjectStore objectStore;
+  private static BasicRootedOzoneClientAdapterImpl adapter;
+
+  private String volumeName;
+  private Path volumePath;
+
+  @Parameterized.Parameters
+  public static Collection<BucketLayout> data() {
+    return Arrays.asList(
+        null,
+        BucketLayout.FILE_SYSTEM_OPTIMIZED,
+        BucketLayout.LEGACY,
+        BucketLayout.OBJECT_STORE
+    );
+  }
+
+  public TestOzoneFSBucketLayout(BucketLayout bucketLayout) {
+    // Ignored. Actual init done in initParam().
+    // This empty constructor is still required to avoid argument exception.
+  }
+
+  @Parameterized.BeforeParam
+  public static void initParam(BucketLayout bucketLayout)
+      throws IOException, InterruptedException, TimeoutException {
+    // Initialize the cluster before EACH set of parameters
+    defaultBucketLayout = bucketLayout;
+
+    initClusterAndEnv();
+  }
+
+  @Parameterized.AfterParam
+  public static void teardownParam() {
+    // Tear down the cluster after EACH set of parameters
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+    IOUtils.closeQuietly(fs);
+  }
+
+  @Before
+  public void createVolumeAndBucket() throws IOException {
+    // create a volume and a bucket to be used by RootedOzoneFileSystem (OFS)
+    volumeName =
+        TestDataUtil.createVolumeAndBucket(cluster)
+            .getVolumeName();
+    volumePath = new Path(OZONE_URI_DELIMITER, volumeName);
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    if (defaultBucketLayout != null &&
+        defaultBucketLayout.equals(BucketLayout.OBJECT_STORE)) {
+      return;
+    }
+    fs.delete(volumePath, true);
+  }
+
+  public static FileSystem getFs() {

Review Comment:
   remove this un used method.



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

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

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


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