Browse Source

HDFS-5709. Improve NameNode upgrade with existing reserved paths and path components. Contributed by Andrew Wang.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1564647 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 years ago
parent
commit
852f941b3c

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

@@ -48,6 +48,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5791. TestHttpsFileSystem should use a random port to avoid binding
     error during testing (Haohui Mai via brandonli)
 
+    HDFS-5709. Improve NameNode upgrade with existing reserved paths and path
+    components. (Andrew Wang via atm)
+
 Release 2.3.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 60 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -262,6 +262,47 @@ public class DFSUtil {
     return true;
   }
 
+  /**
+   * Checks if a string is a valid path component. For instance, components
+   * cannot contain a ":" or "/", and cannot be equal to a reserved component
+   * like ".snapshot".
+   * <p>
+   * The primary use of this method is for validating paths when loading the
+   * FSImage. During normal NN operation, paths are sometimes allowed to
+   * contain reserved components.
+   * 
+   * @return If component is valid
+   */
+  public static boolean isValidNameForComponent(String component) {
+    if (component.equals(".") ||
+        component.equals("..") ||
+        component.indexOf(":") >= 0 ||
+        component.indexOf("/") >= 0) {
+      return false;
+    }
+    return !isReservedPathComponent(component);
+  }
+
+
+  /**
+   * Returns if the component is reserved.
+   * 
+   * <p>
+   * Note that some components are only reserved under certain directories, e.g.
+   * "/.reserved" is reserved, while "/hadoop/.reserved" is not.
+   * 
+   * @param component
+   * @return if the component is reserved
+   */
+  public static boolean isReservedPathComponent(String component) {
+    for (String reserved : HdfsConstants.RESERVED_PATH_COMPONENTS) {
+      if (component.equals(reserved)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   /**
    * Converts a byte array to a string using UTF8 encoding.
    */
@@ -313,7 +354,25 @@ public class DFSUtil {
     }
     return result.toString();
   }
-  
+
+  /**
+   * Converts a list of path components into a path using Path.SEPARATOR.
+   * 
+   * @param components Path components
+   * @return Combined path as a UTF-8 string
+   */
+  public static String strings2PathString(String[] components) {
+    if (components.length == 0) {
+      return "";
+    }
+    if (components.length == 1) {
+      if (components[0] == null || components[0].isEmpty()) {
+        return Path.SEPARATOR;
+      }
+    }
+    return Joiner.on(Path.SEPARATOR).join(components);
+  }
+
   /**
    * Given a list of path components returns a byte array
    */

+ 12 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -22,6 +22,7 @@ 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.namenode.FSDirectory;
 
 /************************************
  * Some handy constants
@@ -108,7 +109,17 @@ public class HdfsConstants {
    */
   public static final int LAYOUT_VERSION = LayoutVersion
       .getCurrentLayoutVersion();
-  
+
+  /**
+   * Path components that are reserved in HDFS.
+   * <p>
+   * .reserved is only reserved under root ("/").
+   */
+  public static final String[] RESERVED_PATH_COMPONENTS = new String[] {
+    HdfsConstants.DOT_SNAPSHOT_DIR,
+    FSDirectory.DOT_RESERVED_STRING
+  };
+
   /**
    * A special path component contained in the path for a snapshot file/dir
    */

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java

@@ -59,7 +59,8 @@ public final class HdfsServerConstants {
     INITIALIZESHAREDEDITS("-initializeSharedEdits"),
     RECOVER  ("-recover"),
     FORCE("-force"),
-    NONINTERACTIVE("-nonInteractive");
+    NONINTERACTIVE("-nonInteractive"),
+    RENAMERESERVED("-renameReserved");
     
     private final String name;
     

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

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.apache.hadoop.hdfs.server.namenode.FSImageFormat.renameReservedPathsOnUpgrade;
 import static org.apache.hadoop.util.Time.now;
 
 import java.io.FilterInputStream;
@@ -295,8 +296,10 @@ public class FSEditLogLoader {
     switch (op.opCode) {
     case OP_ADD: {
       AddCloseOp addCloseOp = (AddCloseOp)op;
+      final String path =
+          renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + addCloseOp.blocks.length +
             " clientHolder " + addCloseOp.clientName +
             " clientMachine " + addCloseOp.clientMachine);
@@ -307,9 +310,9 @@ public class FSEditLogLoader {
       // 3. OP_ADD to open file for append
 
       // See if the file already exists (persistBlocks call)
-      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(path);
       final INodeFile oldFile = INodeFile.valueOf(
-          iip.getINode(0), addCloseOp.path, true);
+          iip.getINode(0), path, true);
       INodeFile newFile = oldFile;
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
         // versions > 0 support per file replication
@@ -322,10 +325,10 @@ public class FSEditLogLoader {
         inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
             lastInodeId);
         newFile = fsDir.unprotectedAddFile(inodeId,
-            addCloseOp.path, addCloseOp.permissions, replication,
+            path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.clientName, addCloseOp.clientMachine);
-        fsNamesys.leaseManager.addLease(addCloseOp.clientName, addCloseOp.path);
+        fsNamesys.leaseManager.addLease(addCloseOp.clientName, path);
 
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
@@ -341,11 +344,11 @@ public class FSEditLogLoader {
             FSNamesystem.LOG.debug("Reopening an already-closed file " +
                 "for append");
           }
-          LocatedBlock lb = fsNamesys.prepareFileForWrite(addCloseOp.path,
+          LocatedBlock lb = fsNamesys.prepareFileForWrite(path,
               oldFile, addCloseOp.clientName, addCloseOp.clientMachine, null,
               false, iip.getLatestSnapshot(), false);
-          newFile = INodeFile.valueOf(fsDir.getINode(addCloseOp.path),
-              addCloseOp.path, true);
+          newFile = INodeFile.valueOf(fsDir.getINode(path),
+              path, true);
           
           // add the op into retry cache is necessary
           if (toAddRetryCache) {
@@ -366,16 +369,17 @@ public class FSEditLogLoader {
     }
     case OP_CLOSE: {
       AddCloseOp addCloseOp = (AddCloseOp)op;
-      
+      final String path =
+          renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + addCloseOp.blocks.length +
             " clientHolder " + addCloseOp.clientName +
             " clientMachine " + addCloseOp.clientMachine);
       }
 
-      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
-      final INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(path);
+      final INodeFile oldFile = INodeFile.valueOf(iip.getINode(0), path);
 
       // Update the salient file attributes.
       oldFile.setAccessTime(addCloseOp.atime, null, fsDir.getINodeMap());
@@ -389,26 +393,28 @@ public class FSEditLogLoader {
         // could show up twice in a row. But after that version, this
         // should be fixed, so we should treat it as an error.
         throw new IOException(
-            "File is not under construction: " + addCloseOp.path);
+            "File is not under construction: " + path);
       }
       // One might expect that you could use removeLease(holder, path) here,
       // but OP_CLOSE doesn't serialize the holder. So, remove by path.
       if (oldFile.isUnderConstruction()) {
         INodeFileUnderConstruction ucFile = (INodeFileUnderConstruction) oldFile;
-        fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
+        fsNamesys.leaseManager.removeLeaseWithPrefixPath(path);
         INodeFile newFile = ucFile.toINodeFile(ucFile.getModificationTime());
-        fsDir.unprotectedReplaceINodeFile(addCloseOp.path, ucFile, newFile);
+        fsDir.unprotectedReplaceINodeFile(path, ucFile, newFile);
       }
       break;
     }
     case OP_UPDATE_BLOCKS: {
       UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
+      final String path =
+          renameReservedPathsOnUpgrade(updateOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + updateOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + updateOp.blocks.length);
       }
-      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(updateOp.path),
-          updateOp.path);
+      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path),
+          path);
       // Update in-memory data structures
       updateBlocks(fsDir, updateOp, oldFile);
       
@@ -419,7 +425,7 @@ public class FSEditLogLoader {
     }
     case OP_ADD_BLOCK: {
       AddBlockOp addBlockOp = (AddBlockOp) op;
-      String path = addBlockOp.getPath();
+      String path = renameReservedPathsOnUpgrade(addBlockOp.getPath(), logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
@@ -433,14 +439,20 @@ public class FSEditLogLoader {
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       short replication = fsNamesys.getBlockManager().adjustReplication(
           setReplicationOp.replication);
-      fsDir.unprotectedSetReplication(setReplicationOp.path,
+      fsDir.unprotectedSetReplication(
+          renameReservedPathsOnUpgrade(setReplicationOp.path, logVersion),
                                       replication, null);
       break;
     }
     case OP_CONCAT_DELETE: {
       ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
-      fsDir.unprotectedConcat(concatDeleteOp.trg, concatDeleteOp.srcs,
-          concatDeleteOp.timestamp);
+      String trg = renameReservedPathsOnUpgrade(concatDeleteOp.trg, logVersion);
+      String[] srcs = new String[concatDeleteOp.srcs.length];
+      for (int i=0; i<srcs.length; i++) {
+        srcs[i] =
+            renameReservedPathsOnUpgrade(concatDeleteOp.srcs[i], logVersion);
+      }
+      fsDir.unprotectedConcat(trg, srcs, concatDeleteOp.timestamp);
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(concatDeleteOp.rpcClientId,
@@ -450,7 +462,9 @@ public class FSEditLogLoader {
     }
     case OP_RENAME_OLD: {
       RenameOldOp renameOp = (RenameOldOp)op;
-      fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
+      final String src = renameReservedPathsOnUpgrade(renameOp.src, logVersion);
+      final String dst = renameReservedPathsOnUpgrade(renameOp.dst, logVersion);
+      fsDir.unprotectedRenameTo(src, dst,
                                 renameOp.timestamp);
       
       if (toAddRetryCache) {
@@ -460,7 +474,9 @@ public class FSEditLogLoader {
     }
     case OP_DELETE: {
       DeleteOp deleteOp = (DeleteOp)op;
-      fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
+      fsDir.unprotectedDelete(
+          renameReservedPathsOnUpgrade(deleteOp.path, logVersion),
+          deleteOp.timestamp);
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(deleteOp.rpcClientId, deleteOp.rpcCallId);
@@ -471,8 +487,9 @@ public class FSEditLogLoader {
       MkdirOp mkdirOp = (MkdirOp)op;
       inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
           lastInodeId);
-      fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
-                             mkdirOp.timestamp);
+      fsDir.unprotectedMkdir(inodeId,
+          renameReservedPathsOnUpgrade(mkdirOp.path, logVersion),
+          mkdirOp.permissions, mkdirOp.timestamp);
       break;
     }
     case OP_SET_GENSTAMP_V1: {
@@ -482,44 +499,46 @@ public class FSEditLogLoader {
     }
     case OP_SET_PERMISSIONS: {
       SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
-      fsDir.unprotectedSetPermission(setPermissionsOp.src,
-                                     setPermissionsOp.permissions);
+      fsDir.unprotectedSetPermission(
+          renameReservedPathsOnUpgrade(setPermissionsOp.src, logVersion),
+          setPermissionsOp.permissions);
       break;
     }
     case OP_SET_OWNER: {
       SetOwnerOp setOwnerOp = (SetOwnerOp)op;
-      fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
-                                setOwnerOp.groupname);
+      fsDir.unprotectedSetOwner(
+          renameReservedPathsOnUpgrade(setOwnerOp.src, logVersion),
+          setOwnerOp.username, setOwnerOp.groupname);
       break;
     }
     case OP_SET_NS_QUOTA: {
       SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
-      fsDir.unprotectedSetQuota(setNSQuotaOp.src,
-                                setNSQuotaOp.nsQuota,
-                                HdfsConstants.QUOTA_DONT_SET);
+      fsDir.unprotectedSetQuota(
+          renameReservedPathsOnUpgrade(setNSQuotaOp.src, logVersion),
+          setNSQuotaOp.nsQuota, HdfsConstants.QUOTA_DONT_SET);
       break;
     }
     case OP_CLEAR_NS_QUOTA: {
       ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
-      fsDir.unprotectedSetQuota(clearNSQuotaOp.src,
-                                HdfsConstants.QUOTA_RESET,
-                                HdfsConstants.QUOTA_DONT_SET);
+      fsDir.unprotectedSetQuota(
+          renameReservedPathsOnUpgrade(clearNSQuotaOp.src, logVersion),
+          HdfsConstants.QUOTA_RESET, HdfsConstants.QUOTA_DONT_SET);
       break;
     }
 
     case OP_SET_QUOTA:
       SetQuotaOp setQuotaOp = (SetQuotaOp)op;
-      fsDir.unprotectedSetQuota(setQuotaOp.src,
-                                setQuotaOp.nsQuota,
-                                setQuotaOp.dsQuota);
+      fsDir.unprotectedSetQuota(
+          renameReservedPathsOnUpgrade(setQuotaOp.src, logVersion),
+          setQuotaOp.nsQuota, setQuotaOp.dsQuota);
       break;
 
     case OP_TIMES: {
       TimesOp timesOp = (TimesOp)op;
 
-      fsDir.unprotectedSetTimes(timesOp.path,
-                                timesOp.mtime,
-                                timesOp.atime, true);
+      fsDir.unprotectedSetTimes(
+          renameReservedPathsOnUpgrade(timesOp.path, logVersion),
+          timesOp.mtime, timesOp.atime, true);
       break;
     }
     case OP_SYMLINK: {
@@ -529,9 +548,10 @@ public class FSEditLogLoader {
       SymlinkOp symlinkOp = (SymlinkOp)op;
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
           lastInodeId);
-      fsDir.unprotectedAddSymlink(inodeId, symlinkOp.path,
-                                  symlinkOp.value, symlinkOp.mtime, 
-                                  symlinkOp.atime, symlinkOp.permissionStatus);
+      fsDir.unprotectedAddSymlink(inodeId,
+          renameReservedPathsOnUpgrade(symlinkOp.path, logVersion),
+          symlinkOp.value, symlinkOp.mtime, symlinkOp.atime,
+          symlinkOp.permissionStatus);
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(symlinkOp.rpcClientId, symlinkOp.rpcCallId);
@@ -540,8 +560,10 @@ public class FSEditLogLoader {
     }
     case OP_RENAME: {
       RenameOp renameOp = (RenameOp)op;
-      fsDir.unprotectedRenameTo(renameOp.src, renameOp.dst,
-                                renameOp.timestamp, renameOp.options);
+      fsDir.unprotectedRenameTo(
+          renameReservedPathsOnUpgrade(renameOp.src, logVersion),
+          renameReservedPathsOnUpgrade(renameOp.dst, logVersion),
+          renameOp.timestamp, renameOp.options);
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(renameOp.rpcClientId, renameOp.rpcCallId);
@@ -584,11 +606,13 @@ public class FSEditLogLoader {
 
       Lease lease = fsNamesys.leaseManager.getLease(
           reassignLeaseOp.leaseHolder);
+      final String path =
+          renameReservedPathsOnUpgrade(reassignLeaseOp.path, logVersion);
       INodeFileUnderConstruction pendingFile =
           INodeFileUnderConstruction.valueOf( 
-              fsDir.getINode(reassignLeaseOp.path), reassignLeaseOp.path);
+              fsDir.getINode(path), path);
       fsNamesys.reassignLeaseInternal(lease,
-          reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
+          path, reassignLeaseOp.newHolder, pendingFile);
       break;
     }
     case OP_START_LOG_SEGMENT:
@@ -598,8 +622,11 @@ public class FSEditLogLoader {
     }
     case OP_CREATE_SNAPSHOT: {
       CreateSnapshotOp createSnapshotOp = (CreateSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(createSnapshotOp.snapshotRoot,
+              logVersion);
       String path = fsNamesys.getSnapshotManager().createSnapshot(
-          createSnapshotOp.snapshotRoot, createSnapshotOp.snapshotName);
+          snapshotRoot, createSnapshotOp.snapshotName);
       if (toAddRetryCache) {
         fsNamesys.addCacheEntryWithPayload(createSnapshotOp.rpcClientId,
             createSnapshotOp.rpcCallId, path);
@@ -610,8 +637,11 @@ public class FSEditLogLoader {
       DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().deleteSnapshot(
-          deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
+          snapshotRoot, deleteSnapshotOp.snapshotName,
           collectedBlocks, removedINodes);
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
@@ -626,8 +656,11 @@ public class FSEditLogLoader {
     }
     case OP_RENAME_SNAPSHOT: {
       RenameSnapshotOp renameSnapshotOp = (RenameSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(renameSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().renameSnapshot(
-          renameSnapshotOp.snapshotRoot, renameSnapshotOp.snapshotOldName,
+          snapshotRoot, renameSnapshotOp.snapshotOldName,
           renameSnapshotOp.snapshotNewName);
       
       if (toAddRetryCache) {
@@ -638,14 +671,19 @@ public class FSEditLogLoader {
     }
     case OP_ALLOW_SNAPSHOT: {
       AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(allowSnapshotOp.snapshotRoot, logVersion);
       fsNamesys.getSnapshotManager().setSnapshottable(
-          allowSnapshotOp.snapshotRoot, false);
+          snapshotRoot, false);
       break;
     }
     case OP_DISALLOW_SNAPSHOT: {
       DisallowSnapshotOp disallowSnapshotOp = (DisallowSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(disallowSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().resetSnapshottable(
-          disallowSnapshotOp.snapshotRoot);
+          snapshotRoot);
       break;
     }
     case OP_SET_GENSTAMP_V2: {

+ 169 - 10
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -32,12 +32,13 @@ import java.security.DigestOutputStream;
 import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -46,13 +47,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.hdfs.DFSUtil;
 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.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.INodeDirectorySnapshottable;
@@ -70,6 +73,10 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
 /**
  * Contains inner classes for reading or writing the on-disk format for
@@ -407,7 +414,8 @@ public class FSImageFormat {
     }
     
   /** 
-   * load fsimage files assuming only local names are stored
+   * load fsimage files assuming only local names are stored. Used when
+   * snapshots are not supported by the layout version.
    *   
    * @param numFiles number of files expected to be read
    * @param in image input stream
@@ -523,6 +531,8 @@ public class FSImageFormat {
     */
    private int loadDirectory(DataInput in, Counter counter) throws IOException {
      String parentPath = FSImageSerialization.readString(in);
+     // Rename .snapshot paths if we're doing an upgrade
+     parentPath = renameReservedPathsOnUpgrade(parentPath, getLayoutVersion());
      final INodeDirectory parent = INodeDirectory.valueOf(
          namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
      return loadChildren(parent, in, counter);
@@ -582,11 +592,9 @@ public class FSImageFormat {
    */
   private void addToParent(INodeDirectory parent, INode child) {
     FSDirectory fsDir = namesystem.dir;
-    if (parent == fsDir.rootDir && FSDirectory.isReservedName(child)) {
-        throw new HadoopIllegalArgumentException("File name \""
-            + child.getLocalName() + "\" is reserved. Please "
-            + " change the name of the existing file or directory to another "
-            + "name before upgrading to this release.");
+    if (parent == fsDir.rootDir) {
+        child.setLocalName(renameReservedRootComponentOnUpgrade(
+            child.getLocalNameBytes(), getLayoutVersion()));
     }
     // NOTE: This does not update space counts for parents
     if (!parent.addChild(child)) {
@@ -623,7 +631,9 @@ public class FSImageFormat {
     public INode loadINodeWithLocalName(boolean isSnapshotINode,
         DataInput in, boolean updateINodeMap, Counter counter)
         throws IOException {
-      final byte[] localName = FSImageSerialization.readLocalName(in);
+      byte[] localName = FSImageSerialization.readLocalName(in);
+      localName =
+          renameReservedComponentOnUpgrade(localName, getLayoutVersion());
       INode inode = loadINode(localName, isSnapshotINode, in, counter);
       if (updateINodeMap
           && LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
@@ -948,7 +958,156 @@ public class FSImageFormat {
       return snapshotMap.get(in.readInt());
     }
   }
-  
+
+  @VisibleForTesting
+  public static TreeMap<String, String> renameReservedMap =
+      new TreeMap<String, String>();
+
+  /**
+   * Use the default key-value pairs that will be used to determine how to
+   * rename reserved paths on upgrade.
+   */
+  @VisibleForTesting
+  public static void useDefaultRenameReservedPairs() {
+    renameReservedMap.clear();
+    for (String key: HdfsConstants.RESERVED_PATH_COMPONENTS) {
+      renameReservedMap.put(
+          key,
+          key + "." + LayoutVersion.getCurrentLayoutVersion() + "."
+              + "UPGRADE_RENAMED");
+    }
+  }
+
+  /**
+   * Set the key-value pairs that will be used to determine how to rename
+   * reserved paths on upgrade.
+   */
+  @VisibleForTesting
+  public static void setRenameReservedPairs(String renameReserved) {
+    // Clear and set the default values
+    useDefaultRenameReservedPairs();
+    // Overwrite with provided values
+    setRenameReservedMapInternal(renameReserved);
+  }
+
+  private static void setRenameReservedMapInternal(String renameReserved) {
+    Collection<String> pairs =
+        StringUtils.getTrimmedStringCollection(renameReserved);
+    for (String p : pairs) {
+      String[] pair = StringUtils.split(p, '/', '=');
+      Preconditions.checkArgument(pair.length == 2,
+          "Could not parse key-value pair " + p);
+      String key = pair[0];
+      String value = pair[1];
+      Preconditions.checkArgument(DFSUtil.isReservedPathComponent(key),
+          "Unknown reserved path " + key);
+      Preconditions.checkArgument(DFSUtil.isValidNameForComponent(value),
+          "Invalid rename path for " + key + ": " + value);
+      LOG.info("Will rename reserved path " + key + " to " + value);
+      renameReservedMap.put(key, value);
+    }
+  }
+
+  /**
+   * When upgrading from an old version, the filesystem could contain paths
+   * that are now reserved in the new version (e.g. .snapshot). This renames
+   * these new reserved paths to a user-specified value to avoid collisions
+   * with the reserved name.
+   * 
+   * @param path Old path potentially containing a reserved path
+   * @return New path with reserved path components renamed to user value
+   */
+  static String renameReservedPathsOnUpgrade(String path,
+      final int layoutVersion) {
+    final String oldPath = path;
+    // If any known LVs aren't supported, we're doing an upgrade
+    if (!LayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) {
+      String[] components = INode.getPathNames(path);
+      // Only need to worry about the root directory
+      if (components.length > 1) {
+        components[1] = DFSUtil.bytes2String(
+            renameReservedRootComponentOnUpgrade(
+                DFSUtil.string2Bytes(components[1]),
+                layoutVersion));
+        path = DFSUtil.strings2PathString(components);
+      }
+    }
+    if (!LayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
+      String[] components = INode.getPathNames(path);
+      // Special case the root path
+      if (components.length == 0) {
+        return path;
+      }
+      for (int i=0; i<components.length; i++) {
+        components[i] = DFSUtil.bytes2String(
+            renameReservedComponentOnUpgrade(
+                DFSUtil.string2Bytes(components[i]),
+                layoutVersion));
+      }
+      path = DFSUtil.strings2PathString(components);
+    }
+
+    if (!path.equals(oldPath)) {
+      LOG.info("Upgrade process renamed reserved path " + oldPath + " to "
+          + path);
+    }
+    return path;
+  }
+
+  private final static String RESERVED_ERROR_MSG = 
+      FSDirectory.DOT_RESERVED_PATH_PREFIX + " is a reserved path and "
+      + HdfsConstants.DOT_SNAPSHOT_DIR + " is a reserved path component in"
+      + " this version of HDFS. Please rollback and delete or rename"
+      + " this path, or upgrade with the "
+      + StartupOption.RENAMERESERVED.getName()
+      + " [key-value pairs]"
+      + " option to automatically rename these paths during upgrade.";
+
+  /**
+   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
+   * byte array path component.
+   */
+  private static byte[] renameReservedComponentOnUpgrade(byte[] component,
+      final int layoutVersion) {
+    // If the LV doesn't support snapshots, we're doing an upgrade
+    if (!LayoutVersion.supports(Feature.SNAPSHOT, layoutVersion)) {
+      if (Arrays.equals(component, HdfsConstants.DOT_SNAPSHOT_DIR_BYTES)) {
+        Preconditions.checkArgument(
+            renameReservedMap != null &&
+            renameReservedMap.containsKey(HdfsConstants.DOT_SNAPSHOT_DIR),
+            RESERVED_ERROR_MSG);
+        component =
+            DFSUtil.string2Bytes(renameReservedMap
+                .get(HdfsConstants.DOT_SNAPSHOT_DIR));
+      }
+    }
+    return component;
+  }
+
+  /**
+   * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single
+   * byte array path component.
+   */
+  private static byte[] renameReservedRootComponentOnUpgrade(byte[] component,
+      final int layoutVersion) {
+    // If the LV doesn't support inode IDs, we're doing an upgrade
+    if (!LayoutVersion.supports(Feature.ADD_INODE_ID, layoutVersion)) {
+      if (Arrays.equals(component, FSDirectory.DOT_RESERVED)) {
+        Preconditions.checkArgument(
+            renameReservedMap != null &&
+            renameReservedMap.containsKey(FSDirectory.DOT_RESERVED_STRING),
+            RESERVED_ERROR_MSG);
+        final String renameString = renameReservedMap
+            .get(FSDirectory.DOT_RESERVED_STRING);
+        component =
+            DFSUtil.string2Bytes(renameString);
+        LOG.info("Renamed root path " + FSDirectory.DOT_RESERVED_STRING
+            + " to " + renameString);
+      }
+    }
+    return component;
+  }
+
   /**
    * A one-shot class responsible for writing an image file.
    * The write() function should be called once, after which the getter

+ 32 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -214,7 +214,9 @@ public class NameNode implements NameNodeStatusMXBean {
       + StartupOption.CLUSTERID.getName() + " cid ] ["
       + StartupOption.FORCE.getName() + "] ["
       + StartupOption.NONINTERACTIVE.getName() + "] ] | ["
-      + StartupOption.UPGRADE.getName() + "] | ["
+      + StartupOption.UPGRADE.getName() + 
+        " [" + StartupOption.CLUSTERID.getName() + " cid]" +
+        " [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | ["
       + StartupOption.ROLLBACK.getName() + "] | ["
       + StartupOption.FINALIZE.getName() + "] | ["
       + StartupOption.IMPORT.getName() + "] | ["
@@ -1052,7 +1054,8 @@ public class NameNode implements NameNodeStatusMXBean {
     out.println(USAGE + "\n");
   }
 
-  private static StartupOption parseArguments(String args[]) {
+  @VisibleForTesting
+  static StartupOption parseArguments(String args[]) {
     int argsLen = (args == null) ? 0 : args.length;
     StartupOption startOpt = StartupOption.REGULAR;
     for(int i=0; i < argsLen; i++) {
@@ -1099,11 +1102,33 @@ public class NameNode implements NameNodeStatusMXBean {
         startOpt = StartupOption.CHECKPOINT;
       } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.UPGRADE;
-        // might be followed by two args
-        if (i + 2 < argsLen
-            && args[i + 1].equalsIgnoreCase(StartupOption.CLUSTERID.getName())) {
-          i += 2;
-          startOpt.setClusterId(args[i]);
+        /* Can be followed by CLUSTERID with a required parameter or
+         * RENAMERESERVED with an optional parameter
+         */
+        while (i + 1 < argsLen) {
+          String flag = args[i + 1];
+          if (flag.equalsIgnoreCase(StartupOption.CLUSTERID.getName())) {
+            if (i + 2 < argsLen) {
+              i += 2;
+              startOpt.setClusterId(args[i]);
+            } else {
+              LOG.fatal("Must specify a valid cluster ID after the "
+                  + StartupOption.CLUSTERID.getName() + " flag");
+              return null;
+            }
+          } else if (flag.equalsIgnoreCase(StartupOption.RENAMERESERVED
+              .getName())) {
+            if (i + 2 < argsLen) {
+              FSImageFormat.setRenameReservedPairs(args[i + 2]);
+              i += 2;
+            } else {
+              FSImageFormat.useDefaultRenameReservedPairs();
+              i += 1;
+            }
+          } else {
+            LOG.fatal("Unknown upgrade flag " + flag);
+            return null;
+          }
         }
       } else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.ROLLBACK;

+ 28 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/apt/HdfsUserGuide.apt.vm

@@ -435,7 +435,7 @@ HDFS Users Guide
    state it was in before the upgrade. HDFS upgrade is described in more
    detail in {{{http://wiki.apache.org/hadoop/Hadoop_Upgrade}Hadoop Upgrade}}
    Wiki page. HDFS can have one such backup at a time. Before upgrading,
-   administrators need to remove existing backupusing bin/hadoop dfsadmin
+   administrators need to remove existing backup using bin/hadoop dfsadmin
    <<<-finalizeUpgrade>>> command. The following briefly describes the
    typical upgrade procedure:
 
@@ -459,6 +459,33 @@ HDFS Users Guide
 
           * start the cluster with rollback option. (<<<bin/start-dfs.sh -rollback>>>).
 
+    When upgrading to a new version of HDFS, it is necessary to rename or
+    delete any paths that are reserved in the new version of HDFS. If the
+    NameNode encounters a reserved path during upgrade, it will print an
+    error like the following:
+
+    <<< /.reserved is a reserved path and .snapshot is a
+    reserved path component in this version of HDFS. Please rollback and delete
+    or rename this path, or upgrade with the -renameReserved [key-value pairs]
+    option to automatically rename these paths during upgrade.>>>
+
+    Specifying <<<-upgrade -renameReserved [optional key-value pairs]>>> causes
+    the NameNode to automatically rename any reserved paths found during
+    startup. For example, to rename all paths named <<<.snapshot>>> to
+    <<<.my-snapshot>>> and <<<.reserved>>> to <<<.my-reserved>>>, a user would
+    specify <<<-upgrade -renameReserved
+    .snapshot=.my-snapshot,.reserved=.my-reserved>>>.
+
+    If no key-value pairs are specified with <<<-renameReserved>>>, the
+    NameNode will then suffix reserved paths with
+    <<<.<LAYOUT-VERSION>.UPGRADE_RENAMED>>>, e.g.
+    <<<.snapshot.-51.UPGRADE_RENAMED>>>.
+
+    There are some caveats to this renaming process. It's recommended,
+    if possible, to first <<<hdfs dfsadmin -saveNamespace>>> before upgrading.
+    This is because data inconsistency can result if an edit log operation
+    refers to the destination of an automatically renamed file.
+
 * File Permissions and Security
 
    The file permissions are designed to be similar to file permissions on

+ 14 - 7
hadoop-hdfs-project/hadoop-hdfs/src/site/xdoc/HdfsSnapshots.xml

@@ -20,7 +20,7 @@
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
 
   <properties>
-    <title>HFDS Snapshots</title>
+    <title>HDFS Snapshots</title>
   </properties>
 
   <body>
@@ -99,15 +99,22 @@
     <li>Copying a file from snapshot <code>s0</code>:
       <source>hdfs dfs -cp /foo/.snapshot/s0/bar /tmp</source></li>
   </ul>
-  <p>
-    <b>Note</b> that the name ".snapshot" is now a reserved file name in HDFS
-    so that users cannot create a file/directory with ".snapshot" as the name.
-    If ".snapshot" is used in a previous version of HDFS, it must be renamed before upgrade;
-    otherwise, upgrade will fail. 
-  </p>
   </subsection>
   </section>
 
+  <section name="Upgrading to a version of HDFS with snapshots" id="Upgrade">
+
+  <p>
+    The HDFS snapshot feature introduces a new reserved path name used to
+    interact with snapshots: <tt>.snapshot</tt>. When upgrading from an
+    older version of HDFS, existing paths named <tt>.snapshot</tt> need
+    to first be renamed or deleted to avoid conflicting with the reserved path.
+    See the upgrade section in
+    <a href="HdfsUserGuide.html#Upgrade_and_Rollback">the HDFS user guide</a>
+    for more information.  </p>
+
+  </section>
+
   <section name="Snapshot Operations" id="SnapshotOperations">
   <subsection name="Administrator Operations" id="AdministratorOperations">
   <p>

+ 85 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -27,6 +27,7 @@ import java.io.File;
 import java.io.FileOutputStream;
 import java.io.FileReader;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.TreeMap;
@@ -43,7 +44,9 @@ import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.apache.hadoop.hdfs.server.namenode.FSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.junit.Test;
@@ -67,6 +70,7 @@ public class TestDFSUpgradeFromImage {
   private static final String HADOOP_DFS_DIR_TXT = "hadoop-dfs-dir.txt";
   private static final String HADOOP22_IMAGE = "hadoop-22-dfs-dir.tgz";
   private static final String HADOOP1_BBW_IMAGE = "hadoop1-bbw.tgz";
+  private static final String HADOOP2_RESERVED_IMAGE = "hadoop-2-reserved.tgz";
 
   private static class ReferenceFileInfo {
     String path;
@@ -320,6 +324,87 @@ public class TestDFSUpgradeFromImage {
       assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
     }
   }
+
+  /**
+   * Test upgrade from 2.0 image with a variety of .snapshot and .reserved
+   * paths to test renaming on upgrade
+   */
+  @Test
+  public void testUpgradeFromRel2ReservedImage() throws IOException {
+    unpackStorage(HADOOP2_RESERVED_IMAGE);
+    MiniDFSCluster cluster = null;
+    // Try it once without setting the upgrade flag to ensure it fails
+    try {
+      cluster =
+          new MiniDFSCluster.Builder(new Configuration())
+              .format(false)
+              .startupOption(StartupOption.UPGRADE)
+              .numDataNodes(0).build();
+    } catch (IllegalArgumentException e) {
+      GenericTestUtils.assertExceptionContains(
+          "reserved path component in this version",
+          e);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+    // Try it again with a custom rename string
+    try {
+      FSImageFormat.setRenameReservedPairs(
+          ".snapshot=.user-snapshot," +
+          ".reserved=.my-reserved");
+      cluster =
+          new MiniDFSCluster.Builder(new Configuration())
+              .format(false)
+              .startupOption(StartupOption.UPGRADE)
+              .numDataNodes(0).build();
+      // Make sure the paths were renamed as expected
+      DistributedFileSystem dfs = cluster.getFileSystem();
+      ArrayList<Path> toList = new ArrayList<Path>();
+      ArrayList<String> found = new ArrayList<String>();
+      toList.add(new Path("/"));
+      while (!toList.isEmpty()) {
+        Path p = toList.remove(0);
+        FileStatus[] statuses = dfs.listStatus(p);
+        for (FileStatus status: statuses) {
+          final String path = status.getPath().toUri().getPath();
+          System.out.println("Found path " + path);
+          found.add(path);
+          if (status.isDirectory()) {
+            toList.add(status.getPath());
+          }
+        }
+      }
+      String[] expected = new String[] {
+          "/edits",
+          "/edits/.reserved",
+          "/edits/.user-snapshot",
+          "/edits/.user-snapshot/editsdir",
+          "/edits/.user-snapshot/editsdir/editscontents",
+          "/edits/.user-snapshot/editsdir/editsdir2",
+          "/image",
+          "/image/.reserved",
+          "/image/.user-snapshot",
+          "/image/.user-snapshot/imagedir",
+          "/image/.user-snapshot/imagedir/imagecontents",
+          "/image/.user-snapshot/imagedir/imagedir2",
+          "/.my-reserved",
+          "/.my-reserved/edits-touch",
+          "/.my-reserved/image-touch"
+      };
+
+      for (String s: expected) {
+        assertTrue("Did not find expected path " + s, found.contains(s));
+      }
+      assertEquals("Found an unexpected path while listing filesystem",
+          found.size(), expected.length);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
     
   static void recoverAllLeases(DFSClient dfs, 
       Path path) throws IOException {

+ 88 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeOptionParsing.java

@@ -0,0 +1,88 @@
+package org.apache.hadoop.hdfs.server.namenode;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
+import org.junit.Test;
+
+public class TestNameNodeOptionParsing {
+
+  @Test(timeout = 10000)
+  public void testUpgrade() {
+    StartupOption opt = null;
+    // UPGRADE is set, but nothing else
+    opt = NameNode.parseArguments(new String[] {"-upgrade"});
+    assertEquals(opt, StartupOption.UPGRADE);
+    assertNull(opt.getClusterId());
+    assertTrue(FSImageFormat.renameReservedMap.isEmpty());
+    // cluster ID is set
+    opt = NameNode.parseArguments(new String[] { "-upgrade", "-clusterid",
+        "mycid" });
+    assertEquals(StartupOption.UPGRADE, opt);
+    assertEquals("mycid", opt.getClusterId());
+    assertTrue(FSImageFormat.renameReservedMap.isEmpty());
+    // Everything is set
+    opt = NameNode.parseArguments(new String[] { "-upgrade", "-clusterid",
+        "mycid", "-renameReserved",
+        ".snapshot=.my-snapshot,.reserved=.my-reserved" });
+    assertEquals(StartupOption.UPGRADE, opt);
+    assertEquals("mycid", opt.getClusterId());
+    assertEquals(".my-snapshot",
+        FSImageFormat.renameReservedMap.get(".snapshot"));
+    assertEquals(".my-reserved",
+        FSImageFormat.renameReservedMap.get(".reserved"));
+    // Reset the map
+    FSImageFormat.renameReservedMap.clear();
+    // Everything is set, but in a different order
+    opt = NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved",
+        ".reserved=.my-reserved,.snapshot=.my-snapshot", "-clusterid",
+        "mycid"});
+    assertEquals(StartupOption.UPGRADE, opt);
+    assertEquals("mycid", opt.getClusterId());
+    assertEquals(".my-snapshot",
+        FSImageFormat.renameReservedMap.get(".snapshot"));
+    assertEquals(".my-reserved",
+        FSImageFormat.renameReservedMap.get(".reserved"));
+    // Try the default renameReserved
+    opt = NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved"});
+    assertEquals(StartupOption.UPGRADE, opt);
+    assertEquals(
+        ".snapshot." + LayoutVersion.getCurrentLayoutVersion()
+            + ".UPGRADE_RENAMED",
+        FSImageFormat.renameReservedMap.get(".snapshot"));
+    assertEquals(
+        ".reserved." + LayoutVersion.getCurrentLayoutVersion()
+            + ".UPGRADE_RENAMED",
+        FSImageFormat.renameReservedMap.get(".reserved"));
+
+    // Try some error conditions
+    try {
+      opt =
+          NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved",
+              ".reserved=.my-reserved,.not-reserved=.my-not-reserved" });
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("Unknown reserved path", e);
+    }
+    try {
+      opt =
+          NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved",
+              ".reserved=.my-reserved,.snapshot=.snapshot" });
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("Invalid rename path", e);
+    }
+    try {
+      opt =
+          NameNode.parseArguments(new String[] { "-upgrade", "-renameReserved",
+              ".snapshot=.reserved" });
+    } catch (IllegalArgumentException e) {
+      assertExceptionContains("Invalid rename path", e);
+    }
+    opt = NameNode.parseArguments(new String[] { "-upgrade", "-cid"});
+    assertNull(opt);
+  }
+
+}

BIN
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz