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/08/24 13:12:41 UTC

[GitHub] [ozone] zhengchenyu opened a new pull request #2571: HDDS-5664. [FSO] Ignore validate bucket layout since bucket layout su…

zhengchenyu opened a new pull request #2571:
URL: https://github.com/apache/ozone/pull/2571


   …pported in HDDS-5362.
   
   ## What changes were proposed in this pull request?
   
   I have a ozone cluster, prefix is enable when setup cluster. when restart om, then error below:
   
   ```
   java.io.IOException: Failed to start OM in PREFIX layout format as existing bucket has a different layout SIMPLE metadata format
   ```
   
   This issue is similar to HDDS-5094. Later in  HDDS-5362, bucket layout is supported, then addFSOptimizedBucketDetails is removed in this https://github.com/apache/ozone/commit/7df192530810a07d3b976c3a56c46328c0b1d591. So om crash when restart if layout is PREFIX.
   
   I think there is no need to validate bucket type since HDDS-5362.
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5664
   
   ## How was this patch tested?
   
   unit tests and manual 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] rakeshadr commented on a change in pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       @zhengchenyu
   I think the overall test needs to be organized as we are replacing simple/prefix concept with bucket layout. I am trying to add pseudo code idea info like,
   
   Test case: testRestartWithMultipleLayouts
                       
                       cluster = MiniOzoneCluster.newBuilder(conf).withoutDatanodes().build();
                       cluster.waitForClusterToBeReady();
                       OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.FILE_SYSTEM_OPTIMIZED);
                       verifyBucketLayout(bucket, BucketLayout.FILE_SYSTEM_OPTIMIZED, true);
                       bucket = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.LEGACY);
                       verifyBucketLayout(bucket, BucketLayout.OBJECT_STORE, true);
                       bucket = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.OBJECT_STORE);
                       verifyBucketLayout(bucket, BucketLayout.OBJECT_STORE, false);
   					
   					// stop cluster
                       cluster.stop();
   
                       // update om default buck layout config value to 
                       conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, BucketLayout.FILE_SYSTEM_OPTIMIZED.name());
                   
                   	// restart OM with updated config
                       cluster.getOzoneManager().restart();
                       
                       bucket = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.LEGACY);
                       verifyBucketLayout(bucket, BucketLayout.FILE_SYSTEM_OPTIMIZED, true);




-- 
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 #2571: HDDS-5664. [FSO] Ignore validate bucket layout since bucket layout su…

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithLayout.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithLayout {

Review comment:
       Please rename the test to `TestOMStartupWithBucketLayout` to make it specific. Also, modify the java comments, Thanks!




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

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

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



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


[GitHub] [ozone] zhengchenyu commented on a change in pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       I will add more verification. Why I write unit-test like this? Because some layout setting dismatched, cluster could not start.




-- 
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 #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       @zhengchenyu
   I think the overall test needs to be organized as we are replacing simple/prefix concept with bucket layout. I am trying to add pseudo code idea info like,
   
   _Test case: testRestartWithMultipleLayouts_
                       
                       cluster = MiniOzoneCluster.newBuilder(conf).withoutDatanodes().build();
                       cluster.waitForClusterToBeReady();
                       OzoneBucket bucket1 = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.FILE_SYSTEM_OPTIMIZED);
                       verifyBucketLayout(bucket1, BucketLayout.FILE_SYSTEM_OPTIMIZED, true);
   
                       // verify OM default behavior with LEGACY
                       OzoneBucket bucket2 = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.LEGACY);
                       verifyBucketLayout(bucket2, BucketLayout.OBJECT_STORE, false);
                       OzoneBucket bucket3 = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.OBJECT_STORE);
                       verifyBucketLayout(bucket3, BucketLayout.OBJECT_STORE, false);
   					
   		        // stop cluster
                       cluster.stop();
   
                       // update om default buck layout config value to 
                       conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT, BucketLayout.FILE_SYSTEM_OPTIMIZED.name());
                   
                   	// restart OM with updated config
                       cluster.getOzoneManager().restart();
                       
                       // verify OM default behavior with LEGACY
                       OzoneBucket bucket4 = TestDataUtil.createVolumeAndBucket(cluster, BucketLayout.LEGACY);
                       verifyBucketLayout(bucket4, BucketLayout.FILE_SYSTEM_OPTIMIZED, true);
   
                       // verify existing buckets after OM restart.
                       OzoneVolume volume = objectStore.getVolume(volumeName);
                       bucket1 = volume.getBucket(bucket1.getName());
                       bucket2 = volume.getBucket(bucket2.getName());
                       bucket3 = volume.getBucket(bucket3.getName());
                       verifyBucketLayout(bucket1, BucketLayout.FILE_SYSTEM_OPTIMIZED, true);
                       verifyBucketLayout(bucket2, BucketLayout.OBJECT_STORE, false);
                       verifyBucketLayout(bucket3, BucketLayout.OBJECT_STORE, false);




