You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ozone.apache.org by av...@apache.org on 2020/08/26 00:26:58 UTC

[hadoop-ozone] branch HDDS-3698-upgrade updated: HDDS-3829. Introduce Layout Feature interface in Ozone. (#1322)

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

avijayan pushed a commit to branch HDDS-3698-upgrade
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git


The following commit(s) were added to refs/heads/HDDS-3698-upgrade by this push:
     new 7bbd480  HDDS-3829. Introduce Layout Feature interface in Ozone. (#1322)
7bbd480 is described below

commit 7bbd4805423933064c11b39dc91b5abfdd9c6c7a
Author: avijayanhwx <14...@users.noreply.github.com>
AuthorDate: Tue Aug 25 17:26:42 2020 -0700

    HDDS-3829. Introduce Layout Feature interface in Ozone. (#1322)
---
 .../upgrade/AbstractLayoutVersionManager.java      | 110 +++++++++++++++++++++
 .../apache/hadoop/ozone/upgrade/LayoutFeature.java |  33 ++++++-
 .../hadoop/ozone/upgrade/LayoutVersionManager.java |  64 ++++++++++++
 .../apache/hadoop/ozone/upgrade}/package-info.java |  12 ++-
 .../upgrade/TestAbstractLayoutVersionManager.java  |  85 ++++++++++++++++
 hadoop-ozone/ozone-manager/pom.xml                 |  31 ++++++
 .../org/apache/hadoop/ozone/om/OzoneManager.java   |   4 +
 .../om/response/s3/security/package-info.java      |   2 +-
 .../NewOmFeatureUpgradeAction.java}                |  20 +++-
 .../OMLayoutFeatureAPI.java}                       |  23 ++++-
 .../ozone/om/upgrade/OMLayoutFeatureAspect.java    |  54 ++++++++++
 .../ozone/om/upgrade/OMLayoutFeatureCatalog.java   |  93 +++++++++++++++++
 .../ozone/om/upgrade/OMLayoutVersionManager.java   |  92 +++++++++++++++++
 .../OmUpgradeAction.java}                          |  16 ++-
 .../s3/security => upgrade}/package-info.java      |   7 +-
 .../request/volume/TestOMVolumeDeleteRequest.java  |   2 +-
 .../om/upgrade/TestOMLayoutFeatureAspect.java      |  71 +++++++++++++
 .../ozone/om/upgrade/TestOMVersionManager.java     |  66 +++++++++++++
 18 files changed, 755 insertions(+), 30 deletions(-)

diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java
new file mode 100644
index 0000000..99f72c8
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/AbstractLayoutVersionManager.java
@@ -0,0 +1,110 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.upgrade;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Optional;
+import java.util.TreeMap;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Layout Version Manager containing generic method implementations.
+ */
+@SuppressWarnings("visibilitymodifier")
+public abstract class AbstractLayoutVersionManager implements
+    LayoutVersionManager {
+
+  protected int metadataLayoutVersion; // MLV.
+  protected int softwareLayoutVersion; // SLV.
+  protected TreeMap<Integer, LayoutFeature> features = new TreeMap<>();
+  protected Map<String, LayoutFeature> featureMap = new HashMap<>();
+  protected volatile boolean isInitialized = false;
+
+  protected void init(int version, LayoutFeature[] lfs) {
+    if (!isInitialized) {
+      metadataLayoutVersion = version;
+      initializeFeatures(lfs);
+      softwareLayoutVersion = features.lastKey();
+      isInitialized = true;
+    }
+  }
+
+  protected void initializeFeatures(LayoutFeature[] lfs) {
+    Arrays.stream(lfs).forEach(f -> {
+      Preconditions.checkArgument(!featureMap.containsKey(f.name()));
+      Preconditions.checkArgument(!features.containsKey(f.layoutVersion()));
+      features.put(f.layoutVersion(), f);
+      featureMap.put(f.name(), f);
+    });
+  }
+
+  public int getMetadataLayoutVersion() {
+    return metadataLayoutVersion;
+  }
+
+  public int getSoftwareLayoutVersion() {
+    return softwareLayoutVersion;
+  }
+
+  public boolean needsFinalization() {
+    return metadataLayoutVersion < softwareLayoutVersion;
+  }
+
+  public boolean isAllowed(LayoutFeature layoutFeature) {
+    return layoutFeature.layoutVersion() <= metadataLayoutVersion;
+  }
+
+  public boolean isAllowed(String featureName) {
+    return featureMap.containsKey(featureName) &&
+        isAllowed(featureMap.get(featureName));
+  }
+
+  public LayoutFeature getFeature(String name) {
+    return featureMap.get(name);
+  }
+
+  public void doFinalize(Object param) {
+    if (needsFinalization()){
+      Iterator<Map.Entry<Integer, LayoutFeature>> iterator = features
+          .tailMap(metadataLayoutVersion + 1).entrySet().iterator();
+      while (iterator.hasNext()) {
+        Map.Entry<Integer, LayoutFeature> f = iterator.next();
+        Optional<? extends LayoutFeature.UpgradeAction> upgradeAction =
+            f.getValue().onFinalizeAction();
+        upgradeAction.ifPresent(action -> action.executeAction(param));
+        // ToDo : Handle shutdown while iterating case (resume from last
+        //  feature).
+        metadataLayoutVersion = f.getKey();
+      }
+      // ToDo : Persist new MLV.
+    }
+  }
+
+  protected void reset() {
+    metadataLayoutVersion = 0;
+    softwareLayoutVersion = 0;
+    featureMap.clear();
+    features.clear();
+    isInitialized = false;
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutFeature.java
similarity index 55%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutFeature.java
index d9024d1..05e944e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutFeature.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -16,7 +16,30 @@
  * limitations under the License.
  */
 
+package org.apache.hadoop.ozone.upgrade;
+
+import java.util.Optional;
+
 /**
- * Package contains classes related to S3 security responses.
+ * Generic Layout feature interface for Ozone.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+public interface LayoutFeature {
+  String name();
+
+  int layoutVersion();
+
+  String description();
+
+  default Optional<? extends UpgradeAction> onFinalizeAction() {
+    return Optional.empty();
+  }
+
+  /**
+   * Generic UpgradeAction interface. An operation that is run on specific
+   * upgrade states like post finalize, pre-downgrade etc.
+   * @param <T>
+   */
+  interface UpgradeAction<T> {
+    void executeAction(T arg);
+  }
+}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutVersionManager.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutVersionManager.java
new file mode 100644
index 0000000..432bd52
--- /dev/null
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/LayoutVersionManager.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.upgrade;
+
+/**
+ * Read Only interface to an Ozone component's Version Manager.
+ */
+public interface LayoutVersionManager {
+
+  /**
+   * Get the Current Metadata Layout Version.
+   * @return MLV
+   */
+  int getMetadataLayoutVersion();
+
+  /**
+   * Get the Current Software Layout Version.
+   * @return SLV
+   */
+  int getSoftwareLayoutVersion();
+
+  /**
+   * Does it need finalization?
+   * @return true/false
+   */
+  boolean needsFinalization();
+
+  /**
+   * Is allowed feature?
+   * @param layoutFeature feature object
+   * @return true/false.
+   */
+  boolean isAllowed(LayoutFeature layoutFeature);
+
+  /**
+   * Is allowed feature?
+   * @param featureName feature name
+   * @return true/false.
+   */
+  boolean isAllowed(String featureName);
+
+  /**
+   * Get Feature given feature name.
+   * @param name Feature name.
+   * @return LayoutFeature instance.
+   */
+  LayoutFeature getFeature(String name);
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/package-info.java
similarity index 80%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/package-info.java
index d9024d1..cf992d8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/upgrade/package-info.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -17,6 +17,8 @@
  */
 
 /**
- * Package contains classes related to S3 security responses.
+ * This package contains classes for the Ozone upgrade and layout version
+ * management.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+package org.apache.hadoop.ozone.upgrade;
+
diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/upgrade/TestAbstractLayoutVersionManager.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/upgrade/TestAbstractLayoutVersionManager.java
new file mode 100644
index 0000000..44fa100
--- /dev/null
+++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/upgrade/TestAbstractLayoutVersionManager.java
@@ -0,0 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.upgrade;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test generic layout management init and APIs.
+ */
+public class TestAbstractLayoutVersionManager {
+
+  private AbstractLayoutVersionManager versionManager =
+      new MockVersionManager();
+
+  @Before
+  public void setUp() {
+    versionManager.reset();
+  }
+
+  @Test
+  public void testInit() {
+    versionManager.init(1,
+        getTestLayoutFeatures(2));
+    assertEquals(2, versionManager.features.size());
+    assertEquals(2, versionManager.featureMap.size());
+    assertEquals(1, versionManager.getMetadataLayoutVersion());
+    assertEquals(2, versionManager.getSoftwareLayoutVersion());
+    assertTrue(versionManager.needsFinalization());
+  }
+
+  @Test
+  public void testNeedsFinalization() {
+    versionManager.init(2, getTestLayoutFeatures(2));
+    assertFalse(versionManager.needsFinalization());
+  }
+
+  private LayoutFeature[] getTestLayoutFeatures(int num) {
+    LayoutFeature[] lfs = new LayoutFeature[num];
+    int k = 0;
+    for (int i = 1; i <= num; i++) {
+      int finalI = i;
+      lfs[k++] = new LayoutFeature() {
+        @Override
+        public String name() {
+          return "LF-" + finalI;
+        }
+
+        @Override
+        public int layoutVersion() {
+          return finalI;
+        }
+
+        @Override
+        public String description() {
+          return null;
+        }
+      };
+    }
+    return lfs;
+  }
+
+  static class MockVersionManager extends AbstractLayoutVersionManager {
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/pom.xml b/hadoop-ozone/ozone-manager/pom.xml
index 056498a..f1bbd97 100644
--- a/hadoop-ozone/ozone-manager/pom.xml
+++ b/hadoop-ozone/ozone-manager/pom.xml
@@ -31,6 +31,18 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
   <dependencies>
 
     <dependency>
+      <groupId>org.aspectj</groupId>
+      <artifactId>aspectjrt</artifactId>
+      <version>1.8.9</version>
+    </dependency>
+
+    <dependency>
+      <groupId>org.aspectj</groupId>
+      <artifactId>aspectjweaver</artifactId>
+      <version>1.8.9</version>
+    </dependency>
+
+    <dependency>
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-ozone-common</artifactId>
     </dependency>
@@ -145,6 +157,25 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd">
           </execution>
         </executions>
       </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>aspectj-maven-plugin</artifactId>
+        <version>1.10</version>
+        <configuration>
+          <source>1.8</source>
+          <target>1.8</target>
+        </configuration>
+        <executions>
+          <execution>
+            <goals>
+              <goal>compile</goal>
+            </goals>
+            <configuration>
+              <complianceLevel>1.8</complianceLevel>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
     </plugins>
     <testResources>
       <testResource>
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
index 13e47a7..6f368e1 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
@@ -140,6 +140,7 @@ import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
 import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerRatisUtils;
 import org.apache.hadoop.ozone.om.request.file.OMFileRequest;
 import org.apache.hadoop.ozone.om.snapshot.OzoneManagerSnapshotProvider;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutVersionManager;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
@@ -1140,6 +1141,9 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     metadataManager.start(configuration);
     startSecretManagerIfNecessary();
 
+    OMLayoutVersionManager omVersionManager =
+        OMLayoutVersionManager.initialize(omStorage);
+
     if (certClient != null) {
       caCertPem = CertificateCodec.getPEMEncodedString(
           certClient.getCACertificate());
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
index d9024d1..eb86413 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
@@ -19,4 +19,4 @@
 /**
  * Package contains classes related to S3 security responses.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+package org.apache.hadoop.ozone.om.response.s3.security;
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/NewOmFeatureUpgradeAction.java
similarity index 66%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/NewOmFeatureUpgradeAction.java
index d9024d1..bb4eb7c 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/NewOmFeatureUpgradeAction.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -16,7 +16,17 @@
  * limitations under the License.
  */
 
+package org.apache.hadoop.ozone.om.upgrade;
+
+import org.apache.hadoop.ozone.om.OzoneManager;
+
 /**
- * Package contains classes related to S3 security responses.
+ * Stub OM Action class to help with understanding. Will be removed.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+public class NewOmFeatureUpgradeAction implements OmUpgradeAction {
+
+  @Override
+  public void executeAction(OzoneManager ozoneManager) {
+    // Do blah....
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAPI.java
similarity index 58%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAPI.java
index d9024d1..2da8b38 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAPI.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -16,7 +16,20 @@
  * limitations under the License.
  */
 
+package org.apache.hadoop.ozone.om.upgrade;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureCatalog.OMLayoutFeature;
+
 /**
- * Package contains classes related to S3 security responses.
+ * Annotation to specify if an API is backed up by a Layout Feature.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+@Target(ElementType.METHOD)
+@Retention(RetentionPolicy.RUNTIME)
+public @interface OMLayoutFeatureAPI {
+  OMLayoutFeature value();
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
new file mode 100644
index 0000000..a92e3b4
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureAspect.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.upgrade;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION;
+
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+import org.aspectj.lang.JoinPoint;
+import org.aspectj.lang.annotation.Aspect;
+import org.aspectj.lang.annotation.Before;
+import org.aspectj.lang.reflect.MethodSignature;
+
+/**
+ * 'Aspect' for OM Layout Feature API. All methods annotated with the
+ * specific annotation will have pre-processing done here to check layout
+ * version compatibility.
+ */
+@Aspect
+public class OMLayoutFeatureAspect {
+
+  @Before("@annotation(OMLayoutFeatureAPI) && execution(* *(..))")
+  public void checkLayoutFeature(JoinPoint joinPoint) throws Throwable {
+    String featureName = ((MethodSignature) joinPoint.getSignature())
+        .getMethod().getAnnotation(OMLayoutFeatureAPI.class).value().name();
+    LayoutVersionManager lvm = OMLayoutVersionManager.getInstance();
+    if (!lvm.isAllowed(featureName)) {
+      LayoutFeature layoutFeature = lvm.getFeature(featureName);
+      throw new OMException(String.format("Operation %s cannot be invoked " +
+          "before finalization. Current layout version = %d, feature's layout" +
+              " version = %d",
+          featureName,
+          lvm.getMetadataLayoutVersion(),
+          layoutFeature.layoutVersion()), NOT_SUPPORTED_OPERATION);
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureCatalog.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureCatalog.java
new file mode 100644
index 0000000..c5ed27d
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutFeatureCatalog.java
@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.upgrade;
+
+import java.util.Optional;
+
+import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+
+/**
+ * Catalog of Ozone Manager features.
+ */
+public class OMLayoutFeatureCatalog {
+
+  /**
+   * List of OM Features.
+   */
+  public enum OMLayoutFeature implements LayoutFeature {
+    INITIAL_VERSION(0, "Initial Layout Version"),
+    CREATE_EC(1, ""),
+    NEW_FEATURE(2, "new feature", new NewOmFeatureUpgradeAction());
+
+
+    private int layoutVersion;
+    private String description;
+    private Optional<OmUpgradeAction> omUpgradeAction = Optional.empty();
+
+    OMLayoutFeature(final int layoutVersion, String description) {
+      this.layoutVersion = layoutVersion;
+      this.description = description;
+    }
+
+    OMLayoutFeature(final int layoutVersion, String description,
+                    OmUpgradeAction upgradeAction) {
+      this.layoutVersion = layoutVersion;
+      this.description = description;
+      omUpgradeAction = Optional.of(upgradeAction);
+    }
+
+    @Override
+    public int layoutVersion() {
+      return layoutVersion;
+    }
+
+    @Override
+    public String description() {
+      return description;
+    }
+
+    @Override
+    public Optional<OmUpgradeAction> onFinalizeAction() {
+      return omUpgradeAction;
+    }
+  }
+
+  /**
+   * This is an example of an "API" that uses a new Layout feature (EC) that is
+   * not yet supported by the current layout version. The following can be
+   * "guarded" by just adding the following annotation, thereby keeping the
+   * method logic and upgrade logic separate.
+   */
+  @OMLayoutFeatureAPI(OMLayoutFeature.CREATE_EC)
+  public String ecMethod() {
+    // Blah Blah EC Blah....
+    return "ec";
+  }
+
+  /**
+   * This is an example of an "API" that uses a Layout feature (EC) that is
+   * supported by the current layout version.
+   */
+  @OMLayoutFeatureAPI(OMLayoutFeature.INITIAL_VERSION)
+  public String basicMethod() {
+    // Blah Blah Basic Blah....
+    return "basic";
+  }
+}
+
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutVersionManager.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutVersionManager.java
new file mode 100644
index 0000000..2f959a9
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OMLayoutVersionManager.java
@@ -0,0 +1,92 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.upgrade;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION;
+
+import org.apache.hadoop.ozone.common.Storage;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureCatalog.OMLayoutFeature;
+import org.apache.hadoop.ozone.upgrade.AbstractLayoutVersionManager;
+import org.apache.hadoop.ozone.upgrade.LayoutVersionManager;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Class to manage layout versions and features for Ozone Manager.
+ */
+public final class OMLayoutVersionManager extends AbstractLayoutVersionManager {
+
+  private static OMLayoutVersionManager omVersionManager;
+
+  private OMLayoutVersionManager() {
+  }
+
+  /**
+   * Read only instance to OM Version Manager.
+   * @return version manager instance.
+   */
+  public static synchronized LayoutVersionManager getInstance() {
+    if (omVersionManager == null) {
+      throw new RuntimeException("OM Layout Version Manager not yet " +
+          "initialized.");
+    }
+    return omVersionManager;
+  }
+
+
+  /**
+   * Initialize OM version manager from storage.
+   * @return version manager instance.
+   */
+  public static synchronized OMLayoutVersionManager initialize(
+      OMStorage omStorage)
+      throws OMException {
+    if (omVersionManager == null) {
+      omVersionManager = new OMLayoutVersionManager();
+      omVersionManager.init(omStorage);
+    }
+    return omVersionManager;
+  }
+
+  /**
+   * Initialize the OM Layout Features and current Layout Version.
+   * @param storage to read the current layout version.
+   * @throws OMException on error.
+   */
+  private void init(Storage storage) throws OMException {
+    init(storage.getLayoutVersion(), OMLayoutFeature.values());
+    if (metadataLayoutVersion > softwareLayoutVersion) {
+      throw new OMException(
+          String.format("Cannot initialize VersionManager. Metadata " +
+                  "layout version (%d) > software layout version (%d)",
+              metadataLayoutVersion, softwareLayoutVersion),
+          NOT_SUPPORTED_OPERATION);
+    }
+  }
+
+  @VisibleForTesting
+  protected synchronized static void resetLayoutVersionManager() {
+    if (omVersionManager != null) {
+      omVersionManager.reset();
+      omVersionManager = null;
+    }
+  }
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OmUpgradeAction.java
similarity index 67%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OmUpgradeAction.java
index d9024d1..da9a063 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/OmUpgradeAction.java
@@ -6,9 +6,9 @@
  * 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>
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS,
  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
@@ -16,7 +16,13 @@
  * limitations under the License.
  */
 
+package org.apache.hadoop.ozone.om.upgrade;
+
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.upgrade.LayoutFeature.UpgradeAction;
+
 /**
- * Package contains classes related to S3 security responses.
+ * Upgrade Action for OzoneManager which takes in an 'OM' instance.
  */
-package org.apache.hadoop.ozone.om.request.s3.security;
+public interface OmUpgradeAction extends UpgradeAction<OzoneManager> {
+}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/package-info.java
similarity index 86%
copy from hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
copy to hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/package-info.java
index d9024d1..d663049 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/response/s3/security/package-info.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/upgrade/package-info.java
@@ -16,7 +16,8 @@
  * limitations under the License.
  */
 
+package org.apache.hadoop.ozone.om.upgrade;
+
 /**
- * Package contains classes related to S3 security responses.
- */
-package org.apache.hadoop.ozone.om.request.s3.security;
+ * This package contains OM  Upgrade related classes.
+ */
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeDeleteRequest.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeDeleteRequest.java
index 49f28d3..73a2888 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeDeleteRequest.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/request/volume/TestOMVolumeDeleteRequest.java
@@ -20,7 +20,7 @@ package org.apache.hadoop.ozone.om.request.volume;
 
 import java.util.UUID;
 
-import org.junit.Assert;;
+import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMLayoutFeatureAspect.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMLayoutFeatureAspect.java
new file mode 100644
index 0000000..b68c7c2
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMLayoutFeatureAspect.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <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.upgrade;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.NOT_SUPPORTED_OPERATION;
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Class to test annotation based interceptor that checks whether layout
+ * feature API is allowed.
+ */
+public class TestOMLayoutFeatureAspect {
+
+  @Rule
+  public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
+  private OzoneConfiguration configuration = new OzoneConfiguration();
+
+  @Before
+  public void setUp() throws IOException {
+    configuration.set("ozone.metadata.dirs",
+        temporaryFolder.newFolder().getAbsolutePath());
+  }
+
+  /**
+   * This unit test invokes the above 2 layout feature APIs. The first one
+   * should fail, and the second one should pass.
+   * @throws Exception
+   */
+  @Test
+  public void testCheckLayoutFeature() throws Exception {
+    OMLayoutVersionManager.initialize(new OMStorage(configuration));
+    OMLayoutFeatureCatalog testObj = new OMLayoutFeatureCatalog();
+    try {
+      testObj.ecMethod();
+      Assert.fail();
+    } catch (Exception ex) {
+      OMException omEx = (OMException) ex;
+      assertEquals(NOT_SUPPORTED_OPERATION, omEx.getResult());
+    }
+    String s = testObj.basicMethod();
+    assertEquals("basic", s);
+  }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMVersionManager.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMVersionManager.java
new file mode 100644
index 0000000..cfcfe24
--- /dev/null
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/upgrade/TestOMVersionManager.java
@@ -0,0 +1,66 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.ozone.om.upgrade;
+
+import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureCatalog.OMLayoutFeature.CREATE_EC;
+import static org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureCatalog.OMLayoutFeature.INITIAL_VERSION;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+
+import org.apache.hadoop.ozone.om.OMStorage;
+import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.upgrade.OMLayoutFeatureCatalog.OMLayoutFeature;
+import org.apache.hadoop.ozone.upgrade.LayoutFeature;
+import org.junit.Test;
+
+/**
+ * Test OM layout version management.
+ */
+public class TestOMVersionManager {
+
+  @Test
+  public void testOMLayoutVersionManager() throws IOException {
+    OMStorage omStorage = mock(OMStorage.class);
+    when(omStorage.getLayoutVersion()).thenReturn(0);
+    OMLayoutVersionManager omVersionManager =
+        OMLayoutVersionManager.initialize(omStorage);
+    assertTrue(omVersionManager.isAllowed(INITIAL_VERSION));
+    assertFalse(omVersionManager.isAllowed(CREATE_EC));
+    assertEquals(0, omVersionManager.getMetadataLayoutVersion());
+    assertTrue(omVersionManager.needsFinalization());
+    omVersionManager.doFinalize(mock(OzoneManager.class));
+    assertFalse(omVersionManager.needsFinalization());
+    assertEquals(2, omVersionManager.getMetadataLayoutVersion());
+  }
+
+  @Test
+  public void testOMLayoutFeatureCatalog() {
+    OMLayoutFeature[] values = OMLayoutFeature.values();
+    int currVersion = Integer.MIN_VALUE;
+    for (LayoutFeature lf : values) {
+      assertTrue(currVersion <= lf.layoutVersion());
+      currVersion = lf.layoutVersion();
+    }
+  }
+}
\ No newline at end of file


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