浏览代码

HADOOP-10020. disable symlinks temporarily (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1550034 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 年之前
父节点
当前提交
c9ad1bb3aa

+ 2 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -251,6 +251,8 @@ Release 2.2.0 - 2013-10-13
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
+    HADOOP-10020. Disable symlinks temporarily (cmccabe)
+
   NEW FEATURES
   NEW FEATURES
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java

@@ -95,6 +95,10 @@ public abstract class FSLinkResolver<T> {
               + " and symlink resolution is disabled ("
               + " and symlink resolution is disabled ("
               + CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY + ").", e);
               + CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY + ").", e);
         }
         }
+        if (!FileSystem.areSymlinksEnabled()) {
+          throw new IOException("Symlink resolution is disabled in"
+              + " this version of Hadoop.");
+        }
         if (count++ > FsConstants.MAX_PATH_LINKS) {
         if (count++ > FsConstants.MAX_PATH_LINKS) {
           throw new IOException("Possible cyclic loop while " +
           throw new IOException("Possible cyclic loop while " +
                                 "following symbolic link " + path);
                                 "following symbolic link " + path);

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java

@@ -1339,11 +1339,15 @@ public final class FileContext {
    *           <code>target</code> or <code>link</code> is not supported
    *           <code>target</code> or <code>link</code> is not supported
    * @throws IOException If an I/O error occurred
    * @throws IOException If an I/O error occurred
    */
    */
+  @SuppressWarnings("deprecation")
   public void createSymlink(final Path target, final Path link,
   public void createSymlink(final Path target, final Path link,
       final boolean createParent) throws AccessControlException,
       final boolean createParent) throws AccessControlException,
       FileAlreadyExistsException, FileNotFoundException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       IOException { 
       IOException { 
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     final Path nonRelLink = fixRelativePart(link);
     final Path nonRelLink = fixRelativePart(link);
     new FSLinkResolver<Void>() {
     new FSLinkResolver<Void>() {
       @Override
       @Override

+ 15 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java

@@ -2935,4 +2935,19 @@ public abstract class FileSystem extends Configured implements Closeable {
                          ": " + pair.getValue());
                          ": " + pair.getValue());
     }
     }
   }
   }
+
+  // Symlinks are temporarily disabled - see HADOOP-10020 and HADOOP-10052
+  private static boolean symlinksEnabled = false;
+
+  private static Configuration conf = null;
+
+  @VisibleForTesting
+  public static boolean areSymlinksEnabled() {
+    return symlinksEnabled;
+  }
+
+  @VisibleForTesting
+  public static void enableSymlinks() {
+    symlinksEnabled = true;
+  }
 }
 }

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java

@@ -87,6 +87,10 @@ public abstract class FileSystemLinkResolver<T> {
               + CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY
               + CommonConfigurationKeys.FS_CLIENT_RESOLVE_REMOTE_SYMLINKS_KEY
               + ").", e);
               + ").", e);
         }
         }
