Parcourir la source

HADOOP-9078. enhance unit-test coverage of class org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1532929 13f79535-47bb-0310-9956-ffa450edef68
Jonathan Turner Eagles il y a 11 ans
Parent
commit
186581fef2

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

@@ -73,6 +73,9 @@ Release 2.3.0 - UNRELEASED
     HADOOP-9897. Add method to get path start position without drive specifier in
     o.a.h.fs.Path. (Binglin Chang via cnauroth)
 
+    HADOOP-9078. enhance unit-test coverage of class
+    org.apache.hadoop.fs.FileContext (Ivan A. Veselovsky via jeagles)
+
   OPTIMIZATIONS
 
     HADOOP-9748. Reduce blocking on UGI.ensureInitialized (daryn)

+ 143 - 2
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextMainOperationsBaseTest.java

@@ -22,6 +22,7 @@ import java.io.File;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.util.EnumSet;
+import java.util.NoSuchElementException;
 
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -30,6 +31,7 @@ import org.apache.hadoop.fs.Options.Rename;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.junit.After;
 import org.junit.Assert;
+import static org.junit.Assert.*;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -92,7 +94,7 @@ public abstract class FileContextMainOperationsBaseTest  {
     }     
   };
   
-  private static byte[] data = getFileData(numBlocks,
+  private static final byte[] data = getFileData(numBlocks,
       getDefaultBlockSize());
   
   @Before
@@ -107,7 +109,8 @@ public abstract class FileContextMainOperationsBaseTest  {
   
   @After
   public void tearDown() throws Exception {
-    fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
+    boolean del = fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
+    assertTrue(del);
     fc.delete(localFsRootPath, true);
   }
   
@@ -194,6 +197,14 @@ public abstract class FileContextMainOperationsBaseTest  {
     fc.setWorkingDirectory(absoluteDir);
     Assert.assertEquals(absoluteDir, fc.getWorkingDirectory());
 
+    Path aRegularFile = new Path("aRegularFile");
+    createFile(aRegularFile);
+    try {
+      fc.setWorkingDirectory(aRegularFile);
+      fail("An IOException expected.");
+    } catch (IOException ioe) {
+      // okay
+    }
   }
   
   @Test
@@ -1195,6 +1206,136 @@ public abstract class FileContextMainOperationsBaseTest  {
         return true;
       }
     return false;
+ }
+
+  @Test
+  public void testOpen2() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    //final Path rootPath = getAbsoluteTestRootPath(fc);
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    final long length = fc.getFileStatus(path).getLen();
+    FSDataInputStream fsdis = fc.open(path, 2048);
+    try {
+      byte[] bb = new byte[(int)length];
+      fsdis.readFully(bb);
+      assertArrayEquals(data, bb);
+    } finally {
+      fsdis.close();
+    }
+  }
+
+  @Test
+  public void testSetVerifyChecksum() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+
+    FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
+        Options.CreateOpts.createParent());
+    try {
+      // instruct FS to verify checksum through the FileContext:
+      fc.setVerifyChecksum(true, path);
+      out.write(data, 0, data.length);
+    } finally {
+      out.close();
+    }
+
+    // NB: underlying FS may be different (this is an abstract test),
+    // so we cannot assert .zoo.crc existence.
+    // Instead, we check that the file is read correctly:
+    FileStatus fileStatus = fc.getFileStatus(path);
+    final long len = fileStatus.getLen();
+    assertTrue(len == data.length);
+    byte[] bb = new byte[(int)len];
+    FSDataInputStream fsdis = fc.open(path);
+    try {
+      fsdis.read(bb);
+    } finally {
+      fsdis.close();
+    }
+    assertArrayEquals(data, bb);
+  }
+
+  @Test
+  public void testListCorruptFileBlocks() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    try {
+      final RemoteIterator<Path> remoteIterator = fc
+          .listCorruptFileBlocks(path);
+      if (listCorruptedBlocksSupported()) {
+        assertTrue(remoteIterator != null);
+        Path p;
+        while (remoteIterator.hasNext()) {
+          p = remoteIterator.next();
+          System.out.println("corrupted block: " + p);
+        }
+        try {
+          remoteIterator.next();
+          fail();
+        } catch (NoSuchElementException nsee) {
+          // okay
+        }
+      } else {
+        fail();
+      }
+    } catch (UnsupportedOperationException uoe) {
+      if (listCorruptedBlocksSupported()) {
+        fail(uoe.toString());
+      } else {
+        // okay
+      }
+    }
+  }
+
+  protected abstract boolean listCorruptedBlocksSupported();
+
+  @Test
+  public void testDeleteOnExitUnexisting() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    boolean registered = fc.deleteOnExit(path);
+    // because "zoo" does not exist:
+    assertTrue(!registered);
+  }
+
+  @Test
+  public void testFileContextStatistics() throws IOException {
+    FileContext.clearStatistics();
+
+    final Path rootPath = getTestRootPath(fc, "test");
+    final Path path = new Path(rootPath, "zoo");
+    createFile(path);
+    byte[] bb = new byte[data.length];
+    FSDataInputStream fsdis = fc.open(path);
+    try {
+      fsdis.read(bb);
+    } finally {
+      fsdis.close();
+    }
+    assertArrayEquals(data, bb);
+
+    FileContext.printStatistics();
+  }
+
+  @Test
+  /*
+   * Test method
+   *  org.apache.hadoop.fs.FileContext.getFileContext(AbstractFileSystem)
+   */
+  public void testGetFileContext1() throws IOException {
+    final Path rootPath = getTestRootPath(fc, "test");
+    AbstractFileSystem asf = fc.getDefaultFileSystem();
+    // create FileContext using the protected #getFileContext(1) method:
+    FileContext fc2 = FileContext.getFileContext(asf);
+    // Now just check that this context can do something reasonable:
+    final Path path = new Path(rootPath, "zoo");
+    FSDataOutputStream out = fc2.create(path, EnumSet.of(CREATE),
+        Options.CreateOpts.createParent());
+    out.close();
+    Path pathResolved = fc2.resolvePath(path);
+    assertEquals(pathResolved.toUri().getPath(), path.toUri().getPath());
   }
   
   private Path getTestRootPath(FileContext fc, String pathString) {

+ 8 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileContextPermissionBase.java

@@ -35,6 +35,7 @@ import org.junit.Test;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.*;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * <p>
@@ -174,6 +175,13 @@ public abstract class FileContextPermissionBase {
         System.out.println("Not testing changing the group since user " +
                            "belongs to only one group.");
       }
+      
+      try {
+        fc.setOwner(f, null, null);
+        fail("Exception expected.");
+      } catch (IllegalArgumentException iae) {
+        // okay
+      }
     } 
     finally {cleanupFile(fc, f);}
   }

