Преглед на файлове

HADOOP-9355. Abstract Symlink tests to use either FileContext or FileSystem (Andrew Wang via Colin Patrick McCabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1496691 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe преди 12 години
родител
ревизия
7880da2743

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

@@ -25,6 +25,12 @@ Release 2.2.0 - UNRELEASED
 
     HADOOP-9661. Allow metrics sources to be extended. (sandyr via tucu)
 
+    HADOOP-9370.  Write FSWrapper class to wrap FileSystem and FileContext for
+    better test coverage.  (Andrew Wang via Colin Patrick McCabe)
+
+    HADOOP-9355.  Abstract symlink tests to use either FileContext or
+    FileSystem.  (Andrew Wang via Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
   BUG FIXES
@@ -147,9 +153,6 @@ Release 2.1.0-beta - UNRELEASED
 
     HADOOP-9619 Mark stability of .proto files (sanjay Radia)
 
-    HADOOP-9370.  Write FSWrapper class to wrap FileSystem and FileContext for
-    better test coverage.  (Andrew Wang via Colin Patrick McCabe)
-
   OPTIMIZATIONS
 
     HADOOP-9150. Avoid unnecessary DNS resolution attempts for logical URIs

Файловите разлики са ограничени, защото са твърде много
+ 294 - 275
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/SymlinkBaseTest.java


+ 34 - 39
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextSymlink.java → hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFS.java

@@ -17,23 +17,25 @@
  */
 package org.apache.hadoop.fs;
 
-import java.io.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
-import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.FileUtil;
-import static org.junit.Assert.*;
-import static org.junit.Assume.assumeTrue;
 import org.junit.Test;
-import org.junit.Before;
 
 /**
- * Test symbolic links using FileContext and LocalFs.
+ * Test symbolic links using LocalFs.
  */
-public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
+abstract public class TestSymlinkLocalFS extends SymlinkBaseTest {
   
   @Override
   protected String getScheme() {
@@ -42,12 +44,12 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
 
   @Override
   protected String testBaseDir1() throws IOException {
-    return fileContextTestHelper.getAbsoluteTestRootDir(fc)+"/test1";
+    return wrapper.getAbsoluteTestRootDir()+"/test1";
   }
   
   @Override
   protected String testBaseDir2() throws IOException {
-    return fileContextTestHelper.getAbsoluteTestRootDir(fc)+"/test2";
+    return wrapper.getAbsoluteTestRootDir()+"/test2";
   }
 
   @Override
@@ -58,13 +60,6 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
       return null;
     }
   }
-  
-  @Override
-  @Before
-  public void setUp() throws Exception {
-    fc = FileContext.getLocalFSFileContext();
-    super.setUp();
-  }
 
   @Override
   public void testCreateDanglingLink() throws IOException {
@@ -103,32 +98,32 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     super.testStatDanglingLink();
   }
 
-  @Test
+  @Test(timeout=1000)
   /** lstat a non-existant file using a partially qualified path */
   public void testDanglingLinkFilePartQual() throws IOException {
     Path filePartQual = new Path(getScheme()+":///doesNotExist");
     try {
-      fc.getFileLinkStatus(filePartQual);
+      wrapper.getFileLinkStatus(filePartQual);
       fail("Got FileStatus for non-existant file");
     } catch (FileNotFoundException f) {
       // Expected
     }
     try {
-      fc.getLinkTarget(filePartQual);
-      fail("Got link target for non-existant file");      
+      wrapper.getLinkTarget(filePartQual);
+      fail("Got link target for non-existant file");
     } catch (FileNotFoundException f) {
       // Expected
     }
   }
   
-  @Test
+  @Test(timeout=1000)
   /** Stat and lstat a dangling link */
   public void testDanglingLink() throws IOException {
     assumeTrue(!Path.WINDOWS);
-    Path fileAbs  = new Path(testBaseDir1()+"/file");    
-    Path fileQual = new Path(testURI().toString(), fileAbs);    
+    Path fileAbs  = new Path(testBaseDir1()+"/file");
+    Path fileQual = new Path(testURI().toString(), fileAbs);
     Path link     = new Path(testBaseDir1()+"/linkToFile");
-    fc.createSymlink(fileAbs, link, false);
+    wrapper.createSymlink(fileAbs, link, false);
     // Deleting the link using FileContext currently fails because
     // resolve looks up LocalFs rather than RawLocalFs for the path 
     // so we call ChecksumFs delete (which doesn't delete dangling 
@@ -137,15 +132,15 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     // because the link is not dangling.
     //assertTrue(fc.delete(link, false));
     FileUtil.fullyDelete(new File(link.toUri().getPath()));
-    fc.createSymlink(fileAbs, link, false);
+    wrapper.createSymlink(fileAbs, link, false);
     try {
-      fc.getFileStatus(link);
+      wrapper.getFileStatus(link);
       fail("Got FileStatus for dangling link");
     } catch (FileNotFoundException f) {
       // Expected. File's exists method returns false for dangling links
     }
     // We can stat a dangling link
-    FileStatus fsd = fc.getFileLinkStatus(link);
+    FileStatus fsd = wrapper.getFileLinkStatus(link);
     assertEquals(fileQual, fsd.getSymlink());
     assertTrue(fsd.isSymlink());
     assertFalse(fsd.isDirectory());
@@ -166,10 +161,10 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     }
     // Creating the file makes the link work
     createAndWriteFile(fileAbs);
-    fc.getFileStatus(link);
+    wrapper.getFileStatus(link);
   }
 
-  @Test
+  @Test(timeout=1000)
   /** 
    * Test getLinkTarget with a partially qualified target. 
    * NB: Hadoop does not support fully qualified URIs for the 
@@ -178,21 +173,21 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
   public void testGetLinkStatusPartQualTarget() throws IOException {
     Path fileAbs  = new Path(testBaseDir1()+"/file");
     Path fileQual = new Path(testURI().toString(), fileAbs);
-    Path dir      = new Path(testBaseDir1());    
+    Path dir      = new Path(testBaseDir1());
     Path link     = new Path(testBaseDir1()+"/linkToFile");
     Path dirNew   = new Path(testBaseDir2());
     Path linkNew  = new Path(testBaseDir2()+"/linkToFile");
-    fc.delete(dirNew, true);
+    wrapper.delete(dirNew, true);
     createAndWriteFile(fileQual);
-    fc.setWorkingDirectory(dir);
+    wrapper.setWorkingDirectory(dir);
     // Link target is partially qualified, we get the same back.
-    fc.createSymlink(fileQual, link, false);
-    assertEquals(fileQual, fc.getFileLinkStatus(link).getSymlink());
+    wrapper.createSymlink(fileQual, link, false);
+    assertEquals(fileQual, wrapper.getFileLinkStatus(link).getSymlink());
     // Because the target was specified with an absolute path the
     // link fails to resolve after moving the parent directory. 
-    fc.rename(dir, dirNew);
+    wrapper.rename(dir, dirNew);
     // The target is still the old path
-    assertEquals(fileQual, fc.getFileLinkStatus(linkNew).getSymlink());    
+    assertEquals(fileQual, wrapper.getFileLinkStatus(linkNew).getSymlink());    
     try {
       readFile(linkNew);
       fail("The link should be dangling now.");
@@ -204,7 +199,7 @@ public class TestLocalFSFileContextSymlink extends FileContextSymlinkBaseTest {
     Path anotherFs = new Path("hdfs://host:1000/dir/file");
     FileUtil.fullyDelete(new File(linkNew.toString()));
     try {
-      fc.createSymlink(anotherFs, linkNew, false);
+      wrapper.createSymlink(anotherFs, linkNew, false);
       fail("Created a local fs link to a non-local fs");
     } catch (IOException x) {
       // Excpected.

+ 30 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestSymlinkLocalFSFileContext.java

@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.junit.BeforeClass;
+
+public class TestSymlinkLocalFSFileContext extends TestSymlinkLocalFS {
+
+  @BeforeClass
+  public static void testSetup() throws Exception {
+    FileContext context = FileContext.getLocalFSFileContext();
+    wrapper = new FileContextTestWrapper(context);
+  }
+
+}

+ 98 - 105
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestFcHdfsSymlink.java → hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfs.java

@@ -36,26 +36,25 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.log4j.Level;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
 /**
- * Test symbolic links using FileContext and Hdfs.
+ * Test symbolic links in Hdfs.
  */
-public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
+abstract public class TestSymlinkHdfs extends SymlinkBaseTest {
 
   {
     ((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL);
   }
 
-  private static FileContextTestHelper fileContextTestHelper =
-      new FileContextTestHelper("/tmp/TestFcHdfsSymlink");
-  private static MiniDFSCluster cluster;
-  private static WebHdfsFileSystem webhdfs;
-  private static DistributedFileSystem dfs;
-  
+  protected static MiniDFSCluster cluster;
+  protected static WebHdfsFileSystem webhdfs;
+  protected static DistributedFileSystem dfs;
+
   @Override
   protected String getScheme() {
     return "hdfs";
@@ -85,90 +84,82 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
   }
 
   @BeforeClass
-  public static void testSetUp() throws Exception {
+  public static void beforeClassSetup() throws Exception {
     Configuration conf = new HdfsConfiguration();
     conf.setBoolean(DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY, true);
     conf.set(FsPermission.UMASK_LABEL, "000");
     cluster = new MiniDFSCluster.Builder(conf).build();
-    fc = FileContext.getFileContext(cluster.getURI(0));
     webhdfs = WebHdfsTestUtil.getWebHdfsFileSystem(conf);
     dfs = cluster.getFileSystem();
   }
-  
+
   @AfterClass
-  public static void testTearDown() throws Exception {
+  public static void afterClassTeardown() throws Exception {
     cluster.shutdown();
   }
-     
-  @Test
+
+  @Test(timeout=10000)
   /** Access a file using a link that spans Hdfs to LocalFs */
   public void testLinkAcrossFileSystems() throws IOException {
-    FileContext localFc = FileContext.getLocalFSFileContext();
-    Path localDir  = new Path("file://"+fileContextTestHelper.getAbsoluteTestRootDir(localFc)+"/test");
-    Path localFile = new Path("file://"+fileContextTestHelper.getAbsoluteTestRootDir(localFc)+"/test/file");
+    Path localDir = new Path("file://" + wrapper.getAbsoluteTestRootDir()
+        + "/test");
+    Path localFile = new Path("file://" + wrapper.getAbsoluteTestRootDir()
+        + "/test/file");
     Path link      = new Path(testBaseDir1(), "linkToFile");
-    localFc.delete(localDir, true);
-    localFc.mkdir(localDir, FileContext.DEFAULT_PERM, true);
-    localFc.setWorkingDirectory(localDir);
-    assertEquals(localDir, localFc.getWorkingDirectory());
-    createAndWriteFile(localFc, localFile);
-    fc.createSymlink(localFile, link, false);
+    FSTestWrapper localWrapper = wrapper.getLocalFSWrapper();
+    localWrapper.delete(localDir, true);
+    localWrapper.mkdir(localDir, FileContext.DEFAULT_PERM, true);
+    localWrapper.setWorkingDirectory(localDir);
+    assertEquals(localDir, localWrapper.getWorkingDirectory());
+    createAndWriteFile(localWrapper, localFile);
+    wrapper.createSymlink(localFile, link, false);
     readFile(link);
-    assertEquals(fileSize, fc.getFileStatus(link).getLen());
+    assertEquals(fileSize, wrapper.getFileStatus(link).getLen());
   }
 
-  @Test
+  @Test(timeout=10000)
   /** Test renaming a file across two file systems using a link */
   public void testRenameAcrossFileSystemsViaLink() throws IOException {
-    FileContext localFc = FileContext.getLocalFSFileContext();
-    Path localDir    = new Path("file://"+fileContextTestHelper.getAbsoluteTestRootDir(localFc)+"/test");
+    Path localDir = new Path("file://" + wrapper.getAbsoluteTestRootDir()
+        + "/test");
     Path hdfsFile    = new Path(testBaseDir1(), "file");
     Path link        = new Path(testBaseDir1(), "link");
     Path hdfsFileNew = new Path(testBaseDir1(), "fileNew");
     Path hdfsFileNewViaLink = new Path(link, "fileNew");
-    localFc.delete(localDir, true);
-    localFc.mkdir(localDir, FileContext.DEFAULT_PERM, true);
-    localFc.setWorkingDirectory(localDir);
-    createAndWriteFile(fc, hdfsFile);
-    fc.createSymlink(localDir, link, false);
+    FSTestWrapper localWrapper = wrapper.getLocalFSWrapper();
+    localWrapper.delete(localDir, true);
+    localWrapper.mkdir(localDir, FileContext.DEFAULT_PERM, true);
+    localWrapper.setWorkingDirectory(localDir);
+    createAndWriteFile(hdfsFile);
+    wrapper.createSymlink(localDir, link, false);
     // Rename hdfs://test1/file to hdfs://test1/link/fileNew
     // which renames to file://TEST_ROOT/test/fileNew which
     // spans AbstractFileSystems and therefore fails.
     try {
-      fc.rename(hdfsFile, hdfsFileNewViaLink);
+      wrapper.rename(hdfsFile, hdfsFileNewViaLink);
       fail("Renamed across file systems");
     } catch (InvalidPathException ipe) {
-      // Expected
+      // Expected from FileContext
+    } catch (IllegalArgumentException e) {
+      // Expected from Filesystem
+      GenericTestUtils.assertExceptionContains("Wrong FS: ", e);
     }
     // Now rename hdfs://test1/link/fileNew to hdfs://test1/fileNew
     // which renames file://TEST_ROOT/test/fileNew to hdfs://test1/fileNew
     // which spans AbstractFileSystems and therefore fails.
-    createAndWriteFile(fc, hdfsFileNewViaLink);
+    createAndWriteFile(hdfsFileNewViaLink);
     try {
-      fc.rename(hdfsFileNewViaLink, hdfsFileNew);
+      wrapper.rename(hdfsFileNewViaLink, hdfsFileNew);
       fail("Renamed across file systems");
     } catch (InvalidPathException ipe) {
-      // Expected
-    }
-  }
-
-  @Test
-  /** Test access a symlink using AbstractFileSystem */
-  public void testAccessLinkFromAbstractFileSystem() throws IOException {
-    Path file = new Path(testBaseDir1(), "file");
-    Path link = new Path(testBaseDir1(), "linkToFile");
-    createAndWriteFile(file);
-    fc.createSymlink(file, link, false);
-    try {
-      AbstractFileSystem afs = fc.getDefaultFileSystem();
-      afs.open(link);
-      fail("Opened a link using AFS");
-    } catch (UnresolvedLinkException x) {
-      // Expected
+      // Expected from FileContext
+    } catch (IllegalArgumentException e) {
+      // Expected from Filesystem
+      GenericTestUtils.assertExceptionContains("Wrong FS: ", e);
     }
   }
 
-  @Test
+  @Test(timeout=10000)
   /** Test create symlink to / */
   public void testCreateLinkToSlash() throws IOException {
     Path dir  = new Path(testBaseDir1());
@@ -177,59 +168,61 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
     Path fileViaLink = new Path(testBaseDir1()+"/linkToSlash"+
                                 testBaseDir1()+"/file");
     createAndWriteFile(file);
-    fc.setWorkingDirectory(dir);
-    fc.createSymlink(new Path("/"), link, false);
+    wrapper.setWorkingDirectory(dir);
+    wrapper.createSymlink(new Path("/"), link, false);
     readFile(fileViaLink);
-    assertEquals(fileSize, fc.getFileStatus(fileViaLink).getLen());
+    assertEquals(fileSize, wrapper.getFileStatus(fileViaLink).getLen());
     // Ditto when using another file context since the file system
     // for the slash is resolved according to the link's parent.
-    FileContext localFc = FileContext.getLocalFSFileContext();
-    Path linkQual = new Path(cluster.getURI(0).toString(), fileViaLink); 
-    assertEquals(fileSize, localFc.getFileStatus(linkQual).getLen());    
+    if (wrapper instanceof FileContextTestWrapper) {
+      FSTestWrapper localWrapper = wrapper.getLocalFSWrapper();
+      Path linkQual = new Path(cluster.getURI(0).toString(), fileViaLink);
+      assertEquals(fileSize, localWrapper.getFileStatus(linkQual).getLen());
+    }
   }
   
   
-  @Test
+  @Test(timeout=10000)
   /** setPermission affects the target not the link */
-  public void testSetPermissionAffectsTarget() throws IOException {    
+  public void testSetPermissionAffectsTarget() throws IOException {
     Path file       = new Path(testBaseDir1(), "file");
     Path dir        = new Path(testBaseDir2());
     Path linkToFile = new Path(testBaseDir1(), "linkToFile");
     Path linkToDir  = new Path(testBaseDir1(), "linkToDir");
     createAndWriteFile(file);
-    fc.createSymlink(file, linkToFile, false);
-    fc.createSymlink(dir, linkToDir, false);
+    wrapper.createSymlink(file, linkToFile, false);
+    wrapper.createSymlink(dir, linkToDir, false);
     
     // Changing the permissions using the link does not modify
     // the permissions of the link..
-    FsPermission perms = fc.getFileLinkStatus(linkToFile).getPermission();
-    fc.setPermission(linkToFile, new FsPermission((short)0664));
-    fc.setOwner(linkToFile, "user", "group");
-    assertEquals(perms, fc.getFileLinkStatus(linkToFile).getPermission());
+    FsPermission perms = wrapper.getFileLinkStatus(linkToFile).getPermission();
+    wrapper.setPermission(linkToFile, new FsPermission((short)0664));
+    wrapper.setOwner(linkToFile, "user", "group");
+    assertEquals(perms, wrapper.getFileLinkStatus(linkToFile).getPermission());
     // but the file's permissions were adjusted appropriately
-    FileStatus stat = fc.getFileStatus(file);
-    assertEquals(0664, stat.getPermission().toShort()); 
+    FileStatus stat = wrapper.getFileStatus(file);
+    assertEquals(0664, stat.getPermission().toShort());
     assertEquals("user", stat.getOwner());
     assertEquals("group", stat.getGroup());
     // Getting the file's permissions via the link is the same
     // as getting the permissions directly.
     assertEquals(stat.getPermission(), 
-                 fc.getFileStatus(linkToFile).getPermission());
+                 wrapper.getFileStatus(linkToFile).getPermission());
 
     // Ditto for a link to a directory
-    perms = fc.getFileLinkStatus(linkToDir).getPermission();
-    fc.setPermission(linkToDir, new FsPermission((short)0664));
-    fc.setOwner(linkToDir, "user", "group");
-    assertEquals(perms, fc.getFileLinkStatus(linkToDir).getPermission());
-    stat = fc.getFileStatus(dir);
-    assertEquals(0664, stat.getPermission().toShort()); 
+    perms = wrapper.getFileLinkStatus(linkToDir).getPermission();
+    wrapper.setPermission(linkToDir, new FsPermission((short)0664));
+    wrapper.setOwner(linkToDir, "user", "group");
+    assertEquals(perms, wrapper.getFileLinkStatus(linkToDir).getPermission());
+    stat = wrapper.getFileStatus(dir);
+    assertEquals(0664, stat.getPermission().toShort());
     assertEquals("user", stat.getOwner());
     assertEquals("group", stat.getGroup());
     assertEquals(stat.getPermission(), 
-                 fc.getFileStatus(linkToDir).getPermission());
+                 wrapper.getFileStatus(linkToDir).getPermission());
   }  
 
-  @Test
+  @Test(timeout=10000)
   /** Create a symlink using a path with scheme but no authority */
   public void testCreateWithPartQualPathFails() throws IOException {
     Path fileWoAuth = new Path("hdfs:///test/file");
@@ -241,27 +234,27 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
       // Expected
     }
     try {
-      fc.createSymlink(new Path("foo"), linkWoAuth, false);
+      wrapper.createSymlink(new Path("foo"), linkWoAuth, false);
       fail("HDFS requires URIs with schemes have an authority");
     } catch (RuntimeException e) {
       // Expected
     }
   }
 
-  @Test
-  /** setReplication affects the target not the link */  
+  @Test(timeout=10000)
+  /** setReplication affects the target not the link */
   public void testSetReplication() throws IOException {
     Path file = new Path(testBaseDir1(), "file");
     Path link = new Path(testBaseDir1(), "linkToFile");
     createAndWriteFile(file);
-    fc.createSymlink(file, link, false);
-    fc.setReplication(link, (short)2);
-    assertEquals(0, fc.getFileLinkStatus(link).getReplication());
-    assertEquals(2, fc.getFileStatus(link).getReplication());      
-    assertEquals(2, fc.getFileStatus(file).getReplication());
+    wrapper.createSymlink(file, link, false);
+    wrapper.setReplication(link, (short)2);
+    assertEquals(0, wrapper.getFileLinkStatus(link).getReplication());
+    assertEquals(2, wrapper.getFileStatus(link).getReplication());
+    assertEquals(2, wrapper.getFileStatus(file).getReplication());
   }
   
-  @Test
+  @Test(timeout=10000)
   /** Test create symlink with a max len name */
   public void testCreateLinkMaxPathLink() throws IOException {
     Path dir  = new Path(testBaseDir1());
@@ -283,47 +276,47 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
     
     // Check that it works
     createAndWriteFile(file);
-    fc.setWorkingDirectory(dir);
-    fc.createSymlink(file, link, false);
+    wrapper.setWorkingDirectory(dir);
+    wrapper.createSymlink(file, link, false);
     readFile(link);
     
     // Now modify the path so it's too large
     link = new Path(sb.toString()+"x");
     try {
-      fc.createSymlink(file, link, false);
+      wrapper.createSymlink(file, link, false);
       fail("Path name should be too long");
     } catch (IOException x) {
       // Expected
     }
   }
 
-  @Test
+  @Test(timeout=10000)
   /** Test symlink owner */
   public void testLinkOwner() throws IOException {
     Path file = new Path(testBaseDir1(), "file");
     Path link = new Path(testBaseDir1(), "symlinkToFile");
     createAndWriteFile(file);
-    fc.createSymlink(file, link, false);
-    FileStatus statFile = fc.getFileStatus(file);
-    FileStatus statLink = fc.getFileStatus(link);
+    wrapper.createSymlink(file, link, false);
+    FileStatus statFile = wrapper.getFileStatus(file);
+    FileStatus statLink = wrapper.getFileStatus(link);
     assertEquals(statLink.getOwner(), statFile.getOwner());
   }
 
-  @Test
-  /** Test WebHdfsFileSystem.craeteSymlink(..). */  
+  @Test(timeout=10000)
+  /** Test WebHdfsFileSystem.createSymlink(..). */
   public void testWebHDFS() throws IOException {
     Path file = new Path(testBaseDir1(), "file");
     Path link = new Path(testBaseDir1(), "linkToFile");
     createAndWriteFile(file);
     webhdfs.createSymlink(file, link, false);
-    fc.setReplication(link, (short)2);
-    assertEquals(0, fc.getFileLinkStatus(link).getReplication());
-    assertEquals(2, fc.getFileStatus(link).getReplication());      
-    assertEquals(2, fc.getFileStatus(file).getReplication());
+    wrapper.setReplication(link, (short)2);
+    assertEquals(0, wrapper.getFileLinkStatus(link).getReplication());
+    assertEquals(2, wrapper.getFileStatus(link).getReplication());
+    assertEquals(2, wrapper.getFileStatus(file).getReplication());
   }
 
-  @Test
-  /** Test craeteSymlink(..) with quota. */  
+  @Test(timeout=10000)
+  /** Test craeteSymlink(..) with quota. */
   public void testQuota() throws IOException {
     final Path dir = new Path(testBaseDir1());
     dfs.setQuota(dir, 3, HdfsConstants.QUOTA_DONT_SET);
@@ -333,12 +326,12 @@ public class TestFcHdfsSymlink extends FileContextSymlinkBaseTest {
 
     //creating the first link should succeed
     final Path link1 = new Path(dir, "link1");
-    fc.createSymlink(file, link1, false);
+    wrapper.createSymlink(file, link1, false);
 
     try {
       //creating the second link should fail with QuotaExceededException.
       final Path link2 = new Path(dir, "link2");
-      fc.createSymlink(file, link2, false);
+      wrapper.createSymlink(file, link2, false);
       fail("Created symlink despite quota violation");
     } catch(QuotaExceededException qee) {
       //expected

+ 52 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileContext.java

@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestSymlinkHdfsFileContext extends TestSymlinkHdfs {
+
+  private static FileContext fc;
+
+  @BeforeClass
+  public static void testSetup() throws Exception {
+    fc = FileContext.getFileContext(cluster.getURI(0));
+    wrapper = new FileContextTestWrapper(fc, "/tmp/TestSymlinkHdfsFileContext");
+  }
+
+  @Test(timeout=1000)
+  /** Test access a symlink using AbstractFileSystem */
+  public void testAccessLinkFromAbstractFileSystem() throws IOException {
+    Path file = new Path(testBaseDir1(), "file");
+    Path link = new Path(testBaseDir1(), "linkToFile");
+    createAndWriteFile(file);
+    wrapper.createSymlink(file, link, false);
+    try {
+      AbstractFileSystem afs = fc.getDefaultFileSystem();
+      afs.open(link);
+      fail("Opened a link using AFS");
+    } catch (UnresolvedLinkException x) {
+      // Expected
+    }
+  }
+}

Някои файлове не бяха показани, защото твърде много файлове са промени