Browse Source

HDFS-5754. Split LayoutVerion into NameNodeLayoutVersion and DataNodeLayoutVersion. Contributed by Brandon Li

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1563041 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
00067895a0
43 changed files with 717 additions and 363 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-5535.txt
  2. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java
  3. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java
  4. 16 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
  5. 83 52
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  6. 10 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  7. 7 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java
  8. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java
  9. 4 32
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java
  10. 48 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java
  11. 5 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java
  12. 12 17
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java
  13. 97 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeLayoutVersion.java
  14. 28 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java
  15. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java
  16. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java
  17. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java
  18. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java
  19. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  20. 66 36
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
  21. 12 13
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  22. 46 28
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  23. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
  24. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java
  25. 11 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java
  26. 98 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  27. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  28. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  29. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java
  30. 4 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java
  31. 35 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java
  32. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
  33. 13 9
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java
  34. 4 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java
  35. 9 6
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java
  36. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java
  37. 5 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java
  38. 38 15
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
  39. 9 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
  40. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java
  41. 6 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java
  42. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
  43. 5 5
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-5535.txt

@@ -15,3 +15,6 @@ HDFS-5535 subtasks:
 
     HDFS-5835. Add a new option for starting Namenode when rolling upgrade is
     in progress. (szetszwo)
+
+    HDFS-5754. Split LayoutVerion into NameNodeLayoutVersion and
+    DataNodeLayoutVersion. (Brandon Li via szetszwo)

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/main/java/org/apache/hadoop/contrib/bkjournal/BookKeeperJournalManager.java