+ 5 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFSFileContextMainOperations.java

@@ -49,6 +49,11 @@ public class TestLocalFSFileContextMainOperations extends FileContextMainOperati
     FileContext fc1 = FileContext.getLocalFSFileContext();
     Assert.assertTrue(fc1 != fc);
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
 
   @Test
   public void testDefaultFilePermission() throws IOException {

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestFcMainOperationsLocalFs.java

@@ -18,13 +18,9 @@
 package org.apache.hadoop.fs.viewfs;
 
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.FileContextMainOperationsBaseTest;
-import org.apache.hadoop.fs.FileContextTestHelper;
-import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.viewfs.ConfigUtil;
 
 import org.junit.After;
 import org.junit.Before;
@@ -49,4 +45,9 @@ public class TestFcMainOperationsLocalFs  extends
     super.tearDown();
     ViewFsTestSetup.tearDownForViewFsLocalFs(fileContextTestHelper);
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return false;
+  }
 }

+ 34 - 9
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSFileContextMainOperations.java

@@ -19,8 +19,10 @@
 package org.apache.hadoop.fs;
 
 import static org.apache.hadoop.fs.FileContextTestHelper.exists;
+import static org.junit.Assert.fail;
 
 import java.io.IOException;
+import java.net.URI;
 import java.net.URISyntaxException;
 
 import javax.security.auth.login.LoginException;