-- 
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 #2571: HDDS-5664. [FSO] Ignore validate bucket layout since bucket layout su…

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithLayout.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,

Review comment:
       Should we need` TestOMRequestUtils.configureFSOptimizedPaths` this config setting because we will be removing this configuration soon.
   
   I think the test case has to modified to reflect the recent code changes.
   
   **Test case:** Create buckets with OBS and FSO layouts in OM. Then restart OM server.




-- 
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] zhengchenyu closed pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

Posted by GitBox <gi...@apache.org>.
zhengchenyu closed pull request #2571:
URL: https://github.com/apache/ozone/pull/2571


   


-- 
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 #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {

Review comment:
       The plan is to remove the simple/prefix related items and maintain only bucket layouts - FSO/OBS/LEGACY. Could you please replace it with OBS




-- 
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] zhengchenyu commented on pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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


   @rakeshadr I have rebase this PR, please check 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] rakeshadr commented on a change in pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)

Review comment:
       @zhengchenyu Can you make zero dns as this test requires only OMs, right
   
   `cluster = MiniOzoneCluster.newBuilder(conf).withoutDatanodes().build();`




-- 
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 #2571: HDDS-5664. [FSO] Ignore validate bucket layout since bucket layout su…

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


   Thanks a lot @zhengchenyu for the contribution. Since we have finished most of the basic patches, now I'm thinking to take this ahead.
   
   Could you please rebase the PR and resolve conflicts.


-- 
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] zhengchenyu commented on a change in pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       I will add more verification and rename simple/prefix to obs/fso. Why I write unit-test like this? Because some layout setting dismatched, cluster could not start.




-- 
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 #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       @zhengchenyu I have reviewed the updated test case, it looks fine to me. I liked the java comments, its pretty clear about the verification cases.




-- 
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 #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithBucketLayout.java
##########
@@ -0,0 +1,151 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithBucketLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    conf.set(OMConfigKeys.OZONE_DEFAULT_BUCKET_LAYOUT,
+        BucketLayout.OBJECT_STORE.name());
+    cluster = MiniOzoneCluster.newBuilder(conf).setClusterId(clusterId)
+        .setScmId(scmId).setOmId(omId).build();
+    cluster.waitForClusterToBeReady();
+  }
+
+  public static void startClusterWithPrefixLayout() throws Exception {

Review comment:
       The plan is to remove the simple/prefix related items and maintain only bucket layouts - FSO/OBS/LEGACY. Could you please replace it with 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] zhengchenyu closed pull request #2571: HDDS-5664. [FSO] Ignore validate bucket layout since bucket layout su…

Posted by GitBox <gi...@apache.org>.
zhengchenyu closed pull request #2571:
URL: https://github.com/apache/ozone/pull/2571


   


-- 
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 #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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


   


-- 
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] zhengchenyu commented on a change in pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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



##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMStartupWithLayout.java
##########
@@ -0,0 +1,156 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.ozone.om;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.TestDataUtil;
+import org.apache.hadoop.ozone.client.OzoneBucket;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
+import org.apache.hadoop.ozone.om.request.TestOMRequestUtils;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.Timeout;
+
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_DEFAULT;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_METADATA_LAYOUT_PREFIX;
+
+/**
+ * Verifies OM startup with different layout.
+ */
+public class TestOMStartupWithLayout {
+
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = Timeout.seconds(300);
+
+  private static MiniOzoneCluster cluster;
+
+  public static void startClusterWithSimpleLayout() throws Exception {
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String clusterId = UUID.randomUUID().toString();
+    String scmId = UUID.randomUUID().toString();
+    String omId = UUID.randomUUID().toString();
+    TestOMRequestUtils.configureFSOptimizedPaths(conf, true,

Review comment:
       Yeah, I also think OZONE_DEFAULT_BUCKET_LAYOUT should be removed. In this issue, just this config trigger the bug.  
   
   I will remove OZONE_DEFAULT_BUCKET_LAYOUT's setting in UT, then the bug will not be triggered. But I think we also should ignore the verifyBucketMetaLayout. 
   




-- 
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] zhengchenyu commented on pull request #2571: [FSO] Allow existing/old buckets with any layout during OM startup

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


   > Thank you @zhengchenyu for the updates. Added few comments, please take a look at it.
   
   Thank you for review, I have rebase to 84c1adeba4a934ac8057d25c5a5b941a18545f3f.


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