You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by jy...@apache.org on 2022/05/13 06:53:56 UTC

[ozone] branch master updated: HDDS-6680. Pre-Finalize behaviour for Bucket Layout Feature. (#3377)

This is an automated email from the ASF dual-hosted git repository.

jyotinder pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new 6edcc13900 HDDS-6680. Pre-Finalize behaviour for Bucket Layout Feature. (#3377)
6edcc13900 is described below

commit 6edcc139003602f25532b383175050a2f3b42317
Author: Jyotinder Singh <jy...@gmail.com>
AuthorDate: Fri May 13 12:23:51 2022 +0530

    HDDS-6680. Pre-Finalize behaviour for Bucket Layout Feature. (#3377)
---
 .../hadoop/ozone/om/helpers/BucketLayout.java      |   4 +
 .../non-rolling-upgrade/1.2.1-1.3.0/callback.sh    |   2 +-
 .../dist/src/main/smoketest/upgrade/finalize.robot |   2 +-
 .../apache/hadoop/ozone/om/OMUpgradeTestUtils.java |  88 +++++++
 .../hadoop/ozone/om/TestOMBucketLayoutUpgrade.java | 272 +++++++++++++++++++++
 .../hadoop/ozone/om/TestOMUpgradeFinalization.java |  52 +---
 .../om/request/bucket/OMBucketCreateRequest.java   |  29 ++-
 .../hadoop/ozone/om/upgrade/OMLayoutFeature.java   |   6 +-
 8 files changed, 400 insertions(+), 55 deletions(-)

diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketLayout.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketLayout.java
index 00118ccad3..a9803e03e4 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketLayout.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/BucketLayout.java
@@ -65,6 +65,10 @@ public enum BucketLayout {
     return this.equals(FILE_SYSTEM_OPTIMIZED);
   }
 
+  public boolean isLegacy() {
+    return this.equals(LEGACY);
+  }
+
   public boolean shouldNormalizePaths(boolean enableFileSystemPaths) {
     switch (this) {
     case OBJECT_STORE:
diff --git a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
index f414ddf43f..d54c9bc89b 100755
--- a/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
+++ b/hadoop-ozone/dist/src/main/compose/upgrade/upgrades/non-rolling-upgrade/1.2.1-1.3.0/callback.sh
@@ -71,7 +71,7 @@ with_old_version_downgraded() {
 
 with_new_version_finalized() {
   _check_hdds_mlvs 3
-  _check_om_mlvs 1
+  _check_om_mlvs 2
 
   validate old1
   validate new1
diff --git a/hadoop-ozone/dist/src/main/smoketest/upgrade/finalize.robot b/hadoop-ozone/dist/src/main/smoketest/upgrade/finalize.robot
index 288f9c0150..b70f3ca147 100644
--- a/hadoop-ozone/dist/src/main/smoketest/upgrade/finalize.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/upgrade/finalize.robot
@@ -19,7 +19,7 @@ Resource            ../commonlib.robot
 Test Timeout        5 minutes
 Test Setup          Run Keyword if    '${SECURITY_ENABLED}' == 'true'    Kinit test user     testuser     testuser.keytab
 
-** Test Cases ***
+*** Test Cases ***
 Finalize SCM
     ${result} =        Execute      ozone admin scm finalizeupgrade
                        #Wait Until Keyword Succeeds      3min       10sec     Should contain   ${result}   OM Preparation successful!
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/OMUpgradeTestUtils.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/OMUpgradeTestUtils.java
new file mode 100644
index 0000000000..093cc0943f
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/OMUpgradeTestUtils.java
@@ -0,0 +1,88 @@
+/**
+ * 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.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.ozone.test.LambdaTestUtils;
+import org.junit.Assert;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeoutException;
+
+import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse.PrepareStatus.PREPARE_COMPLETED;
+import static org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.Status.FINALIZATION_DONE;
+import static org.apache.ozone.test.GenericTestUtils.waitFor;
+
+/**
+ * Utility class to help test OM upgrade scenarios.
+ */
+public final class OMUpgradeTestUtils {
+
+  private OMUpgradeTestUtils() {
+    // Utility class.
+  }
+
+  public static void assertClusterPrepared(
+      long preparedIndex, List<OzoneManager> ozoneManagers) throws Exception {
+    for (OzoneManager om : ozoneManagers) {
+      LambdaTestUtils.await(120000,
+          1000, () -> {
+            if (!om.isRunning()) {
+              return false;
+            } else {
+              boolean preparedAtIndex = false;
+              OzoneManagerPrepareState.State state =
+                  om.getPrepareState().getState();
+
+              if (state.getStatus() == PREPARE_COMPLETED) {
+                if (state.getIndex() == preparedIndex) {
+                  preparedAtIndex = true;
+                } else {
+                  // State will not change if we are prepared at the wrong
+                  // index. Break out of wait.
+                  throw new Exception("OM " + om.getOMNodeId() + " prepared " +
+                      "but prepare index " + state.getIndex() + " does not " +
+                      "match expected prepare index " + preparedIndex);
+                }
+              }
+              return preparedAtIndex;
+            }
+          });
+    }
+  }
+
+  public static void waitForFinalization(OzoneManagerProtocol omClient)
+      throws TimeoutException, InterruptedException {
+    waitFor(() -> {
+      try {
+        UpgradeFinalizer.StatusAndMessages statusAndMessages =
+            omClient.queryUpgradeFinalizationProgress("finalize-test", false,
+                false);
+        System.out.println("Finalization Messages : " +
+            statusAndMessages.msgs());
+        return statusAndMessages.status().equals(FINALIZATION_DONE);
+      } catch (IOException e) {
+        Assert.fail(e.getMessage());
+      }
+      return false;
+    }, 2000, 20000);
+  }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMBucketLayoutUpgrade.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMBucketLayoutUpgrade.java
new file mode 100644
index 0000000000..a4442f2d29
--- /dev/null
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMBucketLayoutUpgrade.java
@@ -0,0 +1,272 @@
+/**
+ * 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.commons.lang3.RandomStringUtils;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.MiniOzoneHAClusterImpl;
+import org.apache.hadoop.ozone.client.ObjectStore;
+import org.apache.hadoop.ozone.client.OzoneClientFactory;
+import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.BucketLayout;
+import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
+import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature;
+import org.apache.hadoop.ozone.upgrade.UpgradeFinalizer;
+import org.apache.ozone.test.LambdaTestUtils;
+import org.junit.Test;
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.rules.Timeout;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.UUID;
+
+import static org.apache.hadoop.ozone.OzoneConsts.LAYOUT_VERSION_KEY;
+import static org.apache.hadoop.ozone.om.OMUpgradeTestUtils.waitForFinalization;
+import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.INITIAL_VERSION;
+import static org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager.maxLayoutVersion;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Upgrade testing for Bucket Layout Feature.
+ * <p>
+ * Expected behavior:
+ * 1. Pre-Finalize: OM should not allow creation of buckets with new bucket
+ * layouts. Only LEGACY buckets are allowed.
+ * <p>
+ * 2. Post-Finalize: OM should allow creation of buckets with new bucket
+ * layouts.
+ */
+@RunWith(Parameterized.class)
+public class TestOMBucketLayoutUpgrade {
+  /**
+   * Set a timeout for each test.
+   */
+  @Rule
+  public Timeout timeout = new Timeout(300000);
+  private MiniOzoneHAClusterImpl cluster;
+  private OzoneManager ozoneManager;
+  private ClientProtocol clientProtocol;
+  private static final String VOLUME_NAME = "vol-" + UUID.randomUUID();
+  private int fromLayoutVersion;
+  private OzoneManagerProtocol omClient;
+
+  private static final Logger LOG =
+      LoggerFactory.getLogger(TestOMBucketLayoutUpgrade.class);
+
+  /**
+   * Defines a "from" layout version to finalize from.
+   *
+   * @return
+   */
+  @Parameterized.Parameters
+  public static Collection<Object[]> data() {
+    return Arrays.asList(new Object[][]{
+        {INITIAL_VERSION}
+    });
+  }
+
+
+  public TestOMBucketLayoutUpgrade(OMLayoutFeature fromVersion) {
+    this.fromLayoutVersion = fromVersion.layoutVersion();
+  }
+
+  /**
+   * Create a MiniDFSCluster for testing.
+   */
+  @Before
+  public void setup() throws Exception {
+    org.junit.Assume.assumeTrue("Check if there is need to finalize.",
+        maxLayoutVersion() > fromLayoutVersion);
+
+    OzoneConfiguration conf = new OzoneConfiguration();
+    String omServiceId = UUID.randomUUID().toString();
+    cluster = (MiniOzoneHAClusterImpl) MiniOzoneCluster.newOMHABuilder(conf)
+        .setClusterId(UUID.randomUUID().toString())
+        .setScmId(UUID.randomUUID().toString())
+        .setOMServiceId(omServiceId)
+        .setNumOfOzoneManagers(3)
+        .setNumDatanodes(1)
+        .setOmLayoutVersion(fromLayoutVersion)
+        .build();
+
+    cluster.waitForClusterToBeReady();
+    ozoneManager = cluster.getOzoneManager();
+    ObjectStore objectStore = OzoneClientFactory.getRpcClient(omServiceId, conf)
+        .getObjectStore();
+    clientProtocol = objectStore.getClientProxy();
+    omClient = clientProtocol.getOzoneManagerClient();
+
+    // create sample volume.
+    omClient.createVolume(
+        new OmVolumeArgs.Builder()
+            .setVolume(VOLUME_NAME)
+            .setOwnerName("user1")
+            .setAdminName("user1")
+            .build());
+  }
+
+  /**
+   * Shutdown MiniDFSCluster.
+   */
+  @After
+  public void shutdown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  /**
+   * Tests that OM blocks all requests to create any buckets with a new bucket
+   * layout.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testCreateBucketWithBucketLayoutsDuringUpgrade()
+      throws Exception {
+    // Assert OM layout version is 'fromLayoutVersion' on deploy.
+    assertEquals(fromLayoutVersion,
+        ozoneManager.getVersionManager().getMetadataLayoutVersion());
+    assertNull(ozoneManager.getMetadataManager().getMetaTable()
+        .get(LAYOUT_VERSION_KEY));
+
+    // Test bucket creation with new bucket layouts.
+    // FSO and OBS bucket creation should fail.
+    verifyBucketCreationBlockedWithNewLayouts();
+
+    // Bucket creation with LEGACY layout should succeed in Pre-Finalized state.
+    LOG.info("Creating legacy bucket during Pre-Finalize");
+    verifyBucketCreationWithLayout(new BucketLayout[]{BucketLayout.LEGACY});
+
+    // Finalize the cluster upgrade.
+    finalizeUpgrade();
+
+    // Cluster upgrade is now complete,
+    // Bucket creation should now succeed with all layouts.
+    verifyBucketCreationWithLayout(new BucketLayout[]{
+        BucketLayout.LEGACY,
+        BucketLayout.FILE_SYSTEM_OPTIMIZED,
+        BucketLayout.OBJECT_STORE
+    });
+  }
+
+
+  /**
+   * Tests that OM allows bucket creation with given bucket layouts.
+   *
+   * @param bucketLayouts bucket layouts to test
+   * @throws Exception if any
+   */
+  private void verifyBucketCreationWithLayout(BucketLayout[] bucketLayouts)
+      throws Exception {
+    String bucketName;
+    for (BucketLayout layout : bucketLayouts) {
+      LOG.info("Creating bucket with layout {} after OM finalization", layout);
+
+      bucketName = createBucketWithLayout(layout);
+
+      // Make sure the bucket exists in the bucket table with the
+      // expected layout.
+      assertEquals(
+          omClient.getBucketInfo(VOLUME_NAME, bucketName).getBucketName(),
+          bucketName);
+      assertEquals(
+          omClient.getBucketInfo(VOLUME_NAME, bucketName).getBucketLayout(),
+          layout);
+    }
+  }
+
+  /**
+   * Tests that OM blocks all requests to create any buckets with a new bucket
+   * layout.
+   *
+   * @throws Exception if any
+   */
+  private void verifyBucketCreationBlockedWithNewLayouts() throws Exception {
+    BucketLayout[] bucketLayouts = new BucketLayout[]{
+        BucketLayout.OBJECT_STORE,
+        BucketLayout.FILE_SYSTEM_OPTIMIZED,
+    };
+
+    for (BucketLayout layout : bucketLayouts) {
+      try {
+        LOG.info("Creating bucket with layout {} during Pre-Finalize", layout);
+        createBucketWithLayout(layout);
+        fail("Expected to fail creating bucket with layout " + layout);
+      } catch (OMException e) {
+        // Expected exception.
+        assertEquals(
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION,
+            e.getResult());
+      }
+    }
+  }
+
+  /**
+   * Helper method to create a bucket with the given layout.
+   *
+   * @param bucketLayout the layout to use for the bucket.
+   * @return the name of the bucket created.
+   * @throws Exception if there is an error creating the bucket.
+   */
+  private String createBucketWithLayout(BucketLayout bucketLayout)
+      throws Exception {
+    String bucketName = RandomStringUtils.randomAlphabetic(10).toLowerCase();
+    omClient.createBucket(
+        new OmBucketInfo.Builder()
+            .setVolumeName(VOLUME_NAME)
+            .setBucketName(bucketName)
+            .setBucketLayout(bucketLayout)
+            .build());
+
+    return bucketName;
+  }
+
+  /**
+   * Complete the cluster upgrade.
+   *
+   * @throws Exception if upgrade fails.
+   */
+  private void finalizeUpgrade() throws Exception {
+    UpgradeFinalizer.StatusAndMessages response =
+        omClient.finalizeUpgrade("finalize-test");
+    System.out.println("Finalization Messages : " + response.msgs());
+
+    waitForFinalization(omClient);
+
+    LambdaTestUtils.await(30000, 3000, () -> {
+      String lvString = ozoneManager.getMetadataManager().getMetaTable()
+          .get(LAYOUT_VERSION_KEY);
+      return maxLayoutVersion() == Integer.parseInt(lvString);
+    });
+  }
+}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMUpgradeFinalization.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMUpgradeFinalization.java
index 5a109f7975..9deb9ed63c 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMUpgradeFinalization.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOMUpgradeFinalization.java
@@ -19,16 +19,15 @@ package org.apache.hadoop.ozone.om;
  */
 
 import static org.apache.hadoop.ozone.OzoneConsts.LAYOUT_VERSION_KEY;
+import static org.apache.hadoop.ozone.om.OMUpgradeTestUtils.assertClusterPrepared;
+import static org.apache.hadoop.ozone.om.OMUpgradeTestUtils.waitForFinalization;
 import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeature.INITIAL_VERSION;
 import static org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager.maxLayoutVersion;
-import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.PrepareStatusResponse.PrepareStatus.PREPARE_COMPLETED;
-import static org.apache.hadoop.ozone.upgrade.UpgradeFinalizer.Status.FINALIZATION_DONE;
 import static org.apache.ozone.test.GenericTestUtils.waitFor;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -204,51 +203,4 @@ public class TestOMUpgradeFinalization {
     assertEquals(maxLayoutVersion(),
         Integer.parseInt(lvString));
   }
-
-  private void assertClusterPrepared(
-      long preparedIndex, List<OzoneManager> ozoneManagers) throws Exception {
-    for (OzoneManager om : ozoneManagers) {
-      LambdaTestUtils.await(120000,
-          1000, () -> {
-            if (!om.isRunning()) {
-              return false;
-            } else {
-              boolean preparedAtIndex = false;
-              OzoneManagerPrepareState.State state =
-                  om.getPrepareState().getState();
-
-              if (state.getStatus() == PREPARE_COMPLETED) {
-                if (state.getIndex() == preparedIndex) {
-                  preparedAtIndex = true;
-                } else {
-                  // State will not change if we are prepared at the wrong
-                  // index. Break out of wait.
-                  throw new Exception("OM " + om.getOMNodeId() + " prepared " +
-                      "but prepare index " + state.getIndex() + " does not " +
-                      "match expected prepare index " + preparedIndex);
-                }
-              }
-              return preparedAtIndex;
-            }
-          });
-    }
-  }
-
-  private void waitForFinalization(OzoneManagerProtocol omClient)
-      throws TimeoutException, InterruptedException {
-    waitFor(() -> {
-      try {
-        StatusAndMessages statusAndMessages =
-            omClient.queryUpgradeFinalizationProgress("finalize-test", false,
-                false);
-        System.out.println("Finalization Messages : " +
-            statusAndMessages.msgs());
-        return statusAndMessages.status().equals(FINALIZATION_DONE);
-      } catch (IOException e) {
-        Assert.fail(e.getMessage());
-      }
-      return false;
-    }, 2000, 20000);
-  }
-
 }
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
index 37d3e71d6b..8d31ca9794 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/request/bucket/OMBucketCreateRequest.java
@@ -130,7 +130,7 @@ public class OMBucketCreateRequest extends OMClientRequest {
     newCreateBucketRequest.setBucketInfo(newBucketInfo.build());
 
     return getOmRequest().toBuilder().setUserInfo(getUserInfo())
-       .setCreateBucketRequest(newCreateBucketRequest.build()).build();
+        .setCreateBucketRequest(newCreateBucketRequest.build()).build();
   }
 
   @Override
@@ -271,6 +271,7 @@ public class OMBucketCreateRequest extends OMClientRequest {
   /**
    * Add default acls for bucket. These acls are inherited from volume
    * default acl list.
+   *
    * @param omBucketInfo
    * @param omVolumeArgs
    */
@@ -393,4 +394,28 @@ public class OMBucketCreateRequest extends OMClientRequest {
     }
     return req;
   }
-}
+
+  @RequestFeatureValidator(
+      conditions = ValidationCondition.CLUSTER_NEEDS_FINALIZATION,
+      processingPhase = RequestProcessingPhase.PRE_PROCESS,
+      requestType = Type.CreateBucket
+  )
+  public static OMRequest disallowCreateBucketWithBucketLayoutDuringPreFinalize(
+      OMRequest req, ValidationContext ctx) throws OMException {
+    if (!ctx.versionManager()
+        .isAllowed(OMLayoutFeature.BUCKET_LAYOUT_SUPPORT)) {
+      if (req.getCreateBucketRequest()
+          .getBucketInfo().hasBucketLayout()
+          &&
+          !BucketLayout.fromProto(req.getCreateBucketRequest().getBucketInfo()
+              .getBucketLayout()).isLegacy()) {
+        throw new OMException("Cluster does not have the Bucket Layout"
+            + " support feature finalized yet, but the request contains"
+            + " a non LEGACY bucket type. Rejecting the request,"
+            + " please finalize the cluster upgrade and then try again.",
+            OMException.ResultCodes.NOT_SUPPORTED_OPERATION_PRIOR_FINALIZATION);
+      }
+    }
+    return req;
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
index aa639b2976..7b1f4185b6 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeature.java
@@ -32,7 +32,11 @@ public enum OMLayoutFeature implements LayoutFeature {
   INITIAL_VERSION(0, "Initial Layout Version"),
 
   ERASURE_CODED_STORAGE_SUPPORT(1, "Ozone version with built in support for"
-      + " Erasure Coded block data storage.");
+      + " Erasure Coded block data storage."),
+
+  BUCKET_LAYOUT_SUPPORT(2, "Ozone version supporting bucket " +
+      "layouts and introducing the FILE_SYSTEM_OPTIMIZED and OBJECT_STORE " +
+      "bucket layout types.");
 
 
   ///////////////////////////////  /////////////////////////////


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