+        if (!FileSystem.areSymlinksEnabled()) {
+          throw new IOException("Symlink resolution is disabled in" +
+              " this version of Hadoop.");
+        }
         if (count++ > FsConstants.MAX_PATH_LINKS) {
         if (count++ > FsConstants.MAX_PATH_LINKS) {
           throw new IOException("Possible cyclic loop while " +
           throw new IOException("Possible cyclic loop while " +
                                 "following symbolic link " + path);
                                 "following symbolic link " + path);

+ 4 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java

@@ -651,9 +651,13 @@ public class RawLocalFileSystem extends FileSystem {
     return true;
     return true;
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Override
   @Override
   public void createSymlink(Path target, Path link, boolean createParent)
   public void createSymlink(Path target, Path link, boolean createParent)
       throws IOException {
       throws IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     final String targetScheme = target.toUri().getScheme();
     final String targetScheme = target.toUri().getScheme();
     if (targetScheme != null && !"file".equals(targetScheme)) {
     if (targetScheme != null && !"file".equals(targetScheme)) {
       throw new IOException("Unable to create symlink to non-local file "+
       throw new IOException("Unable to create symlink to non-local file "+

+ 4 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/SymlinkBaseTest.java

@@ -36,6 +36,10 @@ import org.junit.After;
  * Base test for symbolic links
  * Base test for symbolic links
  */
  */
 public abstract class SymlinkBaseTest {
 public abstract class SymlinkBaseTest {
+  // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
+  static {
+    FileSystem.enableSymlinks();
+  }
   static final long seed = 0xDEADBEEFL;
   static final long seed = 0xDEADBEEFL;
   static final int  blockSize =  8192;
   static final int  blockSize =  8192;
   static final int  fileSize  = 16384;
   static final int  fileSize  = 16384;

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1118,12 +1118,16 @@ public class DistributedFileSystem extends FileSystem {
     }.resolve(this, absF);
     }.resolve(this, absF);
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Override
   @Override
   public void createSymlink(final Path target, final Path link,
   public void createSymlink(final Path target, final Path link,
       final boolean createParent) throws AccessControlException,
       final boolean createParent) throws AccessControlException,
       FileAlreadyExistsException, FileNotFoundException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       IOException {
       IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     statistics.incrementWriteOps(1);
     statistics.incrementWriteOps(1);
     final Path absF = fixRelativePart(link);
     final Path absF = fixRelativePart(link);
     new FileSystemLinkResolver<Void>() {
     new FileSystemLinkResolver<Void>() {

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

@@ -31,6 +31,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
 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.fs.FileSystem;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.Block;
 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;
@@ -501,6 +502,9 @@ public class FSEditLogLoader {
       break;
       break;
     }
     }
     case OP_SYMLINK: {
     case OP_SYMLINK: {
+      if (!FileSystem.areSymlinksEnabled()) {
+        throw new IOException("Symlinks not supported - please remove symlink before upgrading to this version of HDFS");
+      }
       SymlinkOp symlinkOp = (SymlinkOp)op;
       SymlinkOp symlinkOp = (SymlinkOp)op;
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
           lastInodeId);
           lastInodeId);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -41,6 +41,7 @@ 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;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 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;
@@ -632,6 +633,7 @@ public class FSImageFormat {
    * @param counter Counter to increment for namenode startup progress
    * @param counter Counter to increment for namenode startup progress
    * @return an inode
    * @return an inode
    */
    */
+  @SuppressWarnings("deprecation")
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in, Counter counter) throws IOException {
       DataInput in, Counter counter) throws IOException {
     final int imgVersion = getLayoutVersion();
     final int imgVersion = getLayoutVersion();
@@ -740,6 +742,9 @@ public class FSImageFormat {
           : dir;
           : dir;
     } else if (numBlocks == -2) {
     } else if (numBlocks == -2) {
       //symlink
       //symlink
+      if (!FileSystem.areSymlinksEnabled()) {
+        throw new IOException("Symlinks not supported - please remove symlink before upgrading to this version of HDFS");
+      }
 
 
       final String symlink = Text.readString(in);
       final String symlink = Text.readString(in);
       final PermissionStatus permissions = PermissionStatus.read(in);
       final PermissionStatus permissions = PermissionStatus.read(in);

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -130,6 +130,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
@@ -1853,9 +1854,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
   /**
    * Create a symbolic link.
    * Create a symbolic link.
    */
    */
+  @SuppressWarnings("deprecation")
   void createSymlink(String target, String link,
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent) 
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
       throws IOException, UnresolvedLinkException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     if (!DFSUtil.isValidName(link)) {
     if (!DFSUtil.isValidName(link)) {
       throw new InvalidPathException("Invalid link name: " + link);
       throw new InvalidPathException("Invalid link name: " + link);
     }
     }

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

@@ -603,6 +603,9 @@ public class MiniDFSCluster {
   throws IOException {
   throws IOException {
     ExitUtil.disableSystemExit();
     ExitUtil.disableSystemExit();
 
 
+    // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
+    FileSystem.enableSymlinks();
+
     synchronized (MiniDFSCluster.class) {
     synchronized (MiniDFSCluster.class) {
       instanceId = instanceCount++;
       instanceId = instanceCount++;
     }
     }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

@@ -62,6 +62,10 @@ import org.junit.Test;
 import org.mockito.Mockito;
 import org.mockito.Mockito;
 
 
 public class TestINodeFile {
 public class TestINodeFile {
+  // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
+  static {
+    FileSystem.enableSymlinks();
+  }
   public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
   public static final Log LOG = LogFactory.getLog(TestINodeFile.class);
 
 
   static final short BLOCKBITS = 48;
   static final short BLOCKBITS = 48;