Forráskód Böngészése

HADOOP-11852. Disable symlinks in trunk.

Andrew Wang 10 éve
szülő
commit
26971e52ae
15 módosított fájl, 67 hozzáadás és 2 törlés
  1. 2 0
      hadoop-common-project/hadoop-common/CHANGES.txt
  2. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java
  3. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java
  4. 15 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java
  5. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java
  6. 4 0
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java
  7. 4 0
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/SymlinkBaseTest.java
  8. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContextResolveAfs.java
  9. 3 1
      hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java
  10. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  11. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
  12. 5 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
  13. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  14. 3 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java
  15. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java

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

@@ -201,6 +201,8 @@ Trunk (Unreleased)
     HADOOP-11850. Typos in hadoop-common java docs. (Surendra Singh Lilhore
     HADOOP-11850. Typos in hadoop-common java docs. (Surendra Singh Lilhore
     via jghoman)
     via jghoman)
 
 
+    HADOOP-11852. Disable symlinks in trunk.
+
   BUG FIXES
   BUG FIXES
 
 
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1
     HADOOP-11473. test-patch says "-1 overall" even when all checks are +1

+ 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

@@ -1431,11 +1431,15 @@ public 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

@@ -3296,4 +3296,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

@@ -775,9 +775,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;

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestFileContextResolveAfs.java

@@ -31,7 +31,9 @@ import org.junit.Test;
  * Tests resolution of AbstractFileSystems for a given path with symlinks.
  * Tests resolution of AbstractFileSystems for a given path with symlinks.
  */
  */
 public class TestFileContextResolveAfs {
 public class TestFileContextResolveAfs {
-  
+  static {
+    FileSystem.enableSymlinks();
+  }
   private static String TEST_ROOT_DIR_LOCAL
   private static String TEST_ROOT_DIR_LOCAL
     = System.getProperty("test.build.data","/tmp");
     = System.getProperty("test.build.data","/tmp");
   
   

+ 3 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestStat.java

@@ -32,7 +32,9 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
 
 
 public class TestStat extends FileSystemTestHelper {
 public class TestStat extends FileSystemTestHelper {
-
+  static {
+    FileSystem.enableSymlinks();
+  }
   private static Stat stat;
   private static Stat stat;
 
 
   @BeforeClass
   @BeforeClass

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

@@ -1286,12 +1286,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

@@ -33,6 +33,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.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.HdfsConstantsClient;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -630,6 +631,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

@@ -42,6 +42,7 @@ import org.apache.commons.logging.Log;
 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;
@@ -723,6 +724,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();
@@ -836,6 +838,9 @@ public class FSImageFormat {
       return dir;
       return 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);

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

@@ -147,6 +147,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 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;
@@ -2131,6 +2132,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void createSymlink(String target, String link,
   void createSymlink(String target, String link,
       PermissionStatus dirPerms, boolean createParent, boolean logRetryCache)
       PermissionStatus dirPerms, boolean createParent, boolean logRetryCache)
       throws IOException {
       throws IOException {
+    if (!FileSystem.areSymlinksEnabled()) {
+      throw new UnsupportedOperationException("Symlinks not supported");
+    }
     waitForLoadingFSImage();
     waitForLoadingFSImage();
     HdfsFileStatus auditStat = null;
     HdfsFileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     checkOperation(OperationCategory.WRITE);

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

@@ -769,6 +769,9 @@ public class MiniDFSCluster {
     try {
     try {
       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

@@ -71,6 +71,10 @@ import org.mockito.Mockito;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableList;
 
 
 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;