瀏覽代碼

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/trunk@1564645 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 11 年之前
父節點
當前提交
d6bd920bba

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

@@ -334,6 +334,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5791. TestHttpsFileSystem should use a random port to avoid binding
     HDFS-5791. TestHttpsFileSystem should use a random port to avoid binding
     error during testing (Haohui Mai via brandonli)
     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
 Release 2.3.0 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

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

@@ -261,6 +261,47 @@ public class DFSUtil {
     return true;
     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.
    * Converts a byte array to a string using UTF8 encoding.
    */
    */
@@ -312,7 +353,25 @@ public class DFSUtil {
     }
     }
     return result.toString();
     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
    * 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.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 
 
 /************************************
 /************************************
  * Some handy constants
  * Some handy constants
@@ -108,7 +109,17 @@ public class HdfsConstants {
    */
    */
   public static final int LAYOUT_VERSION = LayoutVersion
   public static final int LAYOUT_VERSION = LayoutVersion
       .getCurrentLayoutVersion();
       .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
    * 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"),
     INITIALIZESHAREDEDITS("-initializeSharedEdits"),
     RECOVER  ("-recover"),
     RECOVER  ("-recover"),
     FORCE("-force"),
     FORCE("-force"),
-    NONINTERACTIVE("-nonInteractive");
+    NONINTERACTIVE("-nonInteractive"),
+    RENAMERESERVED("-renameReserved");
     
     
     private final String name;
     private final String name;
     
     

+ 91 - 53
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;
 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 static org.apache.hadoop.util.Time.now;
 
 
 import java.io.FilterInputStream;
 import java.io.FilterInputStream;
@@ -292,8 +293,10 @@ public class FSEditLogLoader {
     switch (op.opCode) {
     switch (op.opCode) {
     case OP_ADD: {
     case OP_ADD: {
       AddCloseOp addCloseOp = (AddCloseOp)op;
       AddCloseOp addCloseOp = (AddCloseOp)op;
+      final String path =
+          renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + addCloseOp.blocks.length +
             " numblocks : " + addCloseOp.blocks.length +
             " clientHolder " + addCloseOp.clientName +
             " clientHolder " + addCloseOp.clientName +
             " clientMachine " + addCloseOp.clientMachine);
             " clientMachine " + addCloseOp.clientMachine);
@@ -304,9 +307,9 @@ public class FSEditLogLoader {
       // 3. OP_ADD to open file for append
       // 3. OP_ADD to open file for append
 
 
       // See if the file already exists (persistBlocks call)
       // 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(
       final INodeFile oldFile = INodeFile.valueOf(
-          iip.getINode(0), addCloseOp.path, true);
+          iip.getINode(0), path, true);
       INodeFile newFile = oldFile;
       INodeFile newFile = oldFile;
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
       if (oldFile == null) { // this is OP_ADD on a new file (case 1)
         // versions > 0 support per file replication
         // versions > 0 support per file replication
@@ -319,10 +322,10 @@ public class FSEditLogLoader {
         inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
         inodeId = getAndUpdateLastInodeId(addCloseOp.inodeId, logVersion,
             lastInodeId);
             lastInodeId);
         newFile = fsDir.unprotectedAddFile(inodeId,
         newFile = fsDir.unprotectedAddFile(inodeId,
-            addCloseOp.path, addCloseOp.permissions, replication,
+            path, addCloseOp.permissions, replication,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.mtime, addCloseOp.atime, addCloseOp.blockSize, true,
             addCloseOp.clientName, addCloseOp.clientMachine);
             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
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
         if (toAddRetryCache) {
@@ -338,11 +341,11 @@ public class FSEditLogLoader {
             FSNamesystem.LOG.debug("Reopening an already-closed file " +
             FSNamesystem.LOG.debug("Reopening an already-closed file " +
                 "for append");
                 "for append");
           }
           }
-          LocatedBlock lb = fsNamesys.prepareFileForWrite(addCloseOp.path,
+          LocatedBlock lb = fsNamesys.prepareFileForWrite(path,
               oldFile, addCloseOp.clientName, addCloseOp.clientMachine, null,
               oldFile, addCloseOp.clientName, addCloseOp.clientMachine, null,
               false, iip.getLatestSnapshotId(), false);
               false, iip.getLatestSnapshotId(), 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
           // add the op into retry cache is necessary
           if (toAddRetryCache) {
           if (toAddRetryCache) {
@@ -363,16 +366,17 @@ public class FSEditLogLoader {
     }
     }
     case OP_CLOSE: {
     case OP_CLOSE: {
       AddCloseOp addCloseOp = (AddCloseOp)op;
       AddCloseOp addCloseOp = (AddCloseOp)op;
-      
+      final String path =
+          renameReservedPathsOnUpgrade(addCloseOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + addCloseOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + addCloseOp.blocks.length +
             " numblocks : " + addCloseOp.blocks.length +
             " clientHolder " + addCloseOp.clientName +
             " clientHolder " + addCloseOp.clientName +
             " clientMachine " + addCloseOp.clientMachine);
             " clientMachine " + addCloseOp.clientMachine);
       }
       }
 
 
-      final INodesInPath iip = fsDir.getLastINodeInPath(addCloseOp.path);
-      final INodeFile file = INodeFile.valueOf(iip.getINode(0), addCloseOp.path);
+      final INodesInPath iip = fsDir.getLastINodeInPath(path);
+      final INodeFile file = INodeFile.valueOf(iip.getINode(0), path);
 
 
       // Update the salient file attributes.
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
@@ -386,24 +390,26 @@ public class FSEditLogLoader {
         // could show up twice in a row. But after that version, this
         // could show up twice in a row. But after that version, this
         // should be fixed, so we should treat it as an error.
         // should be fixed, so we should treat it as an error.
         throw new IOException(
         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,
       // One might expect that you could use removeLease(holder, path) here,
       // but OP_CLOSE doesn't serialize the holder. So, remove by path.
       // but OP_CLOSE doesn't serialize the holder. So, remove by path.
       if (file.isUnderConstruction()) {
       if (file.isUnderConstruction()) {
-        fsNamesys.leaseManager.removeLeaseWithPrefixPath(addCloseOp.path);
+        fsNamesys.leaseManager.removeLeaseWithPrefixPath(path);
         file.toCompleteFile(file.getModificationTime());
         file.toCompleteFile(file.getModificationTime());
       }
       }
       break;
       break;
     }
     }
     case OP_UPDATE_BLOCKS: {
     case OP_UPDATE_BLOCKS: {
       UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
       UpdateBlocksOp updateOp = (UpdateBlocksOp)op;
+      final String path =
+          renameReservedPathsOnUpgrade(updateOp.path, logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
       if (FSNamesystem.LOG.isDebugEnabled()) {
-        FSNamesystem.LOG.debug(op.opCode + ": " + updateOp.path +
+        FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " numblocks : " + updateOp.blocks.length);
             " 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
       // Update in-memory data structures
       updateBlocks(fsDir, updateOp, oldFile);
       updateBlocks(fsDir, updateOp, oldFile);
       
       
@@ -414,7 +420,7 @@ public class FSEditLogLoader {
     }
     }
     case OP_ADD_BLOCK: {
     case OP_ADD_BLOCK: {
       AddBlockOp addBlockOp = (AddBlockOp) op;
       AddBlockOp addBlockOp = (AddBlockOp) op;
-      String path = addBlockOp.getPath();
+      String path = renameReservedPathsOnUpgrade(addBlockOp.getPath(), logVersion);
       if (FSNamesystem.LOG.isDebugEnabled()) {
       if (FSNamesystem.LOG.isDebugEnabled()) {
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
@@ -428,14 +434,20 @@ public class FSEditLogLoader {
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       SetReplicationOp setReplicationOp = (SetReplicationOp)op;
       short replication = fsNamesys.getBlockManager().adjustReplication(
       short replication = fsNamesys.getBlockManager().adjustReplication(
           setReplicationOp.replication);
           setReplicationOp.replication);
-      fsDir.unprotectedSetReplication(setReplicationOp.path,
+      fsDir.unprotectedSetReplication(
+          renameReservedPathsOnUpgrade(setReplicationOp.path, logVersion),
                                       replication, null);
                                       replication, null);
       break;
       break;
     }
     }
     case OP_CONCAT_DELETE: {
     case OP_CONCAT_DELETE: {
       ConcatDeleteOp concatDeleteOp = (ConcatDeleteOp)op;
       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) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(concatDeleteOp.rpcClientId,
         fsNamesys.addCacheEntry(concatDeleteOp.rpcClientId,
@@ -445,7 +457,9 @@ public class FSEditLogLoader {
     }
     }
     case OP_RENAME_OLD: {
     case OP_RENAME_OLD: {
       RenameOldOp renameOp = (RenameOldOp)op;
       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);
                                 renameOp.timestamp);
       
       
       if (toAddRetryCache) {
       if (toAddRetryCache) {
@@ -455,7 +469,9 @@ public class FSEditLogLoader {
     }
     }
     case OP_DELETE: {
     case OP_DELETE: {
       DeleteOp deleteOp = (DeleteOp)op;
       DeleteOp deleteOp = (DeleteOp)op;
-      fsDir.unprotectedDelete(deleteOp.path, deleteOp.timestamp);
+      fsDir.unprotectedDelete(
+          renameReservedPathsOnUpgrade(deleteOp.path, logVersion),
+          deleteOp.timestamp);
       
       
       if (toAddRetryCache) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(deleteOp.rpcClientId, deleteOp.rpcCallId);
         fsNamesys.addCacheEntry(deleteOp.rpcClientId, deleteOp.rpcCallId);
@@ -466,8 +482,9 @@ public class FSEditLogLoader {
       MkdirOp mkdirOp = (MkdirOp)op;
       MkdirOp mkdirOp = (MkdirOp)op;
       inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
       inodeId = getAndUpdateLastInodeId(mkdirOp.inodeId, logVersion,
           lastInodeId);
           lastInodeId);
-      fsDir.unprotectedMkdir(inodeId, mkdirOp.path, mkdirOp.permissions,
-                             mkdirOp.timestamp);
+      fsDir.unprotectedMkdir(inodeId,
+          renameReservedPathsOnUpgrade(mkdirOp.path, logVersion),
+          mkdirOp.permissions, mkdirOp.timestamp);
       break;
       break;
     }
     }
     case OP_SET_GENSTAMP_V1: {
     case OP_SET_GENSTAMP_V1: {
@@ -477,53 +494,56 @@ public class FSEditLogLoader {
     }
     }
     case OP_SET_PERMISSIONS: {
     case OP_SET_PERMISSIONS: {
       SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
       SetPermissionsOp setPermissionsOp = (SetPermissionsOp)op;
-      fsDir.unprotectedSetPermission(setPermissionsOp.src,
-                                     setPermissionsOp.permissions);
+      fsDir.unprotectedSetPermission(
+          renameReservedPathsOnUpgrade(setPermissionsOp.src, logVersion),
+          setPermissionsOp.permissions);
       break;
       break;
     }
     }
     case OP_SET_OWNER: {
     case OP_SET_OWNER: {
       SetOwnerOp setOwnerOp = (SetOwnerOp)op;
       SetOwnerOp setOwnerOp = (SetOwnerOp)op;
-      fsDir.unprotectedSetOwner(setOwnerOp.src, setOwnerOp.username,
-                                setOwnerOp.groupname);
+      fsDir.unprotectedSetOwner(
+          renameReservedPathsOnUpgrade(setOwnerOp.src, logVersion),
+          setOwnerOp.username, setOwnerOp.groupname);
       break;
       break;
     }
     }
     case OP_SET_NS_QUOTA: {
     case OP_SET_NS_QUOTA: {
       SetNSQuotaOp setNSQuotaOp = (SetNSQuotaOp)op;
       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;
       break;
     }
     }
     case OP_CLEAR_NS_QUOTA: {
     case OP_CLEAR_NS_QUOTA: {
       ClearNSQuotaOp clearNSQuotaOp = (ClearNSQuotaOp)op;
       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;
       break;
     }
     }
 
 
     case OP_SET_QUOTA:
     case OP_SET_QUOTA:
       SetQuotaOp setQuotaOp = (SetQuotaOp)op;
       SetQuotaOp setQuotaOp = (SetQuotaOp)op;
-      fsDir.unprotectedSetQuota(setQuotaOp.src,
-                                setQuotaOp.nsQuota,
-                                setQuotaOp.dsQuota);
+      fsDir.unprotectedSetQuota(
+          renameReservedPathsOnUpgrade(setQuotaOp.src, logVersion),
+          setQuotaOp.nsQuota, setQuotaOp.dsQuota);
       break;
       break;
 
 
     case OP_TIMES: {
     case OP_TIMES: {
       TimesOp timesOp = (TimesOp)op;
       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;
       break;
     }
     }
     case OP_SYMLINK: {
     case OP_SYMLINK: {
       SymlinkOp symlinkOp = (SymlinkOp)op;
       SymlinkOp symlinkOp = (SymlinkOp)op;
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
           lastInodeId);
           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) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(symlinkOp.rpcClientId, symlinkOp.rpcCallId);
         fsNamesys.addCacheEntry(symlinkOp.rpcClientId, symlinkOp.rpcCallId);
@@ -532,8 +552,10 @@ public class FSEditLogLoader {
     }
     }
     case OP_RENAME: {
     case OP_RENAME: {
       RenameOp renameOp = (RenameOp)op;
       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) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(renameOp.rpcClientId, renameOp.rpcCallId);
         fsNamesys.addCacheEntry(renameOp.rpcClientId, renameOp.rpcCallId);
@@ -576,10 +598,12 @@ public class FSEditLogLoader {
 
 
       Lease lease = fsNamesys.leaseManager.getLease(
       Lease lease = fsNamesys.leaseManager.getLease(
           reassignLeaseOp.leaseHolder);
           reassignLeaseOp.leaseHolder);
-      INodeFile pendingFile = fsDir.getINode(reassignLeaseOp.path).asFile();
+      final String path =
+          renameReservedPathsOnUpgrade(reassignLeaseOp.path, logVersion);
+      INodeFile pendingFile = fsDir.getINode(path).asFile();
       Preconditions.checkState(pendingFile.isUnderConstruction());
       Preconditions.checkState(pendingFile.isUnderConstruction());
       fsNamesys.reassignLeaseInternal(lease,
       fsNamesys.reassignLeaseInternal(lease,
-          reassignLeaseOp.path, reassignLeaseOp.newHolder, pendingFile);
+          path, reassignLeaseOp.newHolder, pendingFile);
       break;
       break;
     }
     }
     case OP_START_LOG_SEGMENT:
     case OP_START_LOG_SEGMENT:
@@ -589,8 +613,11 @@ public class FSEditLogLoader {
     }
     }
     case OP_CREATE_SNAPSHOT: {
     case OP_CREATE_SNAPSHOT: {
       CreateSnapshotOp createSnapshotOp = (CreateSnapshotOp) op;
       CreateSnapshotOp createSnapshotOp = (CreateSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(createSnapshotOp.snapshotRoot,
+              logVersion);
       String path = fsNamesys.getSnapshotManager().createSnapshot(
       String path = fsNamesys.getSnapshotManager().createSnapshot(
-          createSnapshotOp.snapshotRoot, createSnapshotOp.snapshotName);
+          snapshotRoot, createSnapshotOp.snapshotName);
       if (toAddRetryCache) {
       if (toAddRetryCache) {
         fsNamesys.addCacheEntryWithPayload(createSnapshotOp.rpcClientId,
         fsNamesys.addCacheEntryWithPayload(createSnapshotOp.rpcClientId,
             createSnapshotOp.rpcCallId, path);
             createSnapshotOp.rpcCallId, path);
@@ -601,8 +628,11 @@ public class FSEditLogLoader {
       DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
       DeleteSnapshotOp deleteSnapshotOp = (DeleteSnapshotOp) op;
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       BlocksMapUpdateInfo collectedBlocks = new BlocksMapUpdateInfo();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
       List<INode> removedINodes = new ChunkedArrayList<INode>();
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(deleteSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().deleteSnapshot(
       fsNamesys.getSnapshotManager().deleteSnapshot(
-          deleteSnapshotOp.snapshotRoot, deleteSnapshotOp.snapshotName,
+          snapshotRoot, deleteSnapshotOp.snapshotName,
           collectedBlocks, removedINodes);
           collectedBlocks, removedINodes);
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       fsNamesys.removeBlocksAndUpdateSafemodeTotal(collectedBlocks);
       collectedBlocks.clear();
       collectedBlocks.clear();
@@ -617,8 +647,11 @@ public class FSEditLogLoader {
     }
     }
     case OP_RENAME_SNAPSHOT: {
     case OP_RENAME_SNAPSHOT: {
       RenameSnapshotOp renameSnapshotOp = (RenameSnapshotOp) op;
       RenameSnapshotOp renameSnapshotOp = (RenameSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(renameSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().renameSnapshot(
       fsNamesys.getSnapshotManager().renameSnapshot(
-          renameSnapshotOp.snapshotRoot, renameSnapshotOp.snapshotOldName,
+          snapshotRoot, renameSnapshotOp.snapshotOldName,
           renameSnapshotOp.snapshotNewName);
           renameSnapshotOp.snapshotNewName);
       
       
       if (toAddRetryCache) {
       if (toAddRetryCache) {
@@ -629,14 +662,19 @@ public class FSEditLogLoader {
     }
     }
     case OP_ALLOW_SNAPSHOT: {
     case OP_ALLOW_SNAPSHOT: {
       AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
       AllowSnapshotOp allowSnapshotOp = (AllowSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(allowSnapshotOp.snapshotRoot, logVersion);
       fsNamesys.getSnapshotManager().setSnapshottable(
       fsNamesys.getSnapshotManager().setSnapshottable(
-          allowSnapshotOp.snapshotRoot, false);
+          snapshotRoot, false);
       break;
       break;
     }
     }
     case OP_DISALLOW_SNAPSHOT: {
     case OP_DISALLOW_SNAPSHOT: {
       DisallowSnapshotOp disallowSnapshotOp = (DisallowSnapshotOp) op;
       DisallowSnapshotOp disallowSnapshotOp = (DisallowSnapshotOp) op;
+      final String snapshotRoot =
+          renameReservedPathsOnUpgrade(disallowSnapshotOp.snapshotRoot,
+              logVersion);
       fsNamesys.getSnapshotManager().resetSnapshottable(
       fsNamesys.getSnapshotManager().resetSnapshottable(
-          disallowSnapshotOp.snapshotRoot);
+          snapshotRoot);
       break;
       break;
     }
     }
     case OP_SET_GENSTAMP_V2: {
     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.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.HashMap;
 import java.util.List;
 import java.util.List;
 import java.util.Map;
 import java.util.Map;
+import java.util.TreeMap;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
-import org.apache.hadoop.HadoopIllegalArgumentException;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
@@ -45,13 +46,15 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 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.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 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.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 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.common.InconsistentFSStateException;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.DirectoryWithSnapshotFeature;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
@@ -67,6 +70,10 @@ import org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.hdfs.util.ReadOnlyList;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
 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
  * Contains inner classes for reading or writing the on-disk format for
@@ -405,7 +412,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 numFiles number of files expected to be read
    * @param in image input stream
    * @param in image input stream
@@ -521,6 +529,8 @@ public class FSImageFormat {
     */
     */
    private int loadDirectory(DataInput in, Counter counter) throws IOException {
    private int loadDirectory(DataInput in, Counter counter) throws IOException {
      String parentPath = FSImageSerialization.readString(in);
      String parentPath = FSImageSerialization.readString(in);
+     // Rename .snapshot paths if we're doing an upgrade
+     parentPath = renameReservedPathsOnUpgrade(parentPath, getLayoutVersion());
      final INodeDirectory parent = INodeDirectory.valueOf(
      final INodeDirectory parent = INodeDirectory.valueOf(
          namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
          namesystem.dir.rootDir.getNode(parentPath, true), parentPath);
      return loadChildren(parent, in, counter);
      return loadChildren(parent, in, counter);
@@ -580,11 +590,9 @@ public class FSImageFormat {
    */
    */
   private void addToParent(INodeDirectory parent, INode child) {
   private void addToParent(INodeDirectory parent, INode child) {
     FSDirectory fsDir = namesystem.dir;
     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
     // NOTE: This does not update space counts for parents
     if (!parent.addChild(child)) {
     if (!parent.addChild(child)) {
@@ -621,7 +629,9 @@ public class FSImageFormat {
     public INode loadINodeWithLocalName(boolean isSnapshotINode,
     public INode loadINodeWithLocalName(boolean isSnapshotINode,
         DataInput in, boolean updateINodeMap, Counter counter)
         DataInput in, boolean updateINodeMap, Counter counter)
         throws IOException {
         throws IOException {
-      final byte[] localName = FSImageSerialization.readLocalName(in);
+      byte[] localName = FSImageSerialization.readLocalName(in);
+      localName =
+          renameReservedComponentOnUpgrade(localName, getLayoutVersion());
       INode inode = loadINode(localName, isSnapshotINode, in, counter);
       INode inode = loadINode(localName, isSnapshotINode, in, counter);
       if (updateINodeMap
       if (updateINodeMap
           && LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
           && LayoutVersion.supports(Feature.ADD_INODE_ID, getLayoutVersion())) {
@@ -926,7 +936,156 @@ public class FSImageFormat {
       return snapshotMap.get(in.readInt());
       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.
    * A one-shot class responsible for writing an image file.
    * The write() function should be called once, after which the getter
    * 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

@@ -212,7 +212,9 @@ public class NameNode implements NameNodeStatusMXBean {
       + StartupOption.CLUSTERID.getName() + " cid ] ["
       + StartupOption.CLUSTERID.getName() + " cid ] ["
       + StartupOption.FORCE.getName() + "] ["
       + StartupOption.FORCE.getName() + "] ["
       + StartupOption.NONINTERACTIVE.getName() + "] ] | ["
       + StartupOption.NONINTERACTIVE.getName() + "] ] | ["
-      + StartupOption.UPGRADE.getName() + "] | ["
+      + StartupOption.UPGRADE.getName() + 
+        " [" + StartupOption.CLUSTERID.getName() + " cid]" +
+        " [" + StartupOption.RENAMERESERVED.getName() + "<k-v pairs>] ] | ["
       + StartupOption.ROLLBACK.getName() + "] | ["
       + StartupOption.ROLLBACK.getName() + "] | ["
       + StartupOption.FINALIZE.getName() + "] | ["
       + StartupOption.FINALIZE.getName() + "] | ["
       + StartupOption.IMPORT.getName() + "] | ["
       + StartupOption.IMPORT.getName() + "] | ["
@@ -1056,7 +1058,8 @@ public class NameNode implements NameNodeStatusMXBean {
     out.println(USAGE + "\n");
     out.println(USAGE + "\n");
   }
   }
 
 
-  private static StartupOption parseArguments(String args[]) {
+  @VisibleForTesting
+  static StartupOption parseArguments(String args[]) {
     int argsLen = (args == null) ? 0 : args.length;
     int argsLen = (args == null) ? 0 : args.length;
     StartupOption startOpt = StartupOption.REGULAR;
     StartupOption startOpt = StartupOption.REGULAR;
     for(int i=0; i < argsLen; i++) {
     for(int i=0; i < argsLen; i++) {
@@ -1103,11 +1106,33 @@ public class NameNode implements NameNodeStatusMXBean {
         startOpt = StartupOption.CHECKPOINT;
         startOpt = StartupOption.CHECKPOINT;
       } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
       } else if (StartupOption.UPGRADE.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.UPGRADE;
         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)) {
       } else if (StartupOption.ROLLBACK.getName().equalsIgnoreCase(cmd)) {
         startOpt = StartupOption.ROLLBACK;
         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
    state it was in before the upgrade. HDFS upgrade is described in more
    detail in {{{http://wiki.apache.org/hadoop/Hadoop_Upgrade}Hadoop Upgrade}}
    detail in {{{http://wiki.apache.org/hadoop/Hadoop_Upgrade}Hadoop Upgrade}}
    Wiki page. HDFS can have one such backup at a time. Before upgrading,
    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
    <<<-finalizeUpgrade>>> command. The following briefly describes the
    typical upgrade procedure:
    typical upgrade procedure:
 
 
@@ -459,6 +459,33 @@ HDFS Users Guide
 
 
           * start the cluster with rollback option. (<<<bin/start-dfs.sh -rollback>>>).
           * 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
 * File Permissions and Security
 
 
    The file permissions are designed to be similar to file permissions on
    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">
   xsi:schemaLocation="http://maven.apache.org/XDOC/2.0 http://maven.apache.org/xsd/xdoc-2.0.xsd">
 
 
   <properties>
   <properties>
-    <title>HFDS Snapshots</title>
+    <title>HDFS Snapshots</title>
   </properties>
   </properties>
 
 
   <body>
   <body>
@@ -99,15 +99,22 @@
     <li>Copying a file from snapshot <code>s0</code>:
     <li>Copying a file from snapshot <code>s0</code>:
       <source>hdfs dfs -cp /foo/.snapshot/s0/bar /tmp</source></li>
       <source>hdfs dfs -cp /foo/.snapshot/s0/bar /tmp</source></li>
   </ul>
   </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>
   </subsection>
   </section>
   </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">
   <section name="Snapshot Operations" id="SnapshotOperations">
   <subsection name="Administrator Operations" id="AdministratorOperations">
   <subsection name="Administrator Operations" id="AdministratorOperations">
   <p>
   <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.FileOutputStream;
 import java.io.FileReader;
 import java.io.FileReader;
 import java.io.IOException;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.Iterator;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.TreeMap;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 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.hdfs.server.namenode.FSImageTestUtil;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 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 HADOOP_DFS_DIR_TXT = "hadoop-dfs-dir.txt";
   private static final String HADOOP22_IMAGE = "hadoop-22-dfs-dir.tgz";
   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 HADOOP1_BBW_IMAGE = "hadoop1-bbw.tgz";
+  private static final String HADOOP2_RESERVED_IMAGE = "hadoop-2-reserved.tgz";
 
 
   private static class ReferenceFileInfo {
   private static class ReferenceFileInfo {
     String path;
     String path;
@@ -320,6 +324,87 @@ public class TestDFSUpgradeFromImage {
       assertEquals("Upgrade did not fail with bad MD5", 1, md5failures);
       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, 
   static void recoverAllLeases(DFSClient dfs, 
       Path path) throws IOException {
       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);
+  }
+
+}

二進制
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hadoop-2-reserved.tgz