@@ -397,7 +397,7 @@ public class BookKeeperJournalManager implements JournalManager {
     try {
       String znodePath = inprogressZNode(txId);
       EditLogLedgerMetadata l = new EditLogLedgerMetadata(znodePath,
-          HdfsConstants.LAYOUT_VERSION, currentLedger.getId(), txId);
+          HdfsConstants.NAMENODE_LAYOUT_VERSION, currentLedger.getId(), txId);
       /* Write the ledger metadata out to the inprogress ledger znode
        * This can fail if for some reason our write lock has
        * expired (@see WriteLock) and another process has managed to

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/contrib/bkjournal/src/test/java/org/apache/hadoop/contrib/bkjournal/TestBookKeeperEditLogStreams.java

@@ -68,7 +68,7 @@ public class TestBookKeeperEditLogStreams {
       lh.close();
 
       EditLogLedgerMetadata metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
         new BookKeeperEditLogInputStream(lh, metadata, -1);
         fail("Shouldn't get this far, should have thrown");
@@ -77,7 +77,7 @@ public class TestBookKeeperEditLogStreams {
       }
 
       metadata = new EditLogLedgerMetadata("/foobar",
-          HdfsConstants.LAYOUT_VERSION, lh.getId(), 0x1234);
+          HdfsConstants.NAMENODE_LAYOUT_VERSION, lh.getId(), 0x1234);
       try {
         new BookKeeperEditLogInputStream(lh, metadata, 0);
         fail("Shouldn't get this far, should have thrown");

+ 16 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -25,6 +25,10 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 
 /************************************
  * Some handy constants
@@ -125,11 +129,19 @@ public class HdfsConstants {
 
 
   /**
-   * Please see {@link LayoutVersion} on adding new layout version.
+   * Current layout version for NameNode.
+   * Please see {@link NameNodeLayoutVersion.Feature} on adding new layout version.
    */
-  public static final int LAYOUT_VERSION = LayoutVersion
-      .getCurrentLayoutVersion();
-  
+  public static final int NAMENODE_LAYOUT_VERSION
+      = NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+
+  /**
+   * Current layout version for DataNode.
+   * Please see {@link DataNodeLayoutVersion.Feature} on adding new layout version.
+   */
+  public static final int DATANODE_LAYOUT_VERSION
+      = DataNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+
   /**
    * A special path component contained in the path for a snapshot file/dir
    */

+ 83 - 52
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -17,9 +17,10 @@
  */
 package org.apache.hadoop.hdfs.protocol;
 
-import java.util.EnumSet;
-import java.util.HashMap;
+import java.util.Comparator;
 import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
@@ -54,19 +55,27 @@ public class LayoutVersion {
   public static final int BUGFIX_HDFS_2991_VERSION = -40;
 
   /**
-   * Enums for features that change the layout version.
+   * The interface to be implemented by NameNode and DataNode layout features 
+   */
+  public interface LayoutFeature {
+    public FeatureInfo getInfo();
+  }
+
+  /**
+   * Enums for features that change the layout version before rolling
+   * upgrade is supported.
    * <br><br>
    * To add a new layout version:
    * <ul>
    * <li>Define a new enum constant with a short enum name, the new layout version 
    * and description of the added feature.</li>
    * <li>When adding a layout version with an ancestor that is not same as
-   * its immediate predecessor, use the constructor where a spacific ancestor
+   * its immediate predecessor, use the constructor where a specific ancestor
    * can be passed.
    * </li>
    * </ul>
    */
-  public static enum Feature {
+  public static enum Feature implements LayoutFeature {
     NAMESPACE_QUOTA(-16, "Support for namespace quotas"),
     FILE_ACCESS_TIME(-17, "Support for access time on files"),
     DISKSPACE_QUOTA(-18, "Support for disk space quotas"),
@@ -112,15 +121,10 @@ public class LayoutVersion {
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
         + " Use distinct StorageUuid per storage directory."),
     ADD_LAYOUT_FLAGS(-50, "Add support for layout flags."),
-    CACHING(-51, "Support for cache pools and path-based caching"),
-    ROLLING_UPGRADE_MARKER(-52, "Upgrade marker for rolling upgrade");
+    CACHING(-51, "Support for cache pools and path-based caching");
+
+    private final FeatureInfo info;
 
-    final int lv;
-    final int ancestorLV;
-    final String description;
-    final boolean reserved;
-    final Feature[] specialFeatures;
-    
     /**
      * Feature that is added at layout version {@code lv} - 1. 
      * @param lv new layout version with the addition of this feature
@@ -136,16 +140,35 @@ public class LayoutVersion {
      * @param ancestorLV layout version from which the new lv is derived from.
      * @param description description of the feature
      * @param reserved true when this is a layout version reserved for previous
-     *          verions
+     *        version
      * @param features set of features that are to be enabled for this version
      */
     Feature(final int lv, final int ancestorLV, final String description,
         boolean reserved, Feature... features) {
+      info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
+    }
+    
+    @Override
+    public FeatureInfo getInfo() {
+      return info;
+    }
+  }
+  
+  /** Feature information. */
+  public static class FeatureInfo {
+    private final int lv;
+    private final int ancestorLV;
+    private final String description;
+    private final boolean reserved;
+    private final LayoutFeature[] specialFeatures;
+
+    public FeatureInfo(final int lv, final int ancestorLV, final String description,
+        boolean reserved, LayoutFeature... specialFeatures) {
       this.lv = lv;
       this.ancestorLV = ancestorLV;
       this.description = description;
       this.reserved = reserved;
-      specialFeatures = features;
+      this.specialFeatures = specialFeatures;
     }
     
     /** 
@@ -175,80 +198,88 @@ public class LayoutVersion {
     public boolean isReservedForOldRelease() {
       return reserved;
     }
+    
+    public LayoutFeature[] getSpecialFeatures() {
+      return specialFeatures;
+    }
   }
-  
-  // Build layout version and corresponding feature matrix
-  static final Map<Integer, EnumSet<Feature>>map = 
-    new HashMap<Integer, EnumSet<Feature>>();
-  
-  // Static initialization 
-  static {
-    initMap();
+
+  static class LayoutFeatureComparator implements Comparator<LayoutFeature> {
+    @Override
+    public int compare(LayoutFeature arg0, LayoutFeature arg1) {
+      return arg0.getInfo().getLayoutVersion()
+          - arg1.getInfo().getLayoutVersion();
+    }
   }
-  
-  /**
-   * Initialize the map of a layout version and EnumSet of {@link Feature}s 
-   * supported.
-   */
-  private static void initMap() {
+ 
+  public static void updateMap(Map<Integer, SortedSet<LayoutFeature>> map,
+      LayoutFeature[] features) {
     // Go through all the enum constants and build a map of
-    // LayoutVersion <-> EnumSet of all supported features in that LayoutVersion
-    for (Feature f : Feature.values()) {
-      EnumSet<Feature> ancestorSet = map.get(f.ancestorLV);
+    // LayoutVersion <-> Set of all supported features in that LayoutVersion
+    for (LayoutFeature f : features) {
+      final FeatureInfo info = f.getInfo();
+      SortedSet<LayoutFeature> ancestorSet = map.get(info.getAncestorLayoutVersion());
       if (ancestorSet == null) {
-        ancestorSet = EnumSet.noneOf(Feature.class); // Empty enum set
-        map.put(f.ancestorLV, ancestorSet);
+        // Empty set
+        ancestorSet = new TreeSet<LayoutFeature>(new LayoutFeatureComparator());
+        map.put(info.getAncestorLayoutVersion(), ancestorSet);
       }
-      EnumSet<Feature> featureSet = EnumSet.copyOf(ancestorSet);
-      if (f.specialFeatures != null) {
-        for (Feature specialFeature : f.specialFeatures) {
+      SortedSet<LayoutFeature> featureSet = new TreeSet<LayoutFeature>(ancestorSet);
+      if (info.getSpecialFeatures() != null) {
+        for (LayoutFeature specialFeature : info.getSpecialFeatures()) {
           featureSet.add(specialFeature);
         }
       }
       featureSet.add(f);
-      map.put(f.lv, featureSet);
+      map.put(info.getLayoutVersion(), featureSet);
     }
   }
   
   /**
    * Gets formatted string that describes {@link LayoutVersion} information.
    */
-  public static String getString() {
+  public String getString(Map<Integer, SortedSet<LayoutFeature>> map,
+      LayoutFeature[] values) {
     final StringBuilder buf = new StringBuilder();
     buf.append("Feature List:\n");
-    for (Feature f : Feature.values()) {
+    for (LayoutFeature f : values) {
+      final FeatureInfo info = f.getInfo();
       buf.append(f).append(" introduced in layout version ")
-          .append(f.lv).append(" (").
-      append(f.description).append(")\n");
+          .append(info.getLayoutVersion()).append(" (")
+          .append(info.getDescription()).append(")\n");
     }
-    
+
     buf.append("\n\nLayoutVersion and supported features:\n");
-    for (Feature f : Feature.values()) {
-      buf.append(f.lv).append(": ").append(map.get(f.lv))
-          .append("\n");
+    for (LayoutFeature f : values) {
+      final FeatureInfo info = f.getInfo();
+      buf.append(info.getLayoutVersion()).append(": ")
+          .append(map.get(info.getLayoutVersion())).append("\n");
     }
     return buf.toString();
   }
   
   /**
    * Returns true if a given feature is supported in the given layout version
+   * @param map layout feature map
    * @param f Feature
    * @param lv LayoutVersion
    * @return true if {@code f} is supported in layout version {@code lv}
    */
-  public static boolean supports(final Feature f, final int lv) {
-    final EnumSet<Feature> set =  map.get(lv);
+  public static boolean supports(Map<Integer, SortedSet<LayoutFeature>> map,
+      final LayoutFeature f, final int lv) {
+    final SortedSet<LayoutFeature> set =  map.get(lv);
     return set != null && set.contains(f);
   }
   
   /**
    * Get the current layout version
    */
-  public static int getCurrentLayoutVersion() {
-    Feature[] values = Feature.values();
+  public static int getCurrentLayoutVersion(
+      Map<Integer, SortedSet<LayoutFeature>> map, LayoutFeature[] values) {
     for (int i = values.length -1; i >= 0; i--) {
-      if (!values[i].isReservedForOldRelease()) {
-        return values[i].lv;
+      final FeatureInfo info = values[i].getInfo();
+      if (!info.isReservedForOldRelease()) {
+        return info.getLayoutVersion();
       }
     }
     throw new AssertionError("All layout versions are reserved.");

+ 10 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -138,6 +138,7 @@ import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
@@ -232,9 +233,9 @@ public class PBHelper {
         .setNamespceID(info.getNamespaceID()).build();
   }
 
-  public static StorageInfo convert(StorageInfoProto info) {
+  public static StorageInfo convert(StorageInfoProto info, NodeType type) {
     return new StorageInfo(info.getLayoutVersion(), info.getNamespceID(),
-        info.getClusterID(), info.getCTime());
+        info.getClusterID(), info.getCTime(), type);
   }
 
   public static NamenodeRegistrationProto convert(NamenodeRegistration reg) {
@@ -245,8 +246,9 @@ public class PBHelper {
   }
 
   public static NamenodeRegistration convert(NamenodeRegistrationProto reg) {
+    StorageInfo si = convert(reg.getStorageInfo(), NodeType.NAME_NODE);
     return new NamenodeRegistration(reg.getRpcAddress(), reg.getHttpAddress(),
-        convert(reg.getStorageInfo()), convert(reg.getRole()));
+        si, convert(reg.getRole()));
   }
 
   // DatanodeId
@@ -378,9 +380,9 @@ public class PBHelper {
   }
 
   public static CheckpointSignature convert(CheckpointSignatureProto s) {
-    return new CheckpointSignature(PBHelper.convert(s.getStorageInfo()),
-        s.getBlockPoolId(), s.getMostRecentCheckpointTxId(),
-        s.getCurSegmentTxId());
+    StorageInfo si = PBHelper.convert(s.getStorageInfo(), NodeType.NAME_NODE);
+    return new CheckpointSignature(si, s.getBlockPoolId(),
+        s.getMostRecentCheckpointTxId(), s.getCurSegmentTxId());
   }
 
   public static RemoteEditLogProto convert(RemoteEditLog log) {
@@ -733,9 +735,9 @@ public class PBHelper {
   }
 
   public static DatanodeRegistration convert(DatanodeRegistrationProto proto) {
+    StorageInfo si = convert(proto.getStorageInfo(), NodeType.DATA_NODE);
     return new DatanodeRegistration(PBHelper.convert(proto.getDatanodeID()),
-        PBHelper.convert(proto.getStorageInfo()), PBHelper.convert(proto
-            .getKeys()), proto.getSoftwareVersion());
+        si, PBHelper.convert(proto.getKeys()), proto.getSoftwareVersion());
   }
 
   public static DatanodeCommand convert(DatanodeCommandProto proto) {

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/protocolPB/QJournalProtocolServerSideTranslatorPB.java

@@ -64,6 +64,8 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PurgeLogs
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentRequestProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.StartLogSegmentResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.RequestInfo;
+import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.protocol.JournalProtocol;
 
 import com.google.protobuf.RpcController;
@@ -274,9 +276,9 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
   @Override
   public DoUpgradeResponseProto doUpgrade(RpcController controller,
       DoUpgradeRequestProto request) throws ServiceException {
+    StorageInfo si = PBHelper.convert(request.getSInfo(), NodeType.NAME_NODE);
     try {
-      impl.doUpgrade(convert(request.getJid()),
-          PBHelper.convert(request.getSInfo()));
+      impl.doUpgrade(convert(request.getJid()), si);
       return DoUpgradeResponseProto.getDefaultInstance();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -298,9 +300,9 @@ public class QJournalProtocolServerSideTranslatorPB implements QJournalProtocolP
   public CanRollBackResponseProto canRollBack(RpcController controller,
       CanRollBackRequestProto request) throws ServiceException {
     try {
-      Boolean result = impl.canRollBack(convert(request.getJid()),
-          PBHelper.convert(request.getStorage()),
-          PBHelper.convert(request.getPrevStorage()),
+      StorageInfo si = PBHelper.convert(request.getStorage(), NodeType.NAME_NODE);
+      Boolean result = impl.canRollBack(convert(request.getJid()), si,
+          PBHelper.convert(request.getPrevStorage(), NodeType.NAME_NODE),
           request.getTargetLayoutVersion());
       return CanRollBackResponseProto.newBuilder()
           .setCanRollBack(result)

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java

@@ -45,8 +45,9 @@ public class IncorrectVersionException extends IOException {
         minimumVersion + "'");
   }
   
-  public IncorrectVersionException(int versionReported, String ofWhat) {
-    this(versionReported, ofWhat, HdfsConstants.LAYOUT_VERSION);
+  public IncorrectVersionException(int currentLayoutVersion,
+      int versionReported, String ofWhat) {
+    this(versionReported, ofWhat, currentLayoutVersion);
   }
   
   public IncorrectVersionException(int versionReported,

+ 4 - 32
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -34,7 +34,6 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.util.ToolRunner;
@@ -116,7 +115,6 @@ public abstract class Storage extends StorageInfo {
     public boolean isOfType(StorageDirType type);
   }
   
-  protected NodeType storageType;    // Type of the node using this storage 
   protected List<StorageDirectory> storageDirs = new ArrayList<StorageDirectory>();
   
   private class DirIterator implements Iterator<StorageDirectory> {
@@ -781,13 +779,11 @@ public abstract class Storage extends StorageInfo {
    * Create empty storage info of the specified type
    */
   protected Storage(NodeType type) {
-    super();
-    this.storageType = type;
+    super(type);
   }
   
-  protected Storage(NodeType type, StorageInfo storageInfo) {
+  protected Storage(StorageInfo storageInfo) {
     super(storageInfo);
-    this.storageType = type;
   }
   
   public int getNumStorageDirs() {
@@ -931,35 +927,11 @@ public abstract class Storage extends StorageInfo {
     props.setProperty("storageType", storageType.toString());
     props.setProperty("namespaceID", String.valueOf(namespaceID));
     // Set clusterID in version with federation support
-    if (versionSupportsFederation()) {
+    if (versionSupportsFederation(getServiceLayoutFeatureMap())) {
       props.setProperty("clusterID", clusterID);
     }
     props.setProperty("cTime", String.valueOf(cTime));
   }
-  
-  /**
-   * Get common storage fields.
-   * Should be overloaded if additional fields need to be get.
-   * 
-   * @param props
-   * @throws IOException
-   */
-  protected void setFieldsFromProperties(
-      Properties props, StorageDirectory sd) throws IOException {
-    super.setFieldsFromProperties(props, sd);
-    setStorageType(props, sd);
-  }
-  
-  /** Validate and set storage type from {@link Properties}*/
-  protected void setStorageType(Properties props, StorageDirectory sd)
-      throws InconsistentFSStateException {
-    NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
-    if (!storageType.equals(type)) {
-      throw new InconsistentFSStateException(sd.root,
-          "node type is incompatible with others.");
-    }
-    storageType = type;
-  }
 
   /**
    * Write properties to the VERSION file in the given storage directory.
@@ -1025,7 +997,7 @@ public abstract class Storage extends StorageInfo {
    * @throws IOException
    */
   public void writeAll() throws IOException {
-    this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    this.layoutVersion = getServiceLayoutVersion();
     for (Iterator<StorageDirectory> it = storageDirs.iterator(); it.hasNext();) {
       writeProperties(it.next());
     }

+ 48 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/StorageInfo.java

@@ -21,15 +21,22 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.util.Map;
 import java.util.Properties;
+import java.util.SortedSet;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 
 import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
 
 /**
  * Common class for storage information.
@@ -42,22 +49,26 @@ public class StorageInfo {
   public int   namespaceID;     // id of the file system
   public String clusterID;      // id of the cluster
   public long  cTime;           // creation time of the file system state
+
+  protected final NodeType storageType; // Type of the node using this storage 
   
   protected static final String STORAGE_FILE_VERSION    = "VERSION";
- 
-  public StorageInfo () {
-    this(0, 0, "", 0L);
+
+  public StorageInfo(NodeType type) {
+    this(0, 0, "", 0L, type);
   }
 
-  public StorageInfo(int layoutV, int nsID, String cid, long cT) {
+  public StorageInfo(int layoutV, int nsID, String cid, long cT, NodeType type) {
     layoutVersion = layoutV;
     clusterID = cid;
     namespaceID = nsID;
     cTime = cT;
+    storageType = type;
   }
   
   public StorageInfo(StorageInfo from) {
-    setStorageInfo(from);
+    this(from.layoutVersion, from.namespaceID, from.clusterID, from.cTime,
+        from.storageType);
   }
 
   /**
@@ -84,14 +95,17 @@ public class StorageInfo {
   public long   getCTime()        { return cTime; }
   
   public void   setStorageInfo(StorageInfo from) {
+    Preconditions.checkArgument(from.storageType == storageType);
     layoutVersion = from.layoutVersion;
     clusterID = from.clusterID;
     namespaceID = from.namespaceID;
     cTime = from.cTime;
   }
 
-  public boolean versionSupportsFederation() {
-    return LayoutVersion.supports(Feature.FEDERATION, layoutVersion);
+  public boolean versionSupportsFederation(
+      Map<Integer, SortedSet<LayoutFeature>> map) {
+    return LayoutVersion.supports(map, LayoutVersion.Feature.FEDERATION,
+        layoutVersion);
   }
   
   @Override
@@ -145,6 +159,17 @@ public class StorageInfo {
     setNamespaceID(props, sd);
     setcTime(props, sd);
     setClusterId(props, layoutVersion, sd);
+    checkStorageType(props, sd);
+  }
+  
+  /** Validate and set storage type from {@link Properties}*/
+  protected void checkStorageType(Properties props, StorageDirectory sd)
+      throws InconsistentFSStateException {
+    NodeType type = NodeType.valueOf(getProperty(props, sd, "storageType"));
+    if (!storageType.equals(type)) {
+      throw new InconsistentFSStateException(sd.root,
+          "node type is incompatible with others.");
+    }
   }
   
   /** Validate and set ctime from {@link Properties}*/
@@ -157,7 +182,8 @@ public class StorageInfo {
   protected void setClusterId(Properties props, int layoutVersion,
       StorageDirectory sd) throws InconsistentFSStateException {
     // Set cluster ID in version that supports federation
-    if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+    if (LayoutVersion.supports(getServiceLayoutFeatureMap(),
+        Feature.FEDERATION, layoutVersion)) {
       String cid = getProperty(props, sd, "clusterID");
       if (!(clusterID.equals("") || cid.equals("") || clusterID.equals(cid))) {
         throw new InconsistentFSStateException(sd.getRoot(),
@@ -171,9 +197,9 @@ public class StorageInfo {
   protected void setLayoutVersion(Properties props, StorageDirectory sd)
       throws IncorrectVersionException, InconsistentFSStateException {
     int lv = Integer.parseInt(getProperty(props, sd, "layoutVersion"));
-    if (lv < HdfsConstants.LAYOUT_VERSION) { // future version
-      throw new IncorrectVersionException(lv, "storage directory "
-          + sd.root.getAbsolutePath());
+    if (lv < getServiceLayoutVersion()) { // future version
+      throw new IncorrectVersionException(getServiceLayoutVersion(), lv,
+          "storage directory " + sd.root.getAbsolutePath());
     }
     layoutVersion = lv;
   }
@@ -189,6 +215,16 @@ public class StorageInfo {
     namespaceID = nsId;
   }
   
+  public int getServiceLayoutVersion() {
+    return storageType == NodeType.DATA_NODE ? HdfsConstants.DATANODE_LAYOUT_VERSION
+        : HdfsConstants.NAMENODE_LAYOUT_VERSION;
+  }
+
+  public Map<Integer, SortedSet<LayoutFeature>> getServiceLayoutFeatureMap() {
+    return storageType == NodeType.DATA_NODE? DataNodeLayoutVersion.FEATURES
+        : NameNodeLayoutVersion.FEATURES;
+  }
+  
   static String getProperty(Properties props, StorageDirectory sd,
       String name) throws InconsistentFSStateException {
     String property = props.getProperty(name);
@@ -198,7 +234,7 @@ public class StorageInfo {
     }
     return property;
   }
-  
+
   public static Properties readPropertiesFile(File from) throws IOException {
     RandomAccessFile file = new RandomAccessFile(from, "rws");
     FileInputStream in = null;

+ 5 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java

@@ -203,13 +203,11 @@ class BPServiceActor implements Runnable {
           "DataNode version '" + dnVersion + "' but is within acceptable " +
           "limits. Note: This is normal during a rolling upgrade.");
     }
-
-    if (HdfsConstants.LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
-      LOG.warn("DataNode and NameNode layout versions must be the same." +
-        " Expected: "+ HdfsConstants.LAYOUT_VERSION +
-        " actual "+ nsInfo.getLayoutVersion());
-      throw new IncorrectVersionException(
-          nsInfo.getLayoutVersion(), "namenode");
+    
+    if (HdfsConstants.DATANODE_LAYOUT_VERSION != nsInfo.getLayoutVersion()) {
+      LOG.info("DataNode and NameNode layout versions are different:" +
+        " DataNode version: "+ HdfsConstants.DATANODE_LAYOUT_VERSION +
+        " NameNode version: "+ nsInfo.getLayoutVersion());
     }
   }
 

+ 12 - 17
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java

@@ -32,7 +32,6 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.HardLink;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -65,7 +64,7 @@ public class BlockPoolSliceStorage extends Storage {
   private String blockpoolID = ""; // id of the blockpool
 
   public BlockPoolSliceStorage(StorageInfo storageInfo, String bpid) {
-    super(NodeType.DATA_NODE, storageInfo);
+    super(storageInfo);
     blockpoolID = bpid;
   }
 
@@ -93,9 +92,6 @@ public class BlockPoolSliceStorage extends Storage {
    */
   void recoverTransitionRead(DataNode datanode, NamespaceInfo nsInfo,
       Collection<File> dataDirs, StartupOption startOpt) throws IOException {
-    assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() 
-        : "Block-pool and name-node layout versions must be the same.";
-
     // 1. For each BP data directory analyze the state and
     // check whether all is consistent before transitioning.
     this.storageDirs = new ArrayList<StorageDirectory>(dataDirs.size());
@@ -143,8 +139,6 @@ public class BlockPoolSliceStorage extends Storage {
     // while others could be up-to-date for the regular startup.
     for (int idx = 0; idx < getNumStorageDirs(); idx++) {
       doTransition(getStorageDir(idx), nsInfo, startOpt);
-      assert getLayoutVersion() == nsInfo.getLayoutVersion() 
-          : "Data-node and name-node layout versions must be the same.";
       assert getCTime() == nsInfo.getCTime() 
           : "Data-node and name-node CTimes must be the same.";
     }
@@ -175,11 +169,10 @@ public class BlockPoolSliceStorage extends Storage {
     LOG.info("Formatting block pool " + blockpoolID + " directory "
         + bpSdir.getCurrentDir());
     bpSdir.clearDirectory(); // create directory
-    this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
     this.cTime = nsInfo.getCTime();
     this.namespaceID = nsInfo.getNamespaceID();
     this.blockpoolID = nsInfo.getBlockPoolID();
-    this.storageType = NodeType.DATA_NODE;
     writeProperties(bpSdir);
   }
 
@@ -243,7 +236,7 @@ public class BlockPoolSliceStorage extends Storage {
     
     readProperties(sd);
     checkVersionUpgradable(this.layoutVersion);
-    assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION 
+    assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION 
        : "Future version is not allowed";
     if (getNamespaceID() != nsInfo.getNamespaceID()) {
       throw new IOException("Incompatible namespaceIDs in "
@@ -257,11 +250,11 @@ public class BlockPoolSliceStorage extends Storage {
           + nsInfo.getBlockPoolID() + "; datanode blockpoolID = "
           + blockpoolID);
     }
-    if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION
+    if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION
         && this.cTime == nsInfo.getCTime()) {
       return; // regular startup
     }
-    if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION
+    if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION
         || this.cTime < nsInfo.getCTime()) {
       doUpgrade(sd, nsInfo); // upgrade
       return;
@@ -294,7 +287,8 @@ public class BlockPoolSliceStorage extends Storage {
    */
   void doUpgrade(StorageDirectory bpSd, NamespaceInfo nsInfo) throws IOException {
     // Upgrading is applicable only to release with federation or after
-    if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+    if (!DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       return;
     }
     LOG.info("Upgrading block pool storage directory " + bpSd.getRoot()
@@ -327,7 +321,7 @@ public class BlockPoolSliceStorage extends Storage {
     
     // 3. Create new <SD>/current with block files hardlinks and VERSION
     linkAllBlocks(bpTmpDir, bpCurDir);
-    this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
     assert this.namespaceID == nsInfo.getNamespaceID() 
         : "Data-node and name-node layout versions must be the same.";
     this.cTime = nsInfo.getCTime();
@@ -349,7 +343,8 @@ public class BlockPoolSliceStorage extends Storage {
    * @throws IOException if the directory is not empty or it can not be removed
    */
   private void cleanupDetachDir(File detachDir) throws IOException {
-    if (!LayoutVersion.supports(Feature.APPEND_RBW_DIR, layoutVersion)
+    if (!DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.APPEND_RBW_DIR, layoutVersion)
         && detachDir.exists() && detachDir.isDirectory()) {
 
       if (FileUtil.list(detachDir).length != 0) {
@@ -389,13 +384,13 @@ public class BlockPoolSliceStorage extends Storage {
     // the namespace state or can be further upgraded to it.
     // In another word, we can only roll back when ( storedLV >= software LV)
     // && ( DN.previousCTime <= NN.ctime)
-    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION && 
+    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION && 
         prevInfo.getCTime() <= nsInfo.getCTime())) { // cannot rollback
       throw new InconsistentFSStateException(bpSd.getRoot(),
           "Cannot rollback to a newer state.\nDatanode previous state: LV = "
               + prevInfo.getLayoutVersion() + " CTime = " + prevInfo.getCTime()
               + " is newer than the namespace state: LV = "
-              + nsInfo.getLayoutVersion() + " CTime = " + nsInfo.getCTime());
+              + HdfsConstants.DATANODE_LAYOUT_VERSION + " CTime = " + nsInfo.getCTime());
     }
     
     LOG.info("Rolling back storage directory " + bpSd.getRoot()

+ 97 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNodeLayoutVersion.java

@@ -0,0 +1,97 @@
+/**
+ * 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.hdfs.server.datanode;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.FeatureInfo;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
+
+@InterfaceAudience.Private
+public class DataNodeLayoutVersion {  
+  /** Build layout version and corresponding feature matrix */
+  public final static Map<Integer, SortedSet<LayoutFeature>> FEATURES = 
+    new HashMap<Integer, SortedSet<LayoutFeature>>();
+  
+  public static final int CURRENT_LAYOUT_VERSION
+      = LayoutVersion.getCurrentLayoutVersion(FEATURES, Feature.values());
+
+  static{
+    LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
+    LayoutVersion.updateMap(FEATURES, DataNodeLayoutVersion.Feature.values());
+  }
+  
+  public static SortedSet<LayoutFeature> getFeatures(int lv) {
+    return FEATURES.get(lv);
+  }
+
+  public static boolean supports(final LayoutFeature f, final int lv) {
+    return LayoutVersion.supports(FEATURES, f, lv);
+  }
+
+  /**
+   * Enums for features that change the layout version.
+   * <br><br>
+   * To add a new layout version:
+   * <ul>
+   * <li>Define a new enum constant with a short enum name, the new layout version 
+   * and description of the added feature.</li>
+   * <li>When adding a layout version with an ancestor that is not same as
+   * its immediate predecessor, use the constructor where a specific ancestor
+   * can be passed.
+   * </li>
+   * </ul>
+   */
+  public static enum Feature implements LayoutFeature {
+    FIRST_LAYOUT(-52, "First datenode layout");
+   
+    private final FeatureInfo info;
+
+    /**
+     * DataNodeFeature that is added at layout version {@code lv} - 1. 
+     * @param lv new layout version with the addition of this feature
+     * @param description description of the feature
+     */
+    Feature(final int lv, final String description) {
+      this(lv, lv + 1, description, false);
+    }
+
+    /**
+     * DataNode feature that is added at layout version {@code ancestoryLV}.
+     * @param lv new layout version with the addition of this feature
+     * @param ancestorLV layout version from which the new lv is derived from.
+     * @param description description of the feature
+     * @param reserved true when this is a layout version reserved for previous
+     *        version
+     * @param features set of features that are to be enabled for this version
+     */
+    Feature(final int lv, final int ancestorLV, final String description,
+        boolean reserved, Feature... features) {
+      info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
+    }
+    
+    @Override
+    public FeatureInfo getInfo() {
+      return info;
+    }
+  }
+}

+ 28 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -91,7 +90,7 @@ public class DataStorage extends Storage {
   }
   
   public DataStorage(StorageInfo storageInfo) {
-    super(NodeType.DATA_NODE, storageInfo);
+    super(storageInfo);
   }
 
   public synchronized String getDatanodeUuid() {
@@ -131,10 +130,9 @@ public class DataStorage extends Storage {
       // DN storage has been initialized, no need to do anything
       return;
     }
-    assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
-      "Data-node version " + HdfsConstants.LAYOUT_VERSION + 
-      " and name-node layout version " + nsInfo.getLayoutVersion() + 
-      " must be the same.";
+    if( HdfsConstants.DATANODE_LAYOUT_VERSION == nsInfo.getLayoutVersion())
+      LOG.info("Data-node version: " + HdfsConstants.DATANODE_LAYOUT_VERSION + 
+      " and name-node layout version: " + nsInfo.getLayoutVersion());
     
     // 1. For each data directory calculate its state and 
     // check whether all is consistent before transitioning.
@@ -261,7 +259,7 @@ public class DataStorage extends Storage {
   void format(StorageDirectory sd, NamespaceInfo nsInfo,
               String datanodeUuid) throws IOException {
     sd.clearDirectory(); // create directory
-    this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    this.layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
     this.clusterID = nsInfo.getClusterID();
     this.namespaceID = nsInfo.getNamespaceID();
     this.cTime = 0;
@@ -297,7 +295,8 @@ public class DataStorage extends Storage {
     }
 
     // Set NamespaceID in version before federation
-    if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+    if (!DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       props.setProperty("namespaceID", String.valueOf(namespaceID));
     }
   }
@@ -321,11 +320,12 @@ public class DataStorage extends Storage {
       setLayoutVersion(props, sd);
     }
     setcTime(props, sd);
-    setStorageType(props, sd);
+    checkStorageType(props, sd);
     setClusterId(props, layoutVersion, sd);
     
     // Read NamespaceID in version before federation
-    if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+    if (!DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       setNamespaceID(props, sd);
     }
     
@@ -414,11 +414,12 @@ public class DataStorage extends Storage {
     }
     readProperties(sd);
     checkVersionUpgradable(this.layoutVersion);
-    assert this.layoutVersion >= HdfsConstants.LAYOUT_VERSION :
+    assert this.layoutVersion >= HdfsConstants.DATANODE_LAYOUT_VERSION :
       "Future version is not allowed";
     
     boolean federationSupported = 
-      LayoutVersion.supports(Feature.FEDERATION, layoutVersion);
+      DataNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FEDERATION, layoutVersion);
     // For pre-federation version - validate the namespaceID
     if (!federationSupported &&
         getNamespaceID() != nsInfo.getNamespaceID()) {
@@ -440,11 +441,11 @@ public class DataStorage extends Storage {
     // meaningful at BlockPoolSliceStorage level. 
 
     // regular start up. 
-    if (this.layoutVersion == HdfsConstants.LAYOUT_VERSION)
+    if (this.layoutVersion == HdfsConstants.DATANODE_LAYOUT_VERSION)
       return; // regular startup
     
     // do upgrade
-    if (this.layoutVersion > HdfsConstants.LAYOUT_VERSION) {
+    if (this.layoutVersion > HdfsConstants.DATANODE_LAYOUT_VERSION) {
       doUpgrade(sd, nsInfo);  // upgrade
       return;
     }
@@ -455,7 +456,7 @@ public class DataStorage extends Storage {
     // failed.
     throw new IOException("BUG: The stored LV = " + this.getLayoutVersion()
                           + " is newer than the supported LV = "
-                          + HdfsConstants.LAYOUT_VERSION
+                          + HdfsConstants.DATANODE_LAYOUT_VERSION
                           + " or name node LV = "
                           + nsInfo.getLayoutVersion());
   }
@@ -485,7 +486,8 @@ public class DataStorage extends Storage {
   void doUpgrade(StorageDirectory sd, NamespaceInfo nsInfo) throws IOException {
     // If the existing on-disk layout version supportes federation, simply
     // update its layout version.
-    if (LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+    if (DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, layoutVersion)) {
       // The VERSION file is already read in. Override the layoutVersion 
       // field and overwrite the file.
       LOG.info("Updating layout version from " + layoutVersion + " to "
@@ -528,7 +530,7 @@ public class DataStorage extends Storage {
     linkAllBlocks(tmpDir, bbwDir, new File(curBpDir, STORAGE_DIR_CURRENT));
     
     // 4. Write version file under <SD>/current
-    layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    layoutVersion = HdfsConstants.DATANODE_LAYOUT_VERSION;
     clusterID = nsInfo.getClusterID();
     writeProperties(sd);
     
@@ -548,7 +550,8 @@ public class DataStorage extends Storage {
    * @throws IOException if the directory is not empty or it can not be removed
    */
   private void cleanupDetachDir(File detachDir) throws IOException {
-    if (!LayoutVersion.supports(Feature.APPEND_RBW_DIR, layoutVersion) &&
+    if (!DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.APPEND_RBW_DIR, layoutVersion) &&
         detachDir.exists() && detachDir.isDirectory() ) {
       
         if (FileUtil.list(detachDir).length != 0 ) {
@@ -590,9 +593,10 @@ public class DataStorage extends Storage {
       // running a wrong version.  But this will be detected in block pool
       // level and the invalid VERSION content will be overwritten when
       // the error is corrected and rollback is retried.
-      if (LayoutVersion.supports(Feature.FEDERATION,
-          HdfsConstants.LAYOUT_VERSION) && 
-          HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion()) {
+      if (DataNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FEDERATION,
+          HdfsConstants.DATANODE_LAYOUT_VERSION) && 
+          HdfsConstants.DATANODE_LAYOUT_VERSION == nsInfo.getLayoutVersion()) {
         readProperties(sd, nsInfo.getLayoutVersion());
         writeProperties(sd);
         LOG.info("Layout version rolled back to " +
@@ -605,7 +609,7 @@ public class DataStorage extends Storage {
 
     // We allow rollback to a state, which is either consistent with
     // the namespace state or can be further upgraded to it.
-    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.LAYOUT_VERSION
+    if (!(prevInfo.getLayoutVersion() >= HdfsConstants.DATANODE_LAYOUT_VERSION
           && prevInfo.getCTime() <= nsInfo.getCTime()))  // cannot rollback
       throw new InconsistentFSStateException(sd.getRoot(),
           "Cannot rollback to a newer state.\nDatanode previous state: LV = "
@@ -710,7 +714,8 @@ public class DataStorage extends Storage {
     HardLink hardLink = new HardLink();
     // do the link
     int diskLayoutVersion = this.getLayoutVersion();
-    if (LayoutVersion.supports(Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
+    if (DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.APPEND_RBW_DIR, diskLayoutVersion)) {
       // hardlink finalized blocks in tmpDir/finalized
       linkBlocks(new File(fromDir, STORAGE_DIR_FINALIZED), 
           new File(toDir, STORAGE_DIR_FINALIZED), diskLayoutVersion, hardLink);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -413,9 +413,9 @@ public class BackupNode extends NameNode {
       LOG.fatal(errorMsg);
       throw new IOException(errorMsg);
     }
-    assert HdfsConstants.LAYOUT_VERSION == nsInfo.getLayoutVersion() :
+    assert HdfsConstants.NAMENODE_LAYOUT_VERSION == nsInfo.getLayoutVersion() :
       "Active and backup node layout versions must be the same. Expected: "
-      + HdfsConstants.LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
+      + HdfsConstants.NAMENODE_LAYOUT_VERSION + " actual "+ nsInfo.getLayoutVersion();
     return nsInfo;
   }
 

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CheckpointSignature.java

@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.namenode.FSImage;
 
 import com.google.common.collect.ComparisonChain;
@@ -47,6 +48,7 @@ public class CheckpointSignature extends StorageInfo
   }
 
   CheckpointSignature(String str) {
+    super(NodeType.NAME_NODE);
     String[] fields = str.split(FIELD_SEPARATOR);
     assert fields.length == NUM_FIELDS :
       "Must be " + NUM_FIELDS + " fields in CheckpointSignature";

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -36,7 +36,6 @@ import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@@ -149,7 +148,8 @@ public class EditLogFileInputStream extends EditLogInputStream {
       } catch (EOFException eofe) {
         throw new LogHeaderCorruptException("No header found in log");
       }
-      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_LAYOUT_FLAGS, logVersion)) {
         try {
           LayoutFlags.read(dataIn);
         } catch (EOFException eofe) {
@@ -328,12 +328,12 @@ public class EditLogFileInputStream extends EditLogInputStream {
       throw new LogHeaderCorruptException(
           "Reached EOF when reading log header");
     }
-    if (logVersion < HdfsConstants.LAYOUT_VERSION || // future version
+    if (logVersion < HdfsConstants.NAMENODE_LAYOUT_VERSION || // future version
         logVersion > Storage.LAST_UPGRADABLE_LAYOUT_VERSION) { // unsupported
       throw new LogHeaderCorruptException(
           "Unexpected version of the file system log file: "
           + logVersion + ". Current version = "
-          + HdfsConstants.LAYOUT_VERSION + ".");
+          + HdfsConstants.NAMENODE_LAYOUT_VERSION + ".");
     }
     return logVersion;
   }

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -132,7 +132,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
    */
   @VisibleForTesting
   public static void writeHeader(DataOutputStream out) throws IOException {
-    out.writeInt(HdfsConstants.LAYOUT_VERSION);
+    out.writeInt(HdfsConstants.NAMENODE_LAYOUT_VERSION);
     LayoutFlags.write(out);
   }
 

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java

@@ -35,7 +35,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
@@ -285,7 +284,8 @@ public class FSEditLogLoader {
     long inodeId = inodeIdFromOp;
 
     if (inodeId == INodeId.GRANDFATHER_INODE_ID) {
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         throw new IOException("The layout version " + logVersion
             + " supports inodeId but gave bogus inodeId");
       }
@@ -922,7 +922,7 @@ public class FSEditLogLoader {
     // The editlog must be emptied by restarting the namenode, before proceeding
     // with the upgrade.
     if (Storage.is203LayoutVersion(logVersion)
-        && logVersion != HdfsConstants.LAYOUT_VERSION) {
+        && logVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
       String msg = "During upgrade failed to load the editlog version "
           + logVersion + " from release 0.20.203. Please go back to the old "
           + " release and restart the namenode. This empties the editlog "

+ 66 - 36
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java

@@ -87,7 +87,6 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.util.XMLUtils;
 import org.apache.hadoop.hdfs.util.XMLUtils.InvalidXmlException;
@@ -248,8 +247,8 @@ public abstract class FSEditLogOp {
   
   void readRpcIds(DataInputStream in, int logVersion)
       throws IOException {
-    if (LayoutVersion.supports(Feature.EDITLOG_SUPPORT_RETRYCACHE,
-        logVersion)) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.EDITLOG_SUPPORT_RETRYCACHE, logVersion)) {
       this.rpcClientId = FSImageSerialization.readBytes(in);
       this.rpcCallId = FSImageSerialization.readInt(in);
     }
@@ -385,18 +384,20 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
       }
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         this.inodeId = in.readLong();
       } else {
         // The inodeId should be updated when this editLogOp is applied
         this.inodeId = INodeId.GRANDFATHER_INODE_ID;
       }
       if ((-17 < logVersion && length != 4) ||
-          (logVersion <= -17 && length != 5 && !LayoutVersion.supports(
-              Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
+          (logVersion <= -17 && length != 5 && !NameNodeLayoutVersion.supports(
+              LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion))) {
         throw new IOException("Incorrect data format."  +
                               " logVersion is " + logVersion +
                               " but writables.length is " +
@@ -404,7 +405,8 @@ public abstract class FSEditLogOp {
       }
       this.path = FSImageSerialization.readString(in);
 
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.replication = FSImageSerialization.readShort(in);
         this.mtime = FSImageSerialization.readLong(in);
       } else {
@@ -412,8 +414,10 @@ public abstract class FSEditLogOp {
         this.mtime = readLong(in);
       }
 
-      if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
-        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
           this.atime = FSImageSerialization.readLong(in);
         } else {
           this.atime = readLong(in);
@@ -422,7 +426,8 @@ public abstract class FSEditLogOp {
         this.atime = 0;
       }
 
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.blockSize = FSImageSerialization.readLong(in);
       } else {
         this.blockSize = readLong(in);
@@ -831,7 +836,8 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
       this.path = FSImageSerialization.readString(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.replication = FSImageSerialization.readShort(in);
       } else {
         this.replication = readShort(in);
@@ -922,7 +928,8 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (length < 3) { // trg, srcs.., timestamp
           throw new IOException("Incorrect data format " +
@@ -931,7 +938,8 @@ public abstract class FSEditLogOp {
       }
       this.trg = FSImageSerialization.readString(in);
       int srcSize = 0;
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         srcSize = in.readInt();
       } else {
         srcSize = this.length - 1 - 1; // trg and timestamp
@@ -950,7 +958,8 @@ public abstract class FSEditLogOp {
         srcs[i]= FSImageSerialization.readString(in);
       }
       
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
       } else {
         this.timestamp = readLong(in);
@@ -1056,7 +1065,8 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (this.length != 3) {
           throw new IOException("Incorrect data format. "
@@ -1065,7 +1075,8 @@ public abstract class FSEditLogOp {
       }
       this.src = FSImageSerialization.readString(in);
       this.dst = FSImageSerialization.readString(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
       } else {
         this.timestamp = readLong(in);
@@ -1152,14 +1163,16 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (this.length != 2) {
           throw new IOException("Incorrect data format. " + "delete operation.");
         }
       }
       this.path = FSImageSerialization.readString(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
       } else {
         this.timestamp = readLong(in);
@@ -1253,22 +1266,26 @@ public abstract class FSEditLogOp {
     
     @Override
     void readFields(DataInputStream in, int logVersion) throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
       }
       if (-17 < logVersion && length != 2 ||
           logVersion <= -17 && length != 3
-          && !LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+          && !NameNodeLayoutVersion.supports(
+              LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         throw new IOException("Incorrect data format. Mkdir operation.");
       }
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when this editLogOp is applied
         this.inodeId = INodeId.GRANDFATHER_INODE_ID;
       }
       this.path = FSImageSerialization.readString(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
       } else {
         this.timestamp = readLong(in);
@@ -1277,8 +1294,10 @@ public abstract class FSEditLogOp {
       // The disk format stores atimes for directories as well.
       // However, currently this is not being updated/used because of
       // performance reasons.
-      if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, logVersion)) {
-        if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FILE_ACCESS_TIME, logVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
           FSImageSerialization.readLong(in);
         } else {
           readLong(in);
@@ -1861,7 +1880,8 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (length != 3) {
           throw new IOException("Incorrect data format. " + "times operation.");
@@ -1869,7 +1889,8 @@ public abstract class FSEditLogOp {
       }
       this.path = FSImageSerialization.readString(in);
 
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.mtime = FSImageSerialization.readLong(in);
         this.atime = FSImageSerialization.readLong(in);
       } else {
@@ -1978,14 +1999,16 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (this.length != 4) {
           throw new IOException("Incorrect data format. "
               + "symlink operation.");
         }
       }
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, logVersion)) {
         this.inodeId = FSImageSerialization.readLong(in);
       } else {
         // This id should be updated when the editLogOp is applied
@@ -1994,7 +2017,8 @@ public abstract class FSEditLogOp {
       this.path = FSImageSerialization.readString(in);
       this.value = FSImageSerialization.readString(in);
 
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.mtime = FSImageSerialization.readLong(in);
         this.atime = FSImageSerialization.readLong(in);
       } else {
@@ -2112,7 +2136,8 @@ public abstract class FSEditLogOp {
     @Override
     void readFields(DataInputStream in, int logVersion)
         throws IOException {
-      if (!LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.length = in.readInt();
         if (this.length != 3) {
           throw new IOException("Incorrect data format. " + "Rename operation.");
@@ -2121,7 +2146,8 @@ public abstract class FSEditLogOp {
       this.src = FSImageSerialization.readString(in);
       this.dst = FSImageSerialization.readString(in);
 
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.timestamp = FSImageSerialization.readLong(in);
       } else {
         this.timestamp = readLong(in);
@@ -2332,7 +2358,8 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       this.token.readFields(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.expiryTime = FSImageSerialization.readLong(in);
       } else {
         this.expiryTime = readLong(in);
@@ -2404,7 +2431,8 @@ public abstract class FSEditLogOp {
         throws IOException {
       this.token = new DelegationTokenIdentifier();
       this.token.readFields(in);
-      if (LayoutVersion.supports(Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITLOG_OP_OPTIMIZATION, logVersion)) {
         this.expiryTime = FSImageSerialization.readLong(in);
       } else {
         this.expiryTime = readLong(in);
@@ -3488,7 +3516,8 @@ public abstract class FSEditLogOp {
      */
     public Reader(DataInputStream in, StreamLimiter limiter, int logVersion) {
       this.logVersion = logVersion;
-      if (LayoutVersion.supports(Feature.EDITS_CHESKUM, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.EDITS_CHESKUM, logVersion)) {
         this.checksum = new PureJavaCrc32();
       } else {
         this.checksum = null;
@@ -3627,7 +3656,8 @@ public abstract class FSEditLogOp {
         throw new IOException("Read invalid opcode " + opCode);
       }
 
-      if (LayoutVersion.supports(Feature.STORED_TXIDS, logVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.STORED_TXIDS, logVersion)) {
         // Read the txid
         op.setTransactionId(in.readLong());
       } else {

+ 12 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java

@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -216,11 +215,11 @@ public class FSImage implements Closeable {
     }
     if (startOpt != StartupOption.UPGRADE
         && layoutVersion < Storage.LAST_PRE_UPGRADE_LAYOUT_VERSION
-        && layoutVersion != HdfsConstants.LAYOUT_VERSION) {
+        && layoutVersion != HdfsConstants.NAMENODE_LAYOUT_VERSION) {
       throw new IOException(
           "\nFile system image contains an old layout version " 
           + storage.getLayoutVersion() + ".\nAn upgrade to version "
-          + HdfsConstants.LAYOUT_VERSION + " is required.\n"
+          + HdfsConstants.NAMENODE_LAYOUT_VERSION + " is required.\n"
           + "Please restart NameNode with -upgrade option.");
     }
     
@@ -333,7 +332,7 @@ public class FSImage implements Closeable {
     long oldCTime = storage.getCTime();
     storage.cTime = now();  // generate new cTime for the state
     int oldLV = storage.getLayoutVersion();
-    storage.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    storage.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
     
     List<StorageDirectory> errorSDs =
       Collections.synchronizedList(new ArrayList<StorageDirectory>());
@@ -393,11 +392,11 @@ public class FSImage implements Closeable {
     boolean canRollback = false;
     FSImage prevState = new FSImage(conf);
     try {
-      prevState.getStorage().layoutVersion = HdfsConstants.LAYOUT_VERSION;
+      prevState.getStorage().layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
       for (Iterator<StorageDirectory> it = storage.dirIterator(false); it.hasNext();) {
         StorageDirectory sd = it.next();
         if (!NNUpgradeUtil.canRollBack(sd, storage, prevState.getStorage(),
-            HdfsConstants.LAYOUT_VERSION)) {
+            HdfsConstants.NAMENODE_LAYOUT_VERSION)) {
           continue;
         }
         canRollback = true;
@@ -407,7 +406,7 @@ public class FSImage implements Closeable {
         // If HA is enabled, check if the shared log can be rolled back as well.
         editLog.initJournalsForWrite();
         canRollback |= editLog.canRollBackSharedLog(prevState.getStorage(),
-            HdfsConstants.LAYOUT_VERSION);
+            HdfsConstants.NAMENODE_LAYOUT_VERSION);
       }
       
       if (!canRollback)
@@ -561,8 +560,8 @@ public class FSImage implements Closeable {
 
     initEditLog(startOpt);
 
-    if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, 
-                               getLayoutVersion())) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
       // If we're open for write, we're either non-HA or we're the active NN, so
       // we better be able to load all the edits. If we're the standby NN, it's
       // OK to not be able to read all of edits right now.
@@ -618,13 +617,13 @@ public class FSImage implements Closeable {
     StorageDirectory sdForProperties = imageFile.sd;
     storage.readProperties(sdForProperties);
 
-    if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT,
-                               getLayoutVersion())) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
       // For txid-based layout, we should have a .md5 file
       // next to the image file
       loadFSImage(imageFile.getFile(), target, recovery);
-    } else if (LayoutVersion.supports(Feature.FSIMAGE_CHECKSUM,
-                                      getLayoutVersion())) {
+    } else if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FSIMAGE_CHECKSUM, getLayoutVersion())) {
       // In 0.22, we have the checksum stored in the VERSION file.
       String md5 = storage.getDeprecatedProperty(
           NNStorage.DEPRECATED_MESSAGE_DIGEST_PROPERTY);

+ 46 - 28
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -260,9 +260,10 @@ public class FSImageFormat {
               "imgVersion " + imgVersion +
               " expected to be " + getLayoutVersion());
         }
-        boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
-            imgVersion);
-        if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
+        boolean supportSnapshot = NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.SNAPSHOT, imgVersion);
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
           LayoutFlags.read(in);
         }
 
@@ -275,7 +276,8 @@ public class FSImageFormat {
         long genstamp = in.readLong();
         namesystem.setGenerationStampV1(genstamp);
         
-        if (LayoutVersion.supports(Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imgVersion)) {
           // read the starting generation stamp for sequential block IDs
           genstamp = in.readLong();
           namesystem.setGenerationStampV2(genstamp);
@@ -297,14 +299,16 @@ public class FSImageFormat {
 
         // read the transaction ID of the last edit represented by
         // this image
-        if (LayoutVersion.supports(Feature.STORED_TXIDS, imgVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.STORED_TXIDS, imgVersion)) {
           imgTxId = in.readLong();
         } else {
           imgTxId = 0;
         }
 
         // read the last allocated inode id in the fsimage
-        if (LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.ADD_INODE_ID, imgVersion)) {
           long lastInodeId = in.readLong();
           namesystem.resetLastInodeId(lastInodeId);
           if (LOG.isDebugEnabled()) {
@@ -323,7 +327,8 @@ public class FSImageFormat {
 
         // read compression related info
         FSImageCompression compression;
-        if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imgVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.FSIMAGE_COMPRESSION, imgVersion)) {
           compression = FSImageCompression.readCompressionHeader(conf, in);
         } else {
           compression = FSImageCompression.createNoopCompression();
@@ -336,8 +341,8 @@ public class FSImageFormat {
         LOG.info("Number of files = " + numFiles);
         prog.setTotal(Phase.LOADING_FSIMAGE, step, numFiles);
         Counter counter = prog.getCounter(Phase.LOADING_FSIMAGE, step);
-        if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
-            imgVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imgVersion)) {
           if (supportSnapshot) {
             loadLocalNameINodesWithSnapshot(numFiles, in, counter);
           } else {
@@ -394,9 +399,10 @@ public class FSImageFormat {
      */
     private void loadLocalNameINodesWithSnapshot(long numFiles, DataInput in,
         Counter counter) throws IOException {
-      assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
-          getLayoutVersion());
-      assert LayoutVersion.supports(Feature.SNAPSHOT, getLayoutVersion());
+      assert NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, getLayoutVersion());
+      assert NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.SNAPSHOT, getLayoutVersion());
       
       // load root
       loadRoot(in, counter);
@@ -414,8 +420,8 @@ public class FSImageFormat {
    */  
    private void loadLocalNameINodes(long numFiles, DataInput in, Counter counter)
        throws IOException {
-     assert LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION,
-         getLayoutVersion());
+     assert NameNodeLayoutVersion.supports(
+         LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, getLayoutVersion());
      assert numFiles > 0;
 
      // load root
@@ -624,7 +630,8 @@ public class FSImageFormat {
       final byte[] localName = FSImageSerialization.readLocalName(in);
       INode inode = loadINode(localName, isSnapshotINode, in, counter);
       if (updateINodeMap
-          && LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
+          && NameNodeLayoutVersion.supports(
+              LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) {
         namesystem.dir.addToInodeMap(inode);
       }
       return inode;
@@ -640,18 +647,21 @@ public class FSImageFormat {
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in, Counter counter) throws IOException {
     final int imgVersion = getLayoutVersion();
-    if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
       namesystem.getFSDirectory().verifyINodeName(localName);
     }
 
-    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? 
-           in.readLong() : namesystem.allocateNewInodeId();
+    long inodeId = NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.ADD_INODE_ID, imgVersion) ? in.readLong()
+        : namesystem.allocateNewInodeId();
     
     final short replication = namesystem.getBlockManager().adjustReplication(
         in.readShort());
     final long modificationTime = in.readLong();
     long atime = 0;
-    if (LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imgVersion)) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FILE_ACCESS_TIME, imgVersion)) {
       atime = in.readLong();
     }
     final long blockSize = in.readLong();
@@ -671,7 +681,8 @@ public class FSImageFormat {
       String clientMachine = "";
       boolean underConstruction = false;
       FileDiffList fileDiffs = null;
-      if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
         // read diffs
         fileDiffs = SnapshotFSImageFormat.loadFileDiffList(in, this);
 
@@ -710,14 +721,16 @@ public class FSImageFormat {
       //read quotas
       final long nsQuota = in.readLong();
       long dsQuota = -1L;
-      if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imgVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.DISKSPACE_QUOTA, imgVersion)) {
         dsQuota = in.readLong();
       }
 
       //read snapshot info
       boolean snapshottable = false;
       boolean withSnapshot = false;
-      if (LayoutVersion.supports(Feature.SNAPSHOT, imgVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.SNAPSHOT, imgVersion)) {
         snapshottable = in.readBoolean();
         if (!snapshottable) {
           withSnapshot = in.readBoolean();
@@ -780,7 +793,8 @@ public class FSImageFormat {
         throws IOException {
       final int layoutVersion = getLayoutVersion();
       
-      if (!LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
         return loadINodeWithLocalName(true, in, false).asFile();
       }
   
@@ -801,7 +815,8 @@ public class FSImageFormat {
         throws IOException {
       final int layoutVersion = getLayoutVersion();
       
-      if (!LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, layoutVersion)) {
         return loadINodeWithLocalName(true, in, false).asDirectory();
       }
   
@@ -836,7 +851,8 @@ public class FSImageFormat {
         INodeFile oldnode = null;
         boolean inSnapshot = false;
         if (path != null && FSDirectory.isReservedName(path) && 
-            LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
+            NameNodeLayoutVersion.supports(
+                LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) {
           // TODO: for HDFS-5428, we use reserved path for those INodeFileUC in 
           // snapshot. If we support INode ID in the layout version, we can use
           // the inode id to find the oldnode.
@@ -869,7 +885,8 @@ public class FSImageFormat {
         throws IOException {
       int imgVersion = getLayoutVersion();
 
-      if (!LayoutVersion.supports(Feature.DELEGATION_TOKEN, imgVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.DELEGATION_TOKEN, imgVersion)) {
         //SecretManagerState is not available.
         //This must not happen if security is turned on.
         return; 
@@ -879,7 +896,8 @@ public class FSImageFormat {
 
     private void loadCacheManagerState(DataInput in) throws IOException {
       int imgVersion = getLayoutVersion();
-      if (!LayoutVersion.supports(Feature.CACHING, imgVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.CACHING, imgVersion)) {
         return;
       }
       namesystem.getCacheManager().loadStateCompat(in);
@@ -993,7 +1011,7 @@ public class FSImageFormat {
       DigestOutputStream fos = new DigestOutputStream(fout, digester);
       DataOutputStream out = new DataOutputStream(fos);
       try {
-        out.writeInt(HdfsConstants.LAYOUT_VERSION);
+        out.writeInt(HdfsConstants.NAMENODE_LAYOUT_VERSION);
         LayoutFlags.write(out);
         // We use the non-locked version of getNamespaceInfo here since
         // the coordinating thread of saveNamespace already has read-locked

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java

@@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
@@ -118,8 +117,9 @@ public class FSImageSerialization {
       DataInput in, FSNamesystem fsNamesys, int imgVersion)
       throws IOException {
     byte[] name = readBytes(in);
-    long inodeId = LayoutVersion.supports(Feature.ADD_INODE_ID, imgVersion) ? in
-        .readLong() : fsNamesys.allocateNewInodeId();
+    long inodeId = NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.ADD_INODE_ID, imgVersion) ? in.readLong()
+        : fsNamesys.allocateNewInodeId();
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileJournalManager.java

@@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageErrorReporter;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -531,7 +532,7 @@ public class FileJournalManager implements JournalManager {
 
   @Override
   public long getJournalCTime() throws IOException {
-    StorageInfo sInfo = new StorageInfo();
+    StorageInfo sInfo = new StorageInfo(NodeType.NAME_NODE);
     sInfo.readProperties(sd);
     return sInfo.getCTime();
   }

+ 11 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NNStorage.java

@@ -38,7 +38,6 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -533,7 +532,7 @@ public class NNStorage extends Storage implements Closeable,
    */
   public void format(NamespaceInfo nsInfo) throws IOException {
     Preconditions.checkArgument(nsInfo.getLayoutVersion() == 0 ||
-        nsInfo.getLayoutVersion() == HdfsConstants.LAYOUT_VERSION,
+        nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION,
         "Bad layout version: %s", nsInfo.getLayoutVersion());
     
     this.setStorageInfo(nsInfo);
@@ -552,7 +551,7 @@ public class NNStorage extends Storage implements Closeable,
   }
   
   public void format() throws IOException {
-    this.layoutVersion = HdfsConstants.LAYOUT_VERSION;
+    this.layoutVersion = HdfsConstants.NAMENODE_LAYOUT_VERSION;
     for (Iterator<StorageDirectory> it =
                            dirIterator(); it.hasNext();) {
       StorageDirectory sd = it.next();
@@ -589,7 +588,8 @@ public class NNStorage extends Storage implements Closeable,
     }
 
     // Set Block pool ID in version with federation support
-    if (versionSupportsFederation()) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, getLayoutVersion())) {
       String sbpid = props.getProperty("blockpoolID");
       setBlockPoolID(sd.getRoot(), sbpid);
     }
@@ -614,7 +614,7 @@ public class NNStorage extends Storage implements Closeable,
    * This should only be used during upgrades.
    */
   String getDeprecatedProperty(String prop) {
-    assert getLayoutVersion() > HdfsConstants.LAYOUT_VERSION :
+    assert getLayoutVersion() > HdfsConstants.NAMENODE_LAYOUT_VERSION :
       "getDeprecatedProperty should only be done when loading " +
       "storage from past versions during upgrade.";
     return deprecatedProperties.get(prop);
@@ -636,7 +636,8 @@ public class NNStorage extends Storage implements Closeable,
                            ) throws IOException {
     super.setPropertiesFromFields(props, sd);
     // Set blockpoolID in version with federation support
-    if (versionSupportsFederation()) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, getLayoutVersion())) {
       props.setProperty("blockpoolID", blockpoolID);
     }
   }
@@ -808,7 +809,8 @@ public class NNStorage extends Storage implements Closeable,
       // If upgrade from a release that does not support federation,
       // if clusterId is provided in the startupOptions use it.
       // Else generate a new cluster ID      
-      if (!LayoutVersion.supports(Feature.FEDERATION, layoutVersion)) {
+      if (!NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FEDERATION, layoutVersion)) {
         if (startOpt.getClusterId() == null) {
           startOpt.setClusterId(newClusterID());
         }
@@ -1005,7 +1007,8 @@ public class NNStorage extends Storage implements Closeable,
     // (ie edits_<txnid>) then use the new inspector, which will ignore
     // the old format dirs.
     FSImageStorageInspector inspector;
-    if (LayoutVersion.supports(Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.TXID_BASED_LAYOUT, getLayoutVersion())) {
       inspector = new FSImageTransactionalStorageInspector();
     } else {
       inspector = new FSImagePreTransactionalStorageInspector();

+ 98 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java

@@ -0,0 +1,98 @@
+/**
+ * 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.hdfs.server.namenode;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.FeatureInfo;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
+
+
+@InterfaceAudience.Private
+public class NameNodeLayoutVersion { 
+  /** Build layout version and corresponding feature matrix */
+  public final static Map<Integer, SortedSet<LayoutFeature>> FEATURES
+      = new HashMap<Integer, SortedSet<LayoutFeature>>();
+
+  public static final int CURRENT_LAYOUT_VERSION
+      = LayoutVersion.getCurrentLayoutVersion(FEATURES, Feature.values());
+
+  static {
+    LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
+    LayoutVersion.updateMap(FEATURES, NameNodeLayoutVersion.Feature.values());
+  }
+  
+  public static SortedSet<LayoutFeature> getFeatures(int lv) {
+    return FEATURES.get(lv);
+  }
+
+  public static boolean supports(final LayoutFeature f, final int lv) {
+    return LayoutVersion.supports(FEATURES, f, lv);
+  }
+
+  /**
+   * Enums for features that change the layout version.
+   * <br><br>
+   * To add a new layout version:
+   * <ul>
+   * <li>Define a new enum constant with a short enum name, the new layout version 
+   * and description of the added feature.</li>
+   * <li>When adding a layout version with an ancestor that is not same as
+   * its immediate predecessor, use the constructor where a specific ancestor
+   * can be passed.
+   * </li>
+   * </ul>
+   */
+  public static enum Feature implements LayoutFeature {
+    ROLLING_UPGRADE_MARKER(-52, "Upgrade marker for rolling upgrade");
+    
+    private final FeatureInfo info;
+
+    /**
+     * Feature that is added at layout version {@code lv} - 1. 
+     * @param lv new layout version with the addition of this feature
+     * @param description description of the feature
+     */
+    Feature(final int lv, final String description) {
+      this(lv, lv + 1, description, false);
+    }
+
+    /**
+     * NameNode feature that is added at layout version {@code ancestoryLV}.
+     * @param lv new layout version with the addition of this feature
+     * @param ancestorLV layout version from which the new lv is derived from.
+     * @param description description of the feature
+     * @param reserved true when this is a layout version reserved for previous
+     *        versions
+     * @param features set of features that are to be enabled for this version
+     */
+    Feature(final int lv, final int ancestorLV, final String description,
+        boolean reserved, Feature... features) {
+      info = new FeatureInfo(lv, ancestorLV, description, reserved, features);
+    }
+    
+    @Override
+    public FeatureInfo getInfo() {
+      return info;
+    }
+  }
+}

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -973,7 +973,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
   @Override // DatanodeProtocol
   public DatanodeRegistration registerDatanode(DatanodeRegistration nodeReg)
       throws IOException {
-    verifyLayoutVersion(nodeReg.getVersion());
     verifySoftwareVersion(nodeReg);
     namesystem.registerDatanode(nodeReg);
     return nodeReg;
@@ -1071,7 +1070,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws UnregisteredNodeException if the registration is invalid
    */
   void verifyRequest(NodeRegistration nodeReg) throws IOException {
-    verifyLayoutVersion(nodeReg.getVersion());
     if (!namesystem.getRegistrationID().equals(nodeReg.getRegistrationID())) {
       LOG.warn("Invalid registrationID - expected: "
           + namesystem.getRegistrationID() + " received: "
@@ -1147,8 +1145,9 @@ class NameNodeRpcServer implements NamenodeProtocols {
    * @throws IOException
    */
   void verifyLayoutVersion(int version) throws IOException {
-    if (version != HdfsConstants.LAYOUT_VERSION)
-      throw new IncorrectVersionException(version, "data node");
+    if (version != HdfsConstants.NAMENODE_LAYOUT_VERSION)
+      throw new IncorrectVersionException(
+          HdfsConstants.NAMENODE_LAYOUT_VERSION, version, "data node");
   }
   
   private void verifySoftwareVersion(DatanodeRegistration dnReg)

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -511,8 +511,10 @@ public class SecondaryNameNode implements Runnable {
     boolean loadImage = false;
     boolean isFreshCheckpointer = (checkpointImage.getNamespaceID() == 0);
     boolean isSameCluster =
-        (dstStorage.versionSupportsFederation() && sig.isSameCluster(checkpointImage)) ||
-        (!dstStorage.versionSupportsFederation() && sig.namespaceIdMatches(checkpointImage));
+        (dstStorage.versionSupportsFederation(NameNodeLayoutVersion.FEATURES)
+            && sig.isSameCluster(checkpointImage)) ||
+        (!dstStorage.versionSupportsFederation(NameNodeLayoutVersion.FEATURES)
+            && sig.namespaceIdMatches(checkpointImage));
     if (isFreshCheckpointer ||
         (isSameCluster &&
          !sig.storageVersionMatches(checkpointImage.getStorage()))) {

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/BootstrapStandby.java

@@ -153,9 +153,9 @@ public class BootstrapStandby implements Tool, Configurable {
     }
 
     if (!checkLayoutVersion(nsInfo)) {
-      LOG.fatal("Layout version on remote node (" +
-          nsInfo.getLayoutVersion() + ") does not match " +
-          "this node's layout version (" + HdfsConstants.LAYOUT_VERSION + ")");
+      LOG.fatal("Layout version on remote node (" + nsInfo.getLayoutVersion()
+          + ") does not match " + "this node's layout version ("
+          + HdfsConstants.NAMENODE_LAYOUT_VERSION + ")");
       return ERR_CODE_INVALID_VERSION;
     }
 
@@ -257,7 +257,7 @@ public class BootstrapStandby implements Tool, Configurable {
   }
 
   private boolean checkLayoutVersion(NamespaceInfo nsInfo) throws IOException {
-    return (nsInfo.getLayoutVersion() == HdfsConstants.LAYOUT_VERSION);
+    return (nsInfo.getLayoutVersion() == HdfsConstants.NAMENODE_LAYOUT_VERSION);
   }
   
   private void parseConfAndFindOtherNN() throws IOException {

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/NamespaceInfo.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.util.VersionInfo;
 
@@ -41,13 +42,14 @@ public class NamespaceInfo extends StorageInfo {
   String softwareVersion;
 
   public NamespaceInfo() {
-    super();
+    super(NodeType.NAME_NODE);
     buildVersion = null;
   }
 
   public NamespaceInfo(int nsID, String clusterID, String bpID,
       long cT, String buildVersion, String softwareVersion) {
-    super(HdfsConstants.LAYOUT_VERSION, nsID, clusterID, cT);
+    super(HdfsConstants.NAMENODE_LAYOUT_VERSION, nsID, clusterID, cT,
+        NodeType.NAME_NODE);
     blockPoolID = bpID;
     this.buildVersion = buildVersion;
     this.softwareVersion = softwareVersion;

+ 35 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -30,10 +30,10 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.apache.hadoop.hdfs.tools.offlineImageViewer.ImageVisitor.ImageElement;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.WritableUtils;
@@ -158,7 +158,8 @@ class ImageLoaderCurrent implements ImageLoader {
       imageVersion = in.readInt();
       if( !canLoadVersion(imageVersion))
         throw new IOException("Cannot process fslayout version " + imageVersion);
-      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
         LayoutFlags.read(in);
       }
 
@@ -169,22 +170,25 @@ class ImageLoaderCurrent implements ImageLoader {
 
       v.visit(ImageElement.GENERATION_STAMP, in.readLong());
 
-      if (LayoutVersion.supports(Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.SEQUENTIAL_BLOCK_ID, imageVersion)) {
         v.visit(ImageElement.GENERATION_STAMP_V2, in.readLong());
         v.visit(ImageElement.GENERATION_STAMP_V1_LIMIT, in.readLong());
         v.visit(ImageElement.LAST_ALLOCATED_BLOCK_ID, in.readLong());
       }
 
-      if (LayoutVersion.supports(Feature.STORED_TXIDS, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.STORED_TXIDS, imageVersion)) {
         v.visit(ImageElement.TRANSACTION_ID, in.readLong());
       }
       
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
         v.visit(ImageElement.LAST_INODE_ID, in.readLong());
       }
       
-      boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
-          imageVersion);
+      boolean supportSnapshot = NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.SNAPSHOT, imageVersion);
       if (supportSnapshot) {
         v.visit(ImageElement.SNAPSHOT_COUNTER, in.readInt());
         int numSnapshots = in.readInt();
@@ -194,7 +198,8 @@ class ImageLoaderCurrent implements ImageLoader {
         }
       }
       
-      if (LayoutVersion.supports(Feature.FSIMAGE_COMPRESSION, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.FSIMAGE_COMPRESSION, imageVersion)) {
         boolean isCompressed = in.readBoolean();
         v.visit(ImageElement.IS_COMPRESSED, String.valueOf(isCompressed));
         if (isCompressed) {
@@ -216,11 +221,13 @@ class ImageLoaderCurrent implements ImageLoader {
 
       processINodesUC(in, v, skipBlocks);
 
-      if (LayoutVersion.supports(Feature.DELEGATION_TOKEN, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.DELEGATION_TOKEN, imageVersion)) {
         processDelegationTokens(in, v);
       }
       
-      if (LayoutVersion.supports(Feature.CACHING, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.CACHING, imageVersion)) {
         processCacheManagerState(in, v);
       }
       v.leaveEnclosingElement(); // FSImage
@@ -323,7 +330,8 @@ class ImageLoaderCurrent implements ImageLoader {
       String n = new String(name, "UTF8");
       v.visit(ImageElement.INODE_PATH, n);
       
-      if (LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.ADD_INODE_ID, imageVersion)) {
         long inodeId = in.readLong();
         v.visit(ImageElement.INODE_ID, inodeId);
       }
@@ -443,7 +451,8 @@ class ImageLoaderCurrent implements ImageLoader {
     v.visitEnclosingElement(ImageElement.INODES,
         ImageElement.NUM_INODES, numInodes);
     
-    if (LayoutVersion.supports(Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
+    if (NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FSIMAGE_NAME_OPTIMIZATION, imageVersion)) {
       if (!supportSnapshot) {
         processLocalNameINodes(in, v, numInodes, skipBlocks);
       } else {
@@ -584,7 +593,8 @@ class ImageLoaderCurrent implements ImageLoader {
     if (!useRoot) {
       if (in.readBoolean()) {
         v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_DIRECTORY_ATTRIBUTES);
-        if (LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
+        if (NameNodeLayoutVersion.supports(
+            LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
           processINodeDirectoryAttributes(in, v, currentINodeName);
         } else {
           processINode(in, v, true, currentINodeName, true);
@@ -678,10 +688,10 @@ class ImageLoaderCurrent implements ImageLoader {
   private void processINode(DataInputStream in, ImageVisitor v,
       boolean skipBlocks, String parentName, boolean isSnapshotCopy)
       throws IOException {
-    boolean supportSnapshot = 
-        LayoutVersion.supports(Feature.SNAPSHOT, imageVersion);
-    boolean supportInodeId = 
-        LayoutVersion.supports(Feature.ADD_INODE_ID, imageVersion);
+    boolean supportSnapshot = NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.SNAPSHOT, imageVersion);
+    boolean supportInodeId = NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.ADD_INODE_ID, imageVersion);
     
     v.visitEnclosingElement(ImageElement.INODE);
     final String pathName = readINodePath(in, parentName);
@@ -694,7 +704,8 @@ class ImageLoaderCurrent implements ImageLoader {
     }
     v.visit(ImageElement.REPLICATION, in.readShort());
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-    if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion))
+    if(NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion))
       v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
     v.visit(ImageElement.BLOCK_SIZE, in.readLong());
     int numBlocks = in.readInt();
@@ -723,7 +734,8 @@ class ImageLoaderCurrent implements ImageLoader {
         dirNodeMap.put(inodeId, pathName);
       }
       v.visit(ImageElement.NS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
-      if (LayoutVersion.supports(Feature.DISKSPACE_QUOTA, imageVersion))
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.DISKSPACE_QUOTA, imageVersion))
         v.visit(ImageElement.DS_QUOTA, numBlocks == -1 ? in.readLong() : -1);
       if (supportSnapshot) {
         boolean snapshottable = in.readBoolean();
@@ -770,7 +782,8 @@ class ImageLoaderCurrent implements ImageLoader {
     v.visit(ImageElement.INODE_PATH, pathName);
     processPermission(in, v);
     v.visit(ImageElement.MODIFICATION_TIME, formatDate(in.readLong()));
-    if(LayoutVersion.supports(Feature.FILE_ACCESS_TIME, imageVersion)) {
+    if(NameNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FILE_ACCESS_TIME, imageVersion)) {
       v.visit(ImageElement.ACCESS_TIME, formatDate(in.readLong()));
     }
 
@@ -799,7 +812,8 @@ class ImageLoaderCurrent implements ImageLoader {
     v.visit(ImageElement.SNAPSHOT_FILE_SIZE, in.readLong());
     if (in.readBoolean()) {
       v.visitEnclosingElement(ImageElement.SNAPSHOT_INODE_FILE_ATTRIBUTES);
-      if (LayoutVersion.supports(Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
+      if (NameNodeLayoutVersion.supports(
+          LayoutVersion.Feature.OPTIMIZE_SNAPSHOT_INODES, imageVersion)) {
         processINodeFileAttributes(in, v, currentINodeName);
       } else {
         processINode(in, v, true, currentINodeName, true);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java

@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -884,8 +885,8 @@ public class DFSTestUtil {
   }
   
   public static DatanodeRegistration getLocalDatanodeRegistration() {
-    return new DatanodeRegistration(getLocalDatanodeID(),
-        new StorageInfo(), new ExportedBlockKeys(), VersionInfo.getVersion());
+    return new DatanodeRegistration(getLocalDatanodeID(), new StorageInfo(
+        NodeType.DATA_NODE), new ExportedBlockKeys(), VersionInfo.getVersion());
   }
   
   /** Copy one file's contents into the other **/

+ 13 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSRollback.java

@@ -30,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -190,10 +191,11 @@ public class TestDFSRollback {
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
       // Put newer layout version in current.
       storageInfo = new StorageInfo(
-          UpgradeUtilities.getCurrentLayoutVersion()-1,
+          HdfsConstants.DATANODE_LAYOUT_VERSION - 1,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster),
-          UpgradeUtilities.getCurrentFsscTime(cluster));
+          UpgradeUtilities.getCurrentFsscTime(cluster),
+          NodeType.DATA_NODE);
 
       // Overwrite VERSION file in the current directory of
       // volume directories and block pool slice directories
@@ -250,7 +252,8 @@ public class TestDFSRollback {
       storageInfo = new StorageInfo(Integer.MIN_VALUE, 
           UpgradeUtilities.getCurrentNamespaceID(cluster), 
           UpgradeUtilities.getCurrentClusterID(cluster), 
-          UpgradeUtilities.getCurrentFsscTime(cluster));
+          UpgradeUtilities.getCurrentFsscTime(cluster),
+          NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
@@ -274,10 +277,11 @@ public class TestDFSRollback {
       
       UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "previous");
-      storageInfo = new StorageInfo(UpgradeUtilities.getCurrentLayoutVersion(), 
-            UpgradeUtilities.getCurrentNamespaceID(cluster), 
-            UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE);
-      
+      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
+          UpgradeUtilities.getCurrentNamespaceID(cluster),
+          UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
+          NodeType.DATA_NODE);
+     
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
       
@@ -317,10 +321,10 @@ public class TestDFSRollback {
       log("NameNode rollback with old layout version in previous", numDirs);
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       baseDirs = UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "previous");
-      storageInfo = new StorageInfo(1, 
+      storageInfo = new StorageInfo(1,
           UpgradeUtilities.getCurrentNamespaceID(null),
           UpgradeUtilities.getCurrentClusterID(null),
-          UpgradeUtilities.getCurrentFsscTime(null));
+          UpgradeUtilities.getCurrentFsscTime(null), NodeType.NAME_NODE);
       
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs,
           storageInfo, UpgradeUtilities.getCurrentBlockPoolID(cluster));

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStartupVersions.java

@@ -75,7 +75,7 @@ public class TestDFSStartupVersions {
     StorageData(int layoutVersion, int namespaceId, String clusterId,
         long cTime, String bpid) {
       storageInfo = new StorageInfo(layoutVersion, namespaceId, clusterId,
-          cTime);
+          cTime, NodeType.DATA_NODE);
       blockPoolId = bpid;
     }
   }
@@ -89,7 +89,7 @@ public class TestDFSStartupVersions {
    */
   private StorageData[] initializeVersions() throws Exception {
     int layoutVersionOld = Storage.LAST_UPGRADABLE_LAYOUT_VERSION;
-    int layoutVersionCur = UpgradeUtilities.getCurrentLayoutVersion();
+    int layoutVersionCur = HdfsConstants.DATANODE_LAYOUT_VERSION;
     int layoutVersionNew = Integer.MIN_VALUE;
     int namespaceIdCur = UpgradeUtilities.getCurrentNamespaceID(null);
     int namespaceIdOld = Integer.MIN_VALUE;
@@ -200,7 +200,7 @@ public class TestDFSStartupVersions {
       return false;
     }
     // check #3
-    int softwareLV = HdfsConstants.LAYOUT_VERSION;  // will also be Namenode's LV
+    int softwareLV = HdfsConstants.DATANODE_LAYOUT_VERSION;
     int storedLV = datanodeVer.getLayoutVersion();
     if (softwareLV == storedLV &&  
         datanodeVer.getCTime() == namenodeVer.getCTime()) 
@@ -252,7 +252,7 @@ public class TestDFSStartupVersions {
                                               .startupOption(StartupOption.REGULAR)
                                               .build();
     StorageData nameNodeVersion = new StorageData(
-        UpgradeUtilities.getCurrentLayoutVersion(),
+        HdfsConstants.NAMENODE_LAYOUT_VERSION,
         UpgradeUtilities.getCurrentNamespaceID(cluster),
         UpgradeUtilities.getCurrentClusterID(cluster),
         UpgradeUtilities.getCurrentFsscTime(cluster),

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -34,6 +34,8 @@ import java.util.regex.Pattern;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
@@ -262,10 +264,10 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       cluster = createCluster();
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
-      storageInfo = new StorageInfo(Integer.MIN_VALUE, 
+      storageInfo = new StorageInfo(Integer.MIN_VALUE,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
           UpgradeUtilities.getCurrentClusterID(cluster),
-          UpgradeUtilities.getCurrentFsscTime(cluster));
+          UpgradeUtilities.getCurrentFsscTime(cluster), NodeType.DATA_NODE);
       
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo,
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
@@ -280,9 +282,10 @@ public class TestDFSUpgrade {
       UpgradeUtilities.createNameNodeStorageDirs(nameNodeDirs, "current");
       cluster = createCluster();
       baseDirs = UpgradeUtilities.createDataNodeStorageDirs(dataNodeDirs, "current");
-      storageInfo = new StorageInfo(UpgradeUtilities.getCurrentLayoutVersion(), 
+      storageInfo = new StorageInfo(HdfsConstants.DATANODE_LAYOUT_VERSION,
           UpgradeUtilities.getCurrentNamespaceID(cluster),
-          UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE);
+          UpgradeUtilities.getCurrentClusterID(cluster), Long.MAX_VALUE,
+          NodeType.DATA_NODE);
           
       UpgradeUtilities.createDataNodeVersionFile(baseDirs, storageInfo, 
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
@@ -321,7 +324,7 @@ public class TestDFSUpgrade {
       storageInfo = new StorageInfo(Storage.LAST_UPGRADABLE_LAYOUT_VERSION + 1, 
           UpgradeUtilities.getCurrentNamespaceID(null),
           UpgradeUtilities.getCurrentClusterID(null),
-          UpgradeUtilities.getCurrentFsscTime(null));
+          UpgradeUtilities.getCurrentFsscTime(null), NodeType.NAME_NODE);
       
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs, storageInfo,
           UpgradeUtilities.getCurrentBlockPoolID(cluster));
@@ -334,7 +337,7 @@ public class TestDFSUpgrade {
       storageInfo = new StorageInfo(Integer.MIN_VALUE, 
           UpgradeUtilities.getCurrentNamespaceID(null),
           UpgradeUtilities.getCurrentClusterID(null),
-          UpgradeUtilities.getCurrentFsscTime(null));
+          UpgradeUtilities.getCurrentFsscTime(null), NodeType.NAME_NODE);
       
       UpgradeUtilities.createNameNodeVersionFile(conf, baseDirs, storageInfo,
           UpgradeUtilities.getCurrentBlockPoolID(cluster));

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeRegistration.java

@@ -179,7 +179,7 @@ public class TestDatanodeRegistration {
           .getCTime();
       StorageInfo mockStorageInfo = mock(StorageInfo.class);
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
-      doReturn(HdfsConstants.LAYOUT_VERSION).when(mockStorageInfo)
+      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockStorageInfo)
           .getLayoutVersion();
       DatanodeRegistration dnReg = new DatanodeRegistration(dnId,
           mockStorageInfo, null, VersionInfo.getVersion());
@@ -224,7 +224,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
-      doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
       doReturn(123).when(mockDnReg).getXferPort();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
@@ -272,7 +272,7 @@ public class TestDatanodeRegistration {
       doReturn(nnCTime).when(mockStorageInfo).getCTime();
       
       DatanodeRegistration mockDnReg = mock(DatanodeRegistration.class);
-      doReturn(HdfsConstants.LAYOUT_VERSION).when(mockDnReg).getVersion();
+      doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(mockDnReg).getVersion();
       doReturn("fake-storage-id").when(mockDnReg).getDatanodeUuid();
       doReturn(mockStorageInfo).when(mockDnReg).getStorageInfo();
       

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/UpgradeUtilities.java

@@ -40,13 +40,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
-import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.BlockPoolSliceStorage;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.datanode.DataStorage;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -471,7 +471,8 @@ public class UpgradeUtilities {
   public static void createBlockPoolVersionFile(File bpDir,
       StorageInfo version, String bpid) throws IOException {
     // Create block pool version files
-    if (LayoutVersion.supports(Feature.FEDERATION, version.layoutVersion)) {
+    if (DataNodeLayoutVersion.supports(
+        LayoutVersion.Feature.FEDERATION, version.layoutVersion)) {
       File bpCurDir = new File(bpDir, Storage.STORAGE_DIR_CURRENT);
       BlockPoolSliceStorage bpStorage = new BlockPoolSliceStorage(version,
           bpid);
@@ -514,8 +515,8 @@ public class UpgradeUtilities {
    * Return the layout version inherent in the current version
    * of the Namenode, whether it is running or not.
    */
-  public static int getCurrentLayoutVersion() {
-    return HdfsConstants.LAYOUT_VERSION;
+  public static int getCurrentNameNodeLayoutVersion() {
+    return HdfsConstants.NAMENODE_LAYOUT_VERSION;
   }
   
   /**

+ 38 - 15
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java

@@ -20,9 +20,13 @@ package org.apache.hadoop.hdfs.protocol;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
-import java.util.EnumSet;
+import java.util.SortedSet;
 
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.FeatureInfo;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.LayoutFeature;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion;
 import org.junit.Test;
 
 /**
@@ -36,7 +40,7 @@ public class TestLayoutVersion {
    */
   @Test
   public void testFeaturesFromAncestorSupported() {
-    for (Feature f : Feature.values()) {
+    for (LayoutFeature f : Feature.values()) {
       validateFeatureList(f);
     }
   }
@@ -46,8 +50,8 @@ public class TestLayoutVersion {
    */
   @Test
   public void testRelease203() {
-    assertTrue(LayoutVersion.supports(Feature.DELEGATION_TOKEN, 
-        Feature.RESERVED_REL20_203.lv));
+    assertTrue(NameNodeLayoutVersion.supports(LayoutVersion.Feature.DELEGATION_TOKEN, 
+        Feature.RESERVED_REL20_203.getInfo().getLayoutVersion()));
   }
   
   /**
@@ -55,8 +59,8 @@ public class TestLayoutVersion {
    */
   @Test
   public void testRelease204() {
-    assertTrue(LayoutVersion.supports(Feature.DELEGATION_TOKEN, 
-        Feature.RESERVED_REL20_204.lv));
+    assertTrue(NameNodeLayoutVersion.supports(LayoutVersion.Feature.DELEGATION_TOKEN, 
+        Feature.RESERVED_REL20_204.getInfo().getLayoutVersion()));
   }
   
   /**
@@ -64,23 +68,42 @@ public class TestLayoutVersion {
    */
   @Test
   public void testRelease1_2_0() {
-    assertTrue(LayoutVersion.supports(Feature.CONCAT, 
-        Feature.RESERVED_REL1_2_0.lv));
+    assertTrue(NameNodeLayoutVersion.supports(LayoutVersion.Feature.CONCAT, 
+        Feature.RESERVED_REL1_2_0.getInfo().getLayoutVersion()));
+  }
+  
+  /**
+   * Test to make sure NameNode.Feature support previous features
+   */
+  @Test
+  public void testNameNodeFeature() {
+    assertTrue(NameNodeLayoutVersion.supports(LayoutVersion.Feature.CACHING,
+        NameNodeLayoutVersion.Feature.ROLLING_UPGRADE_MARKER.getInfo().getLayoutVersion()));
+  }
+  
+  /**
+   * Test to make sure DataNode.Feature support previous features
+   */
+  @Test
+  public void testDataNodeFeature() {
+    assertTrue(DataNodeLayoutVersion.supports(LayoutVersion.Feature.CACHING,
+        DataNodeLayoutVersion.Feature.FIRST_LAYOUT.getInfo().getLayoutVersion()));
   }
   
   /**
    * Given feature {@code f}, ensures the layout version of that feature
    * supports all the features supported by it's ancestor.
    */
-  private void validateFeatureList(Feature f) {
-    int lv = f.lv;
-    int ancestorLV = f.ancestorLV;
-    EnumSet<Feature> ancestorSet = LayoutVersion.map.get(ancestorLV);
+  private void validateFeatureList(LayoutFeature f) {
+    final FeatureInfo info = f.getInfo();
+    int lv = info.getLayoutVersion();
+    int ancestorLV = info.getAncestorLayoutVersion();
+    SortedSet<LayoutFeature> ancestorSet = NameNodeLayoutVersion.getFeatures(ancestorLV);
     assertNotNull(ancestorSet);
-    for (Feature  feature : ancestorSet) {
+    for (LayoutFeature  feature : ancestorSet) {
       assertTrue("LV " + lv + " does nto support " + feature
-          + " supported by the ancestor LV " + f.ancestorLV,
-          LayoutVersion.supports(feature, lv));
+          + " supported by the ancestor LV " + info.getAncestorLayoutVersion(),
+          NameNodeLayoutVersion.supports(feature, lv));
     }
   }
 }

+ 9 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature;
 import org.apache.hadoop.hdfs.server.protocol.*;
@@ -100,15 +101,15 @@ public class TestPBHelper {
         PBHelper.convert(NamenodeRoleProto.NAMENODE));
   }
 
-  private static StorageInfo getStorageInfo() {
-    return new StorageInfo(1, 2, "cid", 3);
+  private static StorageInfo getStorageInfo(NodeType type) {
+    return new StorageInfo(1, 2, "cid", 3, type);
   }
 
   @Test
   public void testConvertStoragInfo() {
-    StorageInfo info = getStorageInfo();
+    StorageInfo info = getStorageInfo(NodeType.NAME_NODE);
     StorageInfoProto infoProto = PBHelper.convert(info);
-    StorageInfo info2 = PBHelper.convert(infoProto);
+    StorageInfo info2 = PBHelper.convert(infoProto, NodeType.NAME_NODE);
     assertEquals(info.getClusterID(), info2.getClusterID());
     assertEquals(info.getCTime(), info2.getCTime());
     assertEquals(info.getLayoutVersion(), info2.getLayoutVersion());
@@ -117,7 +118,7 @@ public class TestPBHelper {
 
   @Test
   public void testConvertNamenodeRegistration() {
-    StorageInfo info = getStorageInfo();
+    StorageInfo info = getStorageInfo(NodeType.NAME_NODE);
     NamenodeRegistration reg = new NamenodeRegistration("address:999",
         "http:1000", info, NamenodeRole.NAMENODE);
     NamenodeRegistrationProto regProto = PBHelper.convert(reg);
@@ -243,8 +244,8 @@ public class TestPBHelper {
 
   @Test
   public void testConvertCheckpointSignature() {
-    CheckpointSignature s = new CheckpointSignature(getStorageInfo(), "bpid",
-        100, 1);
+    CheckpointSignature s = new CheckpointSignature(
+        getStorageInfo(NodeType.NAME_NODE), "bpid", 100, 1);
     CheckpointSignatureProto sProto = PBHelper.convert(s);
     CheckpointSignature s1 = PBHelper.convert(sProto);
     assertEquals(s.getBlockpoolID(), s1.getBlockpoolID());
@@ -515,7 +516,7 @@ public class TestPBHelper {
     ExportedBlockKeys expKeys = new ExportedBlockKeys(true, 9, 10,
         getBlockKey(1), keys);
     DatanodeRegistration reg = new DatanodeRegistration(dnId,
-        new StorageInfo(), expKeys, "3.0.0");
+        new StorageInfo(NodeType.DATA_NODE), expKeys, "3.0.0");
     DatanodeRegistrationProto proto = PBHelper.convert(reg);
     DatanodeRegistration reg2 = PBHelper.convert(proto);
     compare(reg.getStorageInfo(), reg2.getStorageInfo());

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -190,7 +190,7 @@ public class TestJournalNode {
     byte[] retrievedViaHttp = DFSTestUtil.urlGetBytes(new URL(urlRoot +
         "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
-            Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
+            Ints.toByteArray(HdfsConstants.NAMENODE_LAYOUT_VERSION),
             (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
 

+ 6 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java

@@ -67,7 +67,8 @@ public class TestDatanodeRegister {
     // Return a a good software version.
     doReturn(VersionInfo.getVersion()).when(fakeNsInfo).getSoftwareVersion();
     // Return a good layout version for now.
-    doReturn(HdfsConstants.LAYOUT_VERSION).when(fakeNsInfo).getLayoutVersion();
+    doReturn(HdfsConstants.DATANODE_LAYOUT_VERSION).when(fakeNsInfo)
+        .getLayoutVersion();
     
     DatanodeProtocolClientSideTranslatorPB fakeDnProt = 
         mock(DatanodeProtocolClientSideTranslatorPB.class);
@@ -103,20 +104,17 @@ public class TestDatanodeRegister {
   @Test
   public void testDifferentLayoutVersions() throws Exception {
     // We expect no exceptions to be thrown when the layout versions match.
-    assertEquals(HdfsConstants.LAYOUT_VERSION,
+    assertEquals(HdfsConstants.NAMENODE_LAYOUT_VERSION,
         actor.retrieveNamespaceInfo().getLayoutVersion());
     
     // We expect an exception to be thrown when the NN reports a layout version
     // different from that of the DN.
-    doReturn(HdfsConstants.LAYOUT_VERSION * 1000).when(fakeNsInfo)
+    doReturn(HdfsConstants.NAMENODE_LAYOUT_VERSION * 1000).when(fakeNsInfo)
         .getLayoutVersion();
     try {
       actor.retrieveNamespaceInfo();
-      fail("Should have failed to retrieve NS info from DN with bad layout version");
-    } catch (IncorrectVersionException ive) {
-      GenericTestUtils.assertExceptionContains(
-          "Unexpected version of namenode", ive);
-      LOG.info("Got expected exception", ive);
+    } catch (IOException e) {
+      fail("Should not fail to retrieve NS info from DN with different layout version");
     }
   }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -61,6 +61,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -1975,7 +1976,7 @@ public class TestCheckpoint {
         .when(dstImage).getFiles(
             Mockito.<NameNodeDirType>anyObject(), Mockito.anyString());
       
-      Mockito.doReturn(new StorageInfo(1, 1, "X", 1).toColonSeparatedString())
+      Mockito.doReturn(new StorageInfo(1, 1, "X", 1, NodeType.NAME_NODE).toColonSeparatedString())
         .when(dstImage).toColonSeparatedString();
 
       try {

+ 5 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartupOptionUpgrade.java

@@ -67,7 +67,7 @@ public class TestStartupOptionUpgrade {
    */
   @Test
   public void testStartupOptUpgradeFrom204() throws Exception {
-    layoutVersion = Feature.RESERVED_REL20_204.getLayoutVersion();
+    layoutVersion = Feature.RESERVED_REL20_204.getInfo().getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
     assertTrue("Clusterid should start with CID", storage.getClusterID()
         .startsWith("CID"));
@@ -83,7 +83,7 @@ public class TestStartupOptionUpgrade {
   @Test
   public void testStartupOptUpgradeFrom22WithCID() throws Exception {
     startOpt.setClusterId("cid");
-    layoutVersion = Feature.RESERVED_REL22.getLayoutVersion();
+    layoutVersion = Feature.RESERVED_REL22.getInfo().getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
     assertEquals("Clusterid should match with the given clusterid",
         "cid", storage.getClusterID());
@@ -101,7 +101,7 @@ public class TestStartupOptionUpgrade {
       throws Exception {
     // Test assumes clusterid already exists, set the clusterid
     storage.setClusterID("currentcid");
-    layoutVersion = Feature.FEDERATION.getLayoutVersion();
+    layoutVersion = Feature.FEDERATION.getInfo().getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
     assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());
@@ -119,7 +119,7 @@ public class TestStartupOptionUpgrade {
       throws Exception {
     startOpt.setClusterId("wrong-cid");
     storage.setClusterID("currentcid");
-    layoutVersion = Feature.FEDERATION.getLayoutVersion();
+    layoutVersion = Feature.FEDERATION.getInfo().getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
     assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());
@@ -137,7 +137,7 @@ public class TestStartupOptionUpgrade {
       throws Exception {
     startOpt.setClusterId("currentcid");
     storage.setClusterID("currentcid");
-    layoutVersion = Feature.FEDERATION.getLayoutVersion();
+    layoutVersion = Feature.FEDERATION.getInfo().getLayoutVersion();
     storage.processStartupOptionsForUpgrade(startOpt, layoutVersion);
     assertEquals("Clusterid should match with the existing one",
         "currentcid", storage.getClusterID());