浏览代码

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
 
+    HADOOP-10020. Disable symlinks temporarily (cmccabe)
+
   NEW FEATURES
 
   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 ("
               + 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) {
           throw new IOException("Possible cyclic loop while " +
                                 "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
    * @throws IOException If an I/O error occurred
    */
+  @SuppressWarnings("deprecation")
   public void createSymlink(final Path target, final Path link,
       final boolean createParent) throws AccessControlException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       IOException { 
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     final Path nonRelLink = fixRelativePart(link);
     new FSLinkResolver<Void>() {
       @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());
     }
   }
+
+  // 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
               + ").", e);
         }
+        if (!FileSystem.areSymlinksEnabled()) {
+          throw new IOException("Symlink resolution is disabled in" +
+              " this version of Hadoop.");
+        }
         if (count++ > FsConstants.MAX_PATH_LINKS) {
           throw new IOException("Possible cyclic loop while " +
                                 "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;
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public void createSymlink(Path target, Path link, boolean createParent)
       throws IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     final String targetScheme = target.toUri().getScheme();
     if (targetScheme != null && !"file".equals(targetScheme)) {
       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
  */
 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 int  blockSize =  8192;
   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);
   }
 
+  @SuppressWarnings("deprecation")
   @Override
   public void createSymlink(final Path target, final Path link,
       final boolean createParent) throws AccessControlException,
       FileAlreadyExistsException, FileNotFoundException,
       ParentNotDirectoryException, UnsupportedFileSystemException, 
       IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     statistics.incrementWriteOps(1);
     final Path absF = fixRelativePart(link);
     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.hadoop.classification.InterfaceAudience;
 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.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
@@ -501,6 +502,9 @@ public class FSEditLogLoader {
       break;
     }
     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;
       inodeId = getAndUpdateLastInodeId(symlinkOp.inodeId, logVersion,
           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.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
@@ -632,6 +633,7 @@ public class FSImageFormat {
    * @param counter Counter to increment for namenode startup progress
    * @return an inode
    */
+  @SuppressWarnings("deprecation")
   INode loadINode(final byte[] localName, boolean isSnapshotINode,
       DataInput in, Counter counter) throws IOException {
     final int imgVersion = getLayoutVersion();
@@ -740,6 +742,9 @@ public class FSImageFormat {
           : dir;
     } else if (numBlocks == -2) {
       //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 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.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsServerDefaults;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.Options;
@@ -1853,9 +1854,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   /**
    * Create a symbolic link.
    */
+  @SuppressWarnings("deprecation")
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent) 
       throws IOException, UnresolvedLinkException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     if (!DFSUtil.isValidName(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 {
     ExitUtil.disableSystemExit();
 
+    // Re-enable symlinks for tests, see HADOOP-10020 and HADOOP-10052
+    FileSystem.enableSymlinks();
+
     synchronized (MiniDFSCluster.class) {
       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;
 
 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);
 
   static final short BLOCKBITS = 48;