Browse Source

HDFS-8432. Introduce a minimum compatible layout version to allow downgrade in more rolling upgrade use cases. Contributed by Chris Nauroth.

cnauroth 10 years ago
parent
commit
71de367c5e
16 changed files with 306 additions and 77 deletions
  1. 3 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  2. 49 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java
  3. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java
  4. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Checkpointer.java
  5. 12 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
  6. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImage.java
  7. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
  8. 76 24
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  9. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java
  10. 32 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
  11. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java
  12. 100 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/CreateEditsLog.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java
  15. 8 8
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java
  16. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -603,6 +603,9 @@ Release 2.8.0 - UNRELEASED
     HDFS-8535. Clarify that dfs usage in dfsadmin -report output includes all
     block replicas. (Eddy Xu via wang)
 
+    HDFS-8432. Introduce a minimum compatible layout version to allow downgrade
+    in more rolling upgrade use cases. (cnauroth)
+
   OPTIMIZATIONS
 
     HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than

+ 49 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -162,14 +162,22 @@ public class LayoutVersion {
   public static class FeatureInfo {
     private final int lv;
     private final int ancestorLV;
+    private final Integer minCompatLV;
     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, ancestorLV, null, description, reserved, specialFeatures);
+    }
+
+    public FeatureInfo(final int lv, final int ancestorLV, Integer minCompatLV,
+        final String description, boolean reserved,
+        LayoutFeature... specialFeatures) {
       this.lv = lv;
       this.ancestorLV = ancestorLV;
+      this.minCompatLV = minCompatLV;
       this.description = description;
       this.reserved = reserved;
       this.specialFeatures = specialFeatures;
@@ -191,7 +199,20 @@ public class LayoutVersion {
       return ancestorLV;
     }
 
-    /** 
+    /**
+     * Accessor method for feature minimum compatible layout version.  If the
+     * feature does not define a minimum compatible layout version, then this
+     * method returns the feature's own layout version.  This would indicate
+     * that the feature cannot provide compatibility with any prior layout
+     * version.
+     *
+     * @return int minimum compatible LV value
+     */
+    public int getMinimumCompatibleLayoutVersion() {
+      return minCompatLV != null ? minCompatLV : lv;
+    }
+
+    /**
      * Accessor method for feature description 
      * @return String feature description 
      */
@@ -220,8 +241,23 @@ public class LayoutVersion {
       LayoutFeature[] features) {
     // Go through all the enum constants and build a map of
     // LayoutVersion <-> Set of all supported features in that LayoutVersion
+    SortedSet<LayoutFeature> existingFeatures = new TreeSet<LayoutFeature>(
+        new LayoutFeatureComparator());
+    for (SortedSet<LayoutFeature> s : map.values()) {
+      existingFeatures.addAll(s);
+    }
+    LayoutFeature prevF = existingFeatures.isEmpty() ? null :
+        existingFeatures.first();
     for (LayoutFeature f : features) {
       final FeatureInfo info = f.getInfo();
+      int minCompatLV = info.getMinimumCompatibleLayoutVersion();
+      if (prevF != null &&
+          minCompatLV > prevF.getInfo().getMinimumCompatibleLayoutVersion()) {
+        throw new AssertionError(String.format(
+            "Features must be listed in order of minimum compatible layout " +
+            "version.  Check features %s and %s.", prevF, f));
+      }
+      prevF = f;
       SortedSet<LayoutFeature> ancestorSet = map.get(info.getAncestorLayoutVersion());
       if (ancestorSet == null) {
         // Empty set
@@ -282,6 +318,18 @@ public class LayoutVersion {
     return getLastNonReservedFeature(features).getInfo().getLayoutVersion();
   }
 
+  /**
+   * Gets the minimum compatible layout version.
+   *
+   * @param features all features to check
+   * @return minimum compatible layout version
+   */
+  public static int getMinimumCompatibleLayoutVersion(
+      LayoutFeature[] features) {
+    return getLastNonReservedFeature(features).getInfo()
+        .getMinimumCompatibleLayoutVersion();
+  }
+
   static LayoutFeature getLastNonReservedFeature(LayoutFeature[] features) {
     for (int i = features.length -1; i >= 0; i--) {
       final FeatureInfo info = features[i].getInfo();
@@ -292,4 +340,3 @@ public class LayoutVersion {
     throw new AssertionError("All layout versions are reserved.");
   }
 }
-

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

@@ -333,7 +333,7 @@ public class BackupImage extends FSImage {
    * directories.
    */
   synchronized void namenodeStartedLogSegment(long txid) throws IOException {
-    editLog.startLogSegment(txid, true);
+    editLog.startLogSegment(txid, true, namesystem.getEffectiveLayoutVersion());
 
     if (bnState == BNState.DROP_UNTIL_NEXT_ROLL) {
       setState(BNState.JOURNAL_ONLY);

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

@@ -257,7 +257,9 @@ class Checkpointer extends Daemon {
         backupNode.namesystem.setBlockTotal();
       }
       bnImage.saveFSImageInAllDirs(backupNode.getNamesystem(), txid);
-      bnStorage.writeAll();
+      if (!backupNode.namesystem.isRollingUpgrade()) {
+        bnStorage.writeAll();
+      }
     } finally {
       backupNode.namesystem.writeUnlock();
     }

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

@@ -301,7 +301,7 @@ public class FSEditLog implements LogsPurgeable {
    * Initialize the output stream for logging, opening the first
    * log segment.
    */
-  synchronized void openForWrite() throws IOException {
+  synchronized void openForWrite(int layoutVersion) throws IOException {
     Preconditions.checkState(state == State.BETWEEN_LOG_SEGMENTS,
         "Bad state: %s", state);
 
@@ -318,7 +318,7 @@ public class FSEditLog implements LogsPurgeable {
       throw new IllegalStateException(error);
     }
     
-    startLogSegmentAndWriteHeaderTxn(segmentTxId);
+    startLogSegmentAndWriteHeaderTxn(segmentTxId, layoutVersion);
     assert state == State.IN_SEGMENT : "Bad state: " + state;
   }
   
@@ -1197,12 +1197,12 @@ public class FSEditLog implements LogsPurgeable {
    * @return the transaction id of the BEGIN_LOG_SEGMENT transaction
    * in the new log.
    */
-  synchronized long rollEditLog() throws IOException {
+  synchronized long rollEditLog(int layoutVersion) throws IOException {
     LOG.info("Rolling edit logs");
     endCurrentLogSegment(true);
     
     long nextTxId = getLastWrittenTxId() + 1;
-    startLogSegmentAndWriteHeaderTxn(nextTxId);
+    startLogSegmentAndWriteHeaderTxn(nextTxId, layoutVersion);
     
     assert curSegmentTxId == nextTxId;
     return nextTxId;
@@ -1212,7 +1212,7 @@ public class FSEditLog implements LogsPurgeable {
    * Remote namenode just has started a log segment, start log segment locally.
    */
   public synchronized void startLogSegment(long txid, 
-      boolean abortCurrentLogSegment) throws IOException {
+      boolean abortCurrentLogSegment, int layoutVersion) throws IOException {
     LOG.info("Started a new log segment at txid " + txid);
     if (isSegmentOpen()) {
       if (getLastWrittenTxId() == txid - 1) {
@@ -1234,14 +1234,15 @@ public class FSEditLog implements LogsPurgeable {
       }
     }
     setNextTxId(txid);
-    startLogSegment(txid);
+    startLogSegment(txid, layoutVersion);
   }
   
   /**
    * Start writing to the log segment with the given txid.
    * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state. 
    */
-  private void startLogSegment(final long segmentTxId) throws IOException {
+  private void startLogSegment(final long segmentTxId, int layoutVersion)
+      throws IOException {
     assert Thread.holdsLock(this);
 
     LOG.info("Starting log segment at " + segmentTxId);
@@ -1263,8 +1264,7 @@ public class FSEditLog implements LogsPurgeable {
     storage.attemptRestoreRemovedStorage();
     
     try {
-      editLogStream = journalSet.startLogSegment(segmentTxId,
-          NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
+      editLogStream = journalSet.startLogSegment(segmentTxId, layoutVersion);
     } catch (IOException ex) {
       throw new IOException("Unable to start log segment " +
           segmentTxId + ": too few journals successfully started.", ex);
@@ -1274,9 +1274,9 @@ public class FSEditLog implements LogsPurgeable {
     state = State.IN_SEGMENT;
   }
 
-  synchronized void startLogSegmentAndWriteHeaderTxn(final long segmentTxId
-      ) throws IOException {
-    startLogSegment(segmentTxId);
+  synchronized void startLogSegmentAndWriteHeaderTxn(final long segmentTxId,
+      int layoutVersion) throws IOException {
+    startLogSegment(segmentTxId, layoutVersion);
 
     logEdit(LogSegmentOp.getInstance(cache.get(),
         FSEditLogOpCodes.OP_START_LOG_SEGMENT));

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

@@ -572,9 +572,9 @@ public class FSImage implements Closeable {
     return editLog;
   }
 
-  void openEditLogForWrite() throws IOException {
+  void openEditLogForWrite(int layoutVersion) throws IOException {
     assert editLog != null : "editLog must be initialized";
-    editLog.openForWrite();
+    editLog.openForWrite(layoutVersion);
     storage.writeTransactionIdFileToStorage(editLog.getCurSegmentTxId());
   }
   
@@ -1127,10 +1127,13 @@ public class FSImage implements Closeable {
     try {
       try {
         saveFSImageInAllDirs(source, nnf, imageTxId, canceler);
-        storage.writeAll();
+        if (!source.isRollingUpgrade()) {
+          storage.writeAll();
+        }
       } finally {
         if (editLogWasOpen) {
-          editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1);
+          editLog.startLogSegmentAndWriteHeaderTxn(imageTxId + 1,
+              source.getEffectiveLayoutVersion());
           // Take this opportunity to note the current transaction.
           // Even if the namespace save was cancelled, this marker
           // is only used to determine what transaction ID is required
@@ -1315,8 +1318,8 @@ public class FSImage implements Closeable {
     }
   }
 
-  CheckpointSignature rollEditLog() throws IOException {
-    getEditLog().rollEditLog();
+  CheckpointSignature rollEditLog(int layoutVersion) throws IOException {
+    getEditLog().rollEditLog(layoutVersion);
     // Record this log segment ID in all of the storage directories, so
     // we won't miss this log segment on a restart if the edits directories
     // go missing.
@@ -1341,7 +1344,8 @@ public class FSImage implements Closeable {
    * @throws IOException
    */
   NamenodeCommand startCheckpoint(NamenodeRegistration bnReg, // backup node
-                                  NamenodeRegistration nnReg) // active name-node
+                                  NamenodeRegistration nnReg,
+                                  int layoutVersion) // active name-node
   throws IOException {
     LOG.info("Start checkpoint at txid " + getEditLog().getLastWrittenTxId());
     String msg = null;
@@ -1370,7 +1374,7 @@ public class FSImage implements Closeable {
     if(storage.getNumStorageDirs(NameNodeDirType.IMAGE) == 0)
       // do not return image if there are no image directories
       needToReturnImg = false;
-    CheckpointSignature sig = rollEditLog();
+    CheckpointSignature sig = rollEditLog(layoutVersion);
     return new CheckpointCommand(sig, needToReturnImg);
   }
 

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

@@ -465,7 +465,8 @@ public final class FSImageFormatProtobuf {
 
       FileSummary.Builder b = FileSummary.newBuilder()
           .setOndiskVersion(FSImageUtil.FILE_VERSION)
-          .setLayoutVersion(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
+          .setLayoutVersion(
+              context.getSourceNamesystem().getEffectiveLayoutVersion());
 
       codec = compression.getImageCodec();
       if (codec != null) {

+ 76 - 24
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -210,7 +210,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -225,6 +224,7 @@ import org.apache.hadoop.hdfs.server.namenode.JournalSet.JournalAndStream;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.namenode.ha.EditLogTailer;
 import org.apache.hadoop.hdfs.server.namenode.ha.HAContext;
 import org.apache.hadoop.hdfs.server.namenode.ha.StandbyCheckpointer;
@@ -992,8 +992,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (needToSave) {
         fsImage.saveNamespace(this);
       } else {
-        updateStorageVersionForRollingUpgrade(fsImage.getLayoutVersion(),
-            startOpt);
         // No need to save, so mark the phase done.
         StartupProgress prog = NameNode.getStartupProgress();
         prog.beginPhase(Phase.SAVING_CHECKPOINT);
@@ -1003,7 +1001,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       // we shouldn't do it when coming up in standby state
       if (!haEnabled || (haEnabled && startOpt == StartupOption.UPGRADE)
           || (haEnabled && startOpt == StartupOption.UPGRADEONLY)) {
-        fsImage.openEditLogForWrite();
+        fsImage.openEditLogForWrite(getEffectiveLayoutVersion());
       }
       success = true;
     } finally {
@@ -1015,18 +1013,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     imageLoadComplete();
   }
 
-  private void updateStorageVersionForRollingUpgrade(final long layoutVersion,
-      StartupOption startOpt) throws IOException {
-    boolean rollingStarted = RollingUpgradeStartupOption.STARTED
-        .matches(startOpt) && layoutVersion > HdfsServerConstants
-        .NAMENODE_LAYOUT_VERSION;
-    boolean rollingRollback = RollingUpgradeStartupOption.ROLLBACK
-        .matches(startOpt);
-    if (rollingRollback || rollingStarted) {
-      fsImage.updateStorageVersion();
-    }
-  }
-
   private void startSecretManager() {
     if (dtSecretManager != null) {
       try {
@@ -1144,7 +1130,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             nextTxId);
         editLog.setNextTxId(nextTxId);
 
-        getFSImage().editLog.openForWrite();
+        getFSImage().editLog.openForWrite(getEffectiveLayoutVersion());
       }
 
       // Enable quota checks.
@@ -1941,6 +1927,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                    String clientName, String clientMachine,
                    long mtime)
       throws IOException, UnresolvedLinkException {
+    requireEffectiveLayoutVersionForFeature(Feature.TRUNCATE);
     boolean ret;
     try {
       ret = truncateInt(src, newLength, clientName, clientMachine, mtime);
@@ -2564,7 +2551,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     if (writeToEditLog) {
-      getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
+      if (NameNodeLayoutVersion.supports(Feature.APPEND_NEW_BLOCK,
+          getEffectiveLayoutVersion())) {
+        getEditLog().logAppendFile(src, file, newBlock, logRetryCache);
+      } else {
+        getEditLog().logOpenFile(src, file, false, logRetryCache);
+      }
     }
     return ret;
   }
@@ -2759,9 +2751,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   LastBlockWithStatus appendFile(String src, String holder,
       String clientMachine, EnumSet<CreateFlag> flag, boolean logRetryCache)
       throws IOException {
+    boolean newBlock = flag.contains(CreateFlag.NEW_BLOCK);
+    if (newBlock) {
+      requireEffectiveLayoutVersionForFeature(Feature.APPEND_NEW_BLOCK);
+    }
     try {
-      return appendFileInt(src, holder, clientMachine,
-          flag.contains(CreateFlag.NEW_BLOCK), logRetryCache);
+      return appendFileInt(src, holder, clientMachine, newBlock, logRetryCache);
     } catch (AccessControlException e) {
       logAuditEvent(false, "append", src);
       throw e;
@@ -3365,6 +3360,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   void setQuota(String src, long nsQuota, long ssQuota, StorageType type)
       throws IOException {
+    if (type != null) {
+      requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
+    }
     checkOperation(OperationCategory.WRITE);
     writeLock();
     boolean success = false;
@@ -5242,7 +5240,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (Server.isRpcInvocation()) {
         LOG.info("Roll Edit Log from " + Server.getRemoteAddress());
       }
-      return getFSImage().rollEditLog();
+      return getFSImage().rollEditLog(getEffectiveLayoutVersion());
     } finally {
       writeUnlock();
     }
@@ -5258,7 +5256,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       
       LOG.info("Start checkpoint for " + backupNode.getAddress());
       NamenodeCommand cmd = getFSImage().startCheckpoint(backupNode,
-          activeNamenode);
+          activeNamenode, getEffectiveLayoutVersion());
       getEditLog().logSync();
       return cmd;
     } finally {
@@ -6882,7 +6880,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       getEditLog().logStartRollingUpgrade(rollingUpgradeInfo.getStartTime());
       if (haEnabled) {
         // roll the edit log to make sure the standby NameNode can tail
-        getFSImage().rollEditLog();
+        getFSImage().rollEditLog(getEffectiveLayoutVersion());
       }
     } finally {
       writeUnlock();
@@ -6985,6 +6983,60 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return rollingUpgradeInfo != null && !rollingUpgradeInfo.isFinalized();
   }
 
+  /**
+   * Returns the layout version in effect.  Under normal operation, this is the
+   * same as the software's current layout version, defined in
+   * {@link NameNodeLayoutVersion#CURRENT_LAYOUT_VERSION}.  During a rolling
+   * upgrade, this can retain the layout version that was persisted to metadata
+   * prior to starting the rolling upgrade, back to a lower bound defined in
+   * {@link NameNodeLayoutVersion#MINIMUM_COMPATIBLE_LAYOUT_VERSION}.  New
+   * fsimage files and edit log segments will continue to be written with this
+   * older layout version, so that the files are still readable by the old
+   * software version if the admin chooses to downgrade.
+   *
+   * @return layout version in effect
+   */
+  public int getEffectiveLayoutVersion() {
+    if (isRollingUpgrade()) {
+      int storageLV = fsImage.getStorage().getLayoutVersion();
+      if (storageLV >=
+          NameNodeLayoutVersion.MINIMUM_COMPATIBLE_LAYOUT_VERSION) {
+        // The prior layout version satisfies the minimum compatible layout
+        // version of the current software.  Keep reporting the prior layout
+        // as the effective one.  Downgrade is possible.
+        return storageLV;
+      }
+    }
+    // The current software cannot satisfy the layout version of the prior
+    // software.  Proceed with using the current layout version.
+    return NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION;
+  }
+
+  /**
+   * Performs a pre-condition check that the layout version in effect is
+   * sufficient to support the requested {@link Feature}.  If not, then the
+   * method throws {@link HadoopIllegalArgumentException} to deny the operation.
+   * This exception class is registered as a terse exception, so it prevents
+   * verbose stack traces in the NameNode log.  During a rolling upgrade, this
+   * method is used to restrict usage of new features.  This prevents writing
+   * new edit log operations that would be unreadable by the old software
+   * version if the admin chooses to downgrade.
+   *
+   * @param f feature to check
+   * @throws HadoopIllegalArgumentException if the current layout version in
+   *     effect is insufficient to support the feature
+   */
+  private void requireEffectiveLayoutVersionForFeature(Feature f)
+      throws HadoopIllegalArgumentException {
+    int lv = getEffectiveLayoutVersion();
+    if (!NameNodeLayoutVersion.supports(f, lv)) {
+      throw new HadoopIllegalArgumentException(String.format(
+          "Feature %s unsupported at NameNode layout version %d.  If a " +
+          "rolling upgrade is in progress, then it must be finalized before " +
+          "using this feature.", f, lv));
+    }
+  }
+
   void checkRollingUpgrade(String action) throws RollingUpgradeException {
     if (isRollingUpgrade()) {
       throw new RollingUpgradeException("Failed to " + action
@@ -7008,7 +7060,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       getEditLog().logFinalizeRollingUpgrade(rollingUpgradeInfo.getFinalizeTime());
       if (haEnabled) {
         // roll the edit log to make sure the standby NameNode can tail
-        getFSImage().rollEditLog();
+        getFSImage().rollEditLog(getEffectiveLayoutVersion());
       }
       getFSImage().updateStorageVersion();
       getFSImage().renameCheckpoint(NameNodeFile.IMAGE_ROLLBACK,

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

@@ -1157,7 +1157,8 @@ public class NameNode implements NameNodeStatusMXBean {
             LOG.trace("copying op: " + op);
           }
           if (!segmentOpen) {
-            newSharedEditLog.startLogSegment(op.txid, false);
+            newSharedEditLog.startLogSegment(op.txid, false,
+                fsns.getEffectiveLayoutVersion());
             segmentOpen = true;
           }
 

+ 32 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java

@@ -35,6 +35,8 @@ public class NameNodeLayoutVersion {
 
   public static final int CURRENT_LAYOUT_VERSION
       = LayoutVersion.getCurrentLayoutVersion(Feature.values());
+  public static final int MINIMUM_COMPATIBLE_LAYOUT_VERSION
+      = LayoutVersion.getMinimumCompatibleLayoutVersion(Feature.values());
 
   static {
     LayoutVersion.updateMap(FEATURES, LayoutVersion.Feature.values());
@@ -60,43 +62,59 @@ public class NameNodeLayoutVersion {
    * its immediate predecessor, use the constructor where a specific ancestor
    * can be passed.
    * </li>
+   * <li>Specify a minimum compatible layout version.  The minimum compatible
+   * layout version is the earliest prior version to which a downgrade is
+   * possible after initiating rolling upgrade.  If the feature cannot satisfy
+   * compatibility with any prior version, then set its minimum compatible
+   * lqyout version to itself to indicate that downgrade is impossible.
+   * Satisfying compatibility might require adding logic to the new feature to
+   * reject operations or handle them differently while rolling upgrade is in
+   * progress.  In general, it's possible to satisfy compatiblity for downgrade
+   * if the new feature just involves adding new edit log ops.  Deeper
+   * structural changes, such as changing the way we place files in the metadata
+   * directories, might be incompatible.  Feature implementations should strive
+   * for compatibility, because it's in the best interest of our users to
+   * support downgrade.
    * </ul>
    */
   public static enum Feature implements LayoutFeature {
-    ROLLING_UPGRADE(-55, -53, "Support rolling upgrade", false),
-    EDITLOG_LENGTH(-56, "Add length field to every edit log op"),
-    XATTRS(-57, "Extended attributes"),
-    CREATE_OVERWRITE(-58, "Use single editlog record for " +
+    ROLLING_UPGRADE(-55, -53, -55, "Support rolling upgrade", false),
+    EDITLOG_LENGTH(-56, -56, "Add length field to every edit log op"),
+    XATTRS(-57, -57, "Extended attributes"),
+    CREATE_OVERWRITE(-58, -58, "Use single editlog record for " +
       "creating file with overwrite"),
-    XATTRS_NAMESPACE_EXT(-59, "Increase number of xattr namespaces"),
-    BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
-    TRUNCATE(-61, "Truncate"),
-    APPEND_NEW_BLOCK(-62, "Support appending to new block"),
-    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
+    XATTRS_NAMESPACE_EXT(-59, -59, "Increase number of xattr namespaces"),
+    BLOCK_STORAGE_POLICY(-60, -60, "Block Storage policy"),
+    TRUNCATE(-61, -61, "Truncate"),
+    APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
+    QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types");
 
     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 minCompatLV minimium compatible layout version
      * @param description description of the feature
      */
-    Feature(final int lv, final String description) {
-      this(lv, lv + 1, description, false);
+    Feature(final int lv, int minCompatLV, final String description) {
+      this(lv, lv + 1, minCompatLV, 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 minCompatLV minimum compatible layout version
      * @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);
+    Feature(final int lv, final int ancestorLV, int minCompatLV,
+        final String description, boolean reserved, Feature... features) {
+      info = new FeatureInfo(lv, ancestorLV, minCompatLV, description, reserved,
+          features);
     }
     
     @Override

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

@@ -1084,6 +1084,8 @@ public class SecondaryNameNode implements Runnable,
     Checkpointer.rollForwardByApplyingLogs(manifest, dstImage, dstNamesystem);
     // The following has the side effect of purging old fsimages/edit logs.
     dstImage.saveFSImageInAllDirs(dstNamesystem, dstImage.getLastAppliedTxId());
-    dstStorage.writeAll();
+    if (!dstNamesystem.isRollingUpgrade()) {
+      dstStorage.writeAll();
+    }
   }
 }

+ 100 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java

@@ -20,7 +20,13 @@ package org.apache.hadoop.hdfs.protocol;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.*;
 
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import java.util.SortedSet;
 
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
@@ -103,7 +109,100 @@ public class TestLayoutVersion {
     assertEquals(LAST_COMMON_FEATURE.getInfo().getLayoutVersion() - 1,
         first.getInfo().getLayoutVersion());
   }
-  
+
+  /**
+   * Tests expected values for minimum compatible layout version in NameNode
+   * features.  TRUNCATE, APPEND_NEW_BLOCK and QUOTA_BY_STORAGE_TYPE are all
+   * features that launched in the same release.  TRUNCATE was added first, so
+   * we expect all 3 features to have a minimum compatible layout version equal
+   * to TRUNCATE's layout version.  All features older than that existed prior
+   * to the concept of a minimum compatible layout version, so for each one, the
+   * minimum compatible layout version must be equal to itself.
+   */
+  @Test
+  public void testNameNodeFeatureMinimumCompatibleLayoutVersions() {
+    int baseLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+        .getLayoutVersion();
+    EnumSet<NameNodeLayoutVersion.Feature> compatibleFeatures = EnumSet.of(
+        NameNodeLayoutVersion.Feature.TRUNCATE,
+        NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
+        NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE);
+    for (LayoutFeature f : compatibleFeatures) {
+      assertEquals(String.format("Expected minimum compatible layout version " +
+          "%d for feature %s.", baseLV, f), baseLV,
+          f.getInfo().getMinimumCompatibleLayoutVersion());
+    }
+    List<LayoutFeature> features = new ArrayList<>();
+    features.addAll(EnumSet.allOf(LayoutVersion.Feature.class));
+    features.addAll(EnumSet.allOf(NameNodeLayoutVersion.Feature.class));
+    for (LayoutFeature f : features) {
+      if (!compatibleFeatures.contains(f)) {
+        assertEquals(String.format("Expected feature %s to have minimum " +
+            "compatible layout version set to itself.", f),
+            f.getInfo().getLayoutVersion(),
+            f.getInfo().getMinimumCompatibleLayoutVersion());
+      }
+    }
+  }
+
+  /**
+   * Tests that NameNode features are listed in order of minimum compatible
+   * layout version.  It would be inconsistent to have features listed out of
+   * order with respect to minimum compatible layout version, because it would
+   * imply going back in time to change compatibility logic in a software release
+   * that had already shipped.
+   */
+  @Test
+  public void testNameNodeFeatureMinimumCompatibleLayoutVersionAscending() {
+    LayoutFeature prevF = null;
+    for (LayoutFeature f : EnumSet.allOf(NameNodeLayoutVersion.Feature.class)) {
+      if (prevF != null) {
+        assertTrue(String.format("Features %s and %s not listed in order of " +
+            "minimum compatible layout version.", prevF, f),
+            f.getInfo().getMinimumCompatibleLayoutVersion() <=
+            prevF.getInfo().getMinimumCompatibleLayoutVersion());
+      } else {
+        prevF = f;
+      }
+    }
+  }
+
+  /**
+   * Tests that attempting to add a new NameNode feature out of order with
+   * respect to minimum compatible layout version will fail fast.
+   */
+  @Test(expected=AssertionError.class)
+  public void testNameNodeFeatureMinimumCompatibleLayoutVersionOutOfOrder() {
+    FeatureInfo ancestorF = LayoutVersion.Feature.RESERVED_REL2_4_0.getInfo();
+    LayoutFeature f = mock(LayoutFeature.class);
+    when(f.getInfo()).thenReturn(new FeatureInfo(
+        ancestorF.getLayoutVersion() - 1, ancestorF.getLayoutVersion(),
+        ancestorF.getMinimumCompatibleLayoutVersion() + 1, "Invalid feature.",
+        false));
+    Map<Integer, SortedSet<LayoutFeature>> features = new HashMap<>();
+    LayoutVersion.updateMap(features, LayoutVersion.Feature.values());
+    LayoutVersion.updateMap(features, new LayoutFeature[] { f });
+  }
+
+  /**
+   * Asserts the current minimum compatible layout version of the software, if a
+   * release were created from the codebase right now.  This test is meant to
+   * make developers stop and reconsider if they introduce a change that requires
+   * a new minimum compatible layout version.  This would make downgrade
+   * impossible.
+   */
+  @Test
+  public void testCurrentMinimumCompatibleLayoutVersion() {
+    int expectedMinCompatLV = NameNodeLayoutVersion.Feature.TRUNCATE.getInfo()
+        .getLayoutVersion();
+    int actualMinCompatLV = LayoutVersion.getMinimumCompatibleLayoutVersion(
+        NameNodeLayoutVersion.Feature.values());
+    assertEquals("The minimum compatible layout version has changed.  " +
+        "Downgrade to prior versions is no longer possible.  Please either " +
+        "restore compatibility, or if the incompatibility is intentional, " +
+        "then update this assertion.", expectedMinCompatLV, actualMinCompatLV);
+  }
+
   /**
    * Given feature {@code f}, ensures the layout version of that feature
    * supports all the features supported by it's ancestor.

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

@@ -203,7 +203,7 @@ public class CreateEditsLog {
 
     FileNameGenerator nameGenerator = new FileNameGenerator(BASE_PATH, 100);
     FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
-    editLog.openForWrite();
+    editLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     addFiles(editLog, numFiles, replication, numBlocksPerFile, startingBlockId,
              blockSize, nameGenerator);
     editLog.logSync();

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

@@ -213,7 +213,7 @@ public abstract class FSImageTestUtil {
       long firstTxId, long newInodeId) throws IOException {
     FSEditLog editLog = FSImageTestUtil.createStandaloneEditLog(editsLogDir);
     editLog.setNextTxId(firstTxId);
-    editLog.openForWrite();
+    editLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     
     PermissionStatus perms = PermissionStatus.createImmutable("fakeuser", "fakegroup",
         FsPermission.createImmutable((short)0755));

+ 8 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -297,7 +297,7 @@ public class TestEditLog {
       editLog.logSetReplication("fakefile", (short) 1);
       editLog.logSync();
       
-      editLog.rollEditLog();
+      editLog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
 
       assertExistsInStorageDirs(
           cluster, NameNodeDirType.EDITS,
@@ -370,7 +370,7 @@ public class TestEditLog {
       
       // Roll log so new output buffer size takes effect
       // we should now be writing to edits_inprogress_3
-      fsimage.rollEditLog();
+      fsimage.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     
       // Remember the current lastInodeId and will reset it back to test
       // loading editlog segments.The transactions in the following allocate new
@@ -401,7 +401,7 @@ public class TestEditLog {
       trans.run();
 
       // Roll another time to finalize edits_inprogress_3
-      fsimage.rollEditLog();
+      fsimage.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
       long expectedTxns = ((NUM_THREADS+1) * 2 * NUM_TRANSACTIONS) + 2; // +2 for start/end txns
    
@@ -940,7 +940,7 @@ public class TestEditLog {
     FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
     try {
       FileUtil.setWritable(logDir, false);
-      log.openForWrite();
+      log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       fail("Did no throw exception on only having a bad dir");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(
@@ -965,7 +965,7 @@ public class TestEditLog {
         new byte[500]);
     
     try {
-      log.openForWrite();
+      log.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       NameNodeMetrics mockMetrics = Mockito.mock(NameNodeMetrics.class);
       log.setMetricsForTests(mockMetrics);
 
@@ -1139,7 +1139,7 @@ public class TestEditLog {
     // logGenerationStamp is used, simply because it doesn't 
     // require complex arguments.
     editlog.initJournalsForWrite();
-    editlog.openForWrite();
+    editlog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
     for (int i = 2; i < TXNS_PER_ROLL; i++) {
       editlog.logGenerationStampV2((long) 0);
     }
@@ -1151,7 +1151,7 @@ public class TestEditLog {
     // the specified journal is aborted. It will be brought
     // back into rotation automatically by rollEditLog
     for (int i = 0; i < numrolls; i++) {
-      editlog.rollEditLog();
+      editlog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       
       editlog.logGenerationStampV2((long) i);
       editlog.logSync();
@@ -1485,7 +1485,7 @@ public class TestEditLog {
             cluster, NameNodeDirType.EDITS,
             NNStorage.getInProgressEditsFileName((i * 3) + 1));
         editLog.logSync();
-        editLog.rollEditLog();
+        editLog.rollEditLog(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
         assertExistsInStorageDirs(
             cluster, NameNodeDirType.EDITS,
             NNStorage.getFinalizedEditsFileName((i * 3) + 1, (i * 3) + 3));

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

@@ -285,7 +285,7 @@ public class TestFSEditLogLoader {
       // FSEditLog#endCurrentLogSegment.  For testing purposes, we
       // disable that here.
       doNothing().when(spyLog).endCurrentLogSegment(true);
-      spyLog.openForWrite();
+      spyLog.openForWrite(NameNodeLayoutVersion.CURRENT_LAYOUT_VERSION);
       assertTrue("should exist: " + inProgressFile, inProgressFile.exists());
       
       for (int i = 0; i < numTx; i++) {