@@ -55,7 +57,8 @@ public class TestHDFSFileContextMainOperations extends
       LoginException, URISyntaxException {
     cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(2).build();
     cluster.waitClusterUp();
-    fc = FileContext.getFileContext(cluster.getURI(0), CONF);
+    URI uri0 = cluster.getURI(0);
+    fc = FileContext.getFileContext(uri0, CONF);
     defaultWorkingDirectory = fc.makeQualified( new Path("/user/" + 
         UserGroupInformation.getCurrentUser().getShortUserName()));
     fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
@@ -77,7 +80,10 @@ public class TestHDFSFileContextMainOperations extends
       
   @AfterClass
   public static void ClusterShutdownAtEnd() throws Exception {
-    cluster.shutdown();   
+    if (cluster != null) {
+      cluster.shutdown();
+      cluster = null;
+    }    
   }
   
   @Override
@@ -111,7 +117,7 @@ public class TestHDFSFileContextMainOperations extends
   
   @Test
   public void testOldRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testOldRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testOldRenameWithQuota/dstdir/dst1");
@@ -146,7 +152,7 @@ public class TestHDFSFileContextMainOperations extends
   
   @Test
   public void testRenameWithQuota() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src1");
     Path src2 = getTestRootPath(fc, "test/testRenameWithQuota/srcdir/src2");
     Path dst1 = getTestRootPath(fc, "test/testRenameWithQuota/dstdir/dst1");
@@ -210,7 +216,7 @@ public class TestHDFSFileContextMainOperations extends
    */
   @Test
   public void testEditsLogOldRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     createFile(src1);
@@ -226,7 +232,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogOldRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogOldRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -239,7 +245,7 @@ public class TestHDFSFileContextMainOperations extends
    */
   @Test
   public void testEditsLogRename() throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     Path src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     Path dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     createFile(src1);
@@ -255,7 +261,7 @@ public class TestHDFSFileContextMainOperations extends
     // Restart the cluster and ensure the above operations can be
     // loaded from the edits log
     restartCluster();
-    fs = (DistributedFileSystem)cluster.getFileSystem();
+    fs = cluster.getFileSystem();
     src1 = getTestRootPath(fc, "testEditsLogRename/srcdir/src1");
     dst1 = getTestRootPath(fc, "testEditsLogRename/dstdir/dst1");
     Assert.assertFalse(fs.exists(src1));   // ensure src1 is already renamed
@@ -279,7 +285,7 @@ public class TestHDFSFileContextMainOperations extends
 
   private void oldRename(Path src, Path dst, boolean renameSucceeds,
       boolean exception) throws Exception {
-    DistributedFileSystem fs = (DistributedFileSystem) cluster.getFileSystem();
+    DistributedFileSystem fs = cluster.getFileSystem();
     try {
       Assert.assertEquals(renameSucceeds, fs.rename(src, dst));
     } catch (Exception ex) {
@@ -301,4 +307,23 @@ public class TestHDFSFileContextMainOperations extends
     Assert.assertEquals(renameSucceeds, !exists(fc, src));
     Assert.assertEquals((dstExists||renameSucceeds), exists(fc, dst));
   }
+  
+  @Override
+  protected boolean listCorruptedBlocksSupported() {
+    return true;
+  }
+  
+  @Test
+  public void testCrossFileSystemRename() throws IOException {
+    try {
+      fc.rename(
+        new Path("hdfs://127.0.0.1/aaa/bbb/Foo"), 
+        new Path("file://aaa/bbb/Moo"), 
+        Options.Rename.OVERWRITE);
+      fail("IOexception expected.");
+    } catch (IOException ioe) {
+      // okay
+    }
+  }
+  
 }