Browse Source

HADOOP-10020. Disable symlinks temporarily (branch-2.1-beta only change). Contributed by Sanjay Radia.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.1-beta@1529712 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 11 years ago
parent
commit
d56ba205cf

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

@@ -19,6 +19,9 @@ Release 2.1.2 - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES
 
 
+    HADOOP-10020. Disable symlinks temporarily (branch-2.1-beta only change)
+    (sanjay via suresh)
+
   NEW FEATURES
   NEW FEATURES
 
 
     HDFS-4817.  Make HDFS advisory caching configurable on a per-file basis.
     HDFS-4817.  Make HDFS advisory caching configurable on a per-file basis.

+ 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.isSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     final Path nonRelLink = fixRelativePart(link);
     final Path nonRelLink = fixRelativePart(link);
     new FSLinkResolver<Void>() {
     new FSLinkResolver<Void>() {
       @Override
       @Override

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

@@ -2807,4 +2807,24 @@ public abstract class FileSystem extends Configured implements Closeable {
                          ": " + pair.getValue());
                          ": " + pair.getValue());
     }
     }
   }
   }
+  
+  // Symlinks are temporarily disabled - see Hadoop-10020
+  private static boolean symlinkEnabled = false;
+  private static Configuration conf = null;
+  
+  @Deprecated
+  @VisibleForTesting
+  public static boolean isSymlinksEnabled() {
+    if (conf == null) {
+      Configuration conf = new Configuration();
+      symlinkEnabled = conf.getBoolean("test.SymlinkEnabledForTesting", false); 
+    }
+    return symlinkEnabled;
+  }
+  
+  @Deprecated
+  @VisibleForTesting
+  public static void enableSymlinks() {
+    symlinkEnabled = true;
+  }
 }
 }

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/resources/core-site.xml

@@ -78,4 +78,9 @@
   <name>nfs3.mountd.port</name>
   <name>nfs3.mountd.port</name>
   <value>4272</value>
   <value>4272</value>
 </property>
 </property>
+
+<property>
+  <name>test.SymlinkEnabledForTesting</name>
+  <value>true</value>
+</property>
 </configuration>
 </configuration>

+ 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.isSymlinksEnabled()) {
+      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

@@ -32,6 +32,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.isSymlinksEnabled()) {
+        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

@@ -40,6 +40,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;
@@ -627,6 +628,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();
@@ -724,6 +726,9 @@ public class FSImageFormat {
           : dir;
           : dir;
     } else if (numBlocks == -2) {
     } else if (numBlocks == -2) {
       //symlink
       //symlink
+      if (!FileSystem.isSymlinksEnabled()) {
+        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

@@ -125,6 +125,7 @@ import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.CreateFlag;
 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;
@@ -1769,9 +1770,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.isSymlinksEnabled()) {
+      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);
     }
     }

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/hdfs-site.xml

@@ -31,4 +31,8 @@
     <value>0</value>
     <value>0</value>
   </property>
   </property>
 
 
+ <property>
+   <name>test.SymlinkEnabledForTesting</name>
+   <value>true</value>
+ </property>
 </configuration>
 </configuration>