瀏覽代碼

HADOOP-9258 Add stricter tests to FileSystemContractTestBase (includes fixes for production code HADOOP-9261 & HADOOP-9265 and test enhancements HADOOP-9228, HADOOP-9227 & HADOOP-9259)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1460646 13f79535-47bb-0310-9956-ffa450edef68
Steve Loughran 12 年之前
父節點
當前提交
a10055cf6d

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

@@ -162,6 +162,7 @@ Trunk (Unreleased)
 
     HADOOP-9218 Document the Rpc-wrappers used internally (sanjay Radia)
 
+	HADOOP-9258 Add stricter tests to FileSystemContractTestBase (stevel)
 
   BUG FIXES
 
@@ -355,6 +356,12 @@ Trunk (Unreleased)
     HADOOP-9431 TestSecurityUtil#testLocalHostNameForNullOrWild on systems where hostname
     contains capital letters  (Chris Nauroth via sanjay)
 
+	HADOOP-9261 S3n filesystem can move a directory under itself -and so lose data
+	(fixed in HADOOP-9258) (stevel)
+
+	HADOOP-9265 S3 blockstore filesystem breaks part of the Filesystem contract
+	(fixed in HADOOP-9258) (stevel)
+
   OPTIMIZATIONS
 
     HADOOP-7761. Improve the performance of raw comparisons. (todd)

+ 19 - 3
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3/Jets3tFileSystemStore.java

@@ -137,9 +137,15 @@ class Jets3tFileSystemStore implements FileSystemStore {
 
   @Override
   public boolean inodeExists(Path path) throws IOException {
-    InputStream in = get(pathToKey(path), true);
+    String key = pathToKey(path);
+    InputStream in = get(key, true);
     if (in == null) {
-      return false;
+      if (isRoot(key)) {
+        storeINode(path, INode.DIRECTORY_INODE);
+        return true;
+      } else {
+        return false;
+      }
     }
     in.close();
     return true;
@@ -211,7 +217,13 @@ class Jets3tFileSystemStore implements FileSystemStore {
 
   @Override
   public INode retrieveINode(Path path) throws IOException {
-    return INode.deserialize(get(pathToKey(path), true));
+    String key = pathToKey(path);
+    InputStream in = get(key, true);
+    if (in == null && isRoot(key)) {
+      storeINode(path, INode.DIRECTORY_INODE);
+      return INode.DIRECTORY_INODE;
+    }
+    return INode.deserialize(in);
   }
 
   @Override
@@ -366,6 +378,10 @@ class Jets3tFileSystemStore implements FileSystemStore {
     return blockToKey(block.getId());
   }
 
+  private boolean isRoot(String key) {
+    return key.isEmpty() || key.equals("/");
+  }
+
   @Override
   public void purge() throws IOException {
     try {

+ 105 - 12
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3/S3FileSystem.java

@@ -252,32 +252,125 @@ public class S3FileSystem extends FileSystem {
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
     Path absoluteSrc = makeAbsolute(src);
+    final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - ";
     INode srcINode = store.retrieveINode(absoluteSrc);
+    boolean debugEnabled = LOG.isDebugEnabled();
     if (srcINode == null) {
       // src path doesn't exist
+      if (debugEnabled) {
+        LOG.debug(debugPreamble + "returning false as src does not exist");
+      }
       return false; 
     }
+
     Path absoluteDst = makeAbsolute(dst);
-    INode dstINode = store.retrieveINode(absoluteDst);
-    if (dstINode != null && dstINode.isDirectory()) {
-      absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
-      dstINode = store.retrieveINode(absoluteDst);
-    }
-    if (dstINode != null) {
-      // dst path already exists - can't overwrite
-      return false;
-    }
+
+    //validate the parent dir of the destination
     Path dstParent = absoluteDst.getParent();
     if (dstParent != null) {
+      //if the dst parent is not root, make sure it exists
       INode dstParentINode = store.retrieveINode(dstParent);
-      if (dstParentINode == null || dstParentINode.isFile()) {
-        // dst parent doesn't exist or is a file
+      if (dstParentINode == null) {
+        // dst parent doesn't exist
+        if (debugEnabled) {
+          LOG.debug(debugPreamble +
+                    "returning false as dst parent does not exist");
+        }
+        return false;
+      }
+      if (dstParentINode.isFile()) {
+        // dst parent exists but is a file
+        if (debugEnabled) {
+          LOG.debug(debugPreamble +
+                    "returning false as dst parent exists and is a file");
+        }
         return false;
       }
     }
+
+    //get status of source
+    boolean srcIsFile = srcINode.isFile();
+
+    INode dstINode = store.retrieveINode(absoluteDst);
+    boolean destExists = dstINode != null;
+    boolean destIsDir = destExists && !dstINode.isFile();
+    if (srcIsFile) {
+
+      //source is a simple file
+      if (destExists) {
+        if (destIsDir) {
+          //outcome #1 dest exists and is dir -filename to subdir of dest
+          if (debugEnabled) {
+            LOG.debug(debugPreamble +
+                      "copying src file under dest dir to " + absoluteDst);
+          }
+          absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
+        } else {
+          //outcome #2 dest it's a file: fail iff different from src
+          boolean renamingOnToSelf = absoluteSrc.equals(absoluteDst);
+          if (debugEnabled) {
+            LOG.debug(debugPreamble +
+                      "copying file onto file, outcome is " + renamingOnToSelf);
+          }
+          return renamingOnToSelf;
+        }
+      } else {
+        // #3 dest does not exist: use dest as path for rename
+        if (debugEnabled) {
+          LOG.debug(debugPreamble +
+                    "copying file onto file");
+        }
+      }
+    } else {
+      //here the source exists and is a directory
+      // outcomes (given we know the parent dir exists if we get this far)
+      // #1 destination is a file: fail
+      // #2 destination is a directory: create a new dir under that one
+      // #3 destination doesn't exist: create a new dir with that name
+      // #3 and #4 are only allowed if the dest path is not == or under src
+
+      if (destExists) {
+        if (!destIsDir) {
+          // #1 destination is a file: fail
+          if (debugEnabled) {
+            LOG.debug(debugPreamble +
+                      "returning false as src is a directory, but not dest");
+          }
+          return false;
+        } else {
+          // the destination dir exists
+          // destination for rename becomes a subdir of the target name
+          absoluteDst = new Path(absoluteDst, absoluteSrc.getName());
+          if (debugEnabled) {
+            LOG.debug(debugPreamble +
+                      "copying src dir under dest dir to " + absoluteDst);
+          }
+        }
+      }
+      //the final destination directory is now know, so validate it for
+      //illegal moves
+
+      if (absoluteSrc.equals(absoluteDst)) {
+        //you can't rename a directory onto itself
+        if (debugEnabled) {
+          LOG.debug(debugPreamble +
+                    "Dest==source && isDir -failing");
+        }
+        return false;
+      }
+      if (absoluteDst.toString().startsWith(absoluteSrc.toString() + "/")) {
+        //you can't move a directory under itself
+        if (debugEnabled) {
+          LOG.debug(debugPreamble +
+                    "dst is equal to or under src dir -failing");
+        }
+        return false;
+      }
+    }
+    //here the dest path is set up -so rename
     return renameRecursive(absoluteSrc, absoluteDst);
   }
-  
+
   private boolean renameRecursive(Path src, Path dst) throws IOException {
     INode srcINode = store.retrieveINode(src);
     store.storeINode(dst, srcINode);

+ 47 - 13
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/s3native/NativeS3FileSystem.java

@@ -582,35 +582,58 @@ public class NativeS3FileSystem extends FileSystem {
   public boolean rename(Path src, Path dst) throws IOException {
 
     String srcKey = pathToKey(makeAbsolute(src));
+    final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - ";
 
     if (srcKey.length() == 0) {
       // Cannot rename root of file system
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(debugPreamble +
+                  "returning false as cannot rename the root of a filesystem");
+      }
       return false;
     }
 
-    final String debugPreamble = "Renaming '" + src + "' to '" + dst + "' - ";
-
+    //get status of source
+    boolean srcIsFile;
+    try {
+      srcIsFile = getFileStatus(src).isFile();
+    } catch (FileNotFoundException e) {
+      //bail out fast if the source does not exist
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(debugPreamble + "returning false as src does not exist");
+      }
+      return false;
+    }
     // Figure out the final destination
-    String dstKey;
+    String dstKey = pathToKey(makeAbsolute(dst));
+
     try {
       boolean dstIsFile = getFileStatus(dst).isFile();
       if (dstIsFile) {
+        //destination is a file.
+        //you can't copy a file or a directory onto an existing file
+        //except for the special case of dest==src, which is a no-op
         if(LOG.isDebugEnabled()) {
           LOG.debug(debugPreamble +
-              "returning false as dst is an already existing file");
+              "returning without rename as dst is an already existing file");
         }
-        return false;
+        //exit, returning true iff the rename is onto self
+        return srcKey.equals(dstKey);
       } else {
+        //destination exists and is a directory
         if(LOG.isDebugEnabled()) {
           LOG.debug(debugPreamble + "using dst as output directory");
         }
+        //destination goes under the dst path, with the name of the
+        //source entry
         dstKey = pathToKey(makeAbsolute(new Path(dst, src.getName())));
       }
     } catch (FileNotFoundException e) {
+      //destination does not exist => the source file or directory
+      //is copied over with the name of the destination
       if(LOG.isDebugEnabled()) {
         LOG.debug(debugPreamble + "using dst as output destination");
       }
-      dstKey = pathToKey(makeAbsolute(dst));
       try {
         if (getFileStatus(dst.getParent()).isFile()) {
           if(LOG.isDebugEnabled()) {
@@ -628,16 +651,17 @@ public class NativeS3FileSystem extends FileSystem {
       }
     }
 
-    boolean srcIsFile;
-    try {
-      srcIsFile = getFileStatus(src).isFile();
-    } catch (FileNotFoundException e) {
-      if(LOG.isDebugEnabled()) {
-        LOG.debug(debugPreamble + "returning false as src does not exist");
+    //rename to self behavior follows Posix rules and is different
+    //for directories and files -the return code is driven by src type
+    if (srcKey.equals(dstKey)) {
+      //fully resolved destination key matches source: fail
+      if (LOG.isDebugEnabled()) {
+        LOG.debug(debugPreamble + "renamingToSelf; returning true");
       }
-      return false;
+      return true;
     }
     if (srcIsFile) {
+      //source is a file; COPY then DELETE
       if(LOG.isDebugEnabled()) {
         LOG.debug(debugPreamble +
             "src is file, so doing copy then delete in S3");
@@ -645,9 +669,19 @@ public class NativeS3FileSystem extends FileSystem {
       store.copy(srcKey, dstKey);
       store.delete(srcKey);
     } else {
+      //src is a directory
       if(LOG.isDebugEnabled()) {
         LOG.debug(debugPreamble + "src is directory, so copying contents");
       }
+      //Verify dest is not a child of the parent
+      if (dstKey.startsWith(srcKey + "/")) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug(
+            debugPreamble + "cannot rename a directory to a subdirectory of self");
+        }
+        return false;
+      }
+      //create the subdir under the destination
       store.storeEmptyFile(dstKey + FOLDER_SUFFIX);
 
       List<String> keysToDelete = new ArrayList<String>();

+ 283 - 16
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/FileSystemContractBaseTest.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.fs;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.Locale;
 
 import junit.framework.TestCase;
 
@@ -51,7 +52,13 @@ public abstract class FileSystemContractBaseTest extends TestCase {
 
   @Override
   protected void tearDown() throws Exception {
-    fs.delete(path("/test"), true);
+    try {
+      if (fs != null) {
+        fs.delete(path("/test"), true);
+      }
+    } catch (IOException e) {
+      LOG.error("Error deleting /test: " + e, e);
+    }
   }
   
   protected int getBlockSize() {
@@ -62,10 +69,23 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     return "/user/" + System.getProperty("user.name");
   }
 
+  /**
+   * Override this if the filesystem does not support rename
+   * @return true if the FS supports rename -and rename related tests
+   * should be run
+   */
   protected boolean renameSupported() {
     return true;
   }
 
+  /**
+   * Override this if the filesystem is not case sensitive
+   * @return true if the case detection/preservation tests should run
+   */
+  protected boolean filesystemIsCaseSensitive() {
+    return true;
+  }
+
   public void testFsStatus() throws Exception {
     FsStatus fsStatus = fs.getStatus();
     assertNotNull(fsStatus);
@@ -109,6 +129,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     assertTrue(fs.mkdirs(testDir));
 
     assertTrue(fs.exists(testDir));
+    assertTrue("Should be a directory", fs.isDirectory(testDir));
     assertFalse(fs.isFile(testDir));
 
     Path parentDir = testDir.getParent();
@@ -118,17 +139,17 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     Path grandparentDir = parentDir.getParent();
     assertTrue(fs.exists(grandparentDir));
     assertFalse(fs.isFile(grandparentDir));
-    
+
   }
-  
+
   public void testMkdirsFailsForSubdirectoryOfExistingFile() throws Exception {
     Path testDir = path("/test/hadoop");
     assertFalse(fs.exists(testDir));
     assertTrue(fs.mkdirs(testDir));
     assertTrue(fs.exists(testDir));
-    
+
     createFile(path("/test/hadoop/file"));
-    
+
     Path testSubDir = path("/test/hadoop/file/subdir");
     try {
       fs.mkdirs(testSubDir);
@@ -137,7 +158,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
       // expected
     }
     assertFalse(fs.exists(testSubDir));
-    
+
     Path testDeepSubDir = path("/test/hadoop/file/deep/sub/dir");
     try {
       fs.mkdirs(testDeepSubDir);
@@ -146,7 +167,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
       // expected
     }
     assertFalse(fs.exists(testDeepSubDir));
-    
+
   }
 
   public void testMkdirsWithUmask() throws Exception {
@@ -177,7 +198,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
       // expected
     }
   }
-  
+
   public void testListStatusThrowsExceptionForNonExistentFile() throws Exception {
     try {
       fs.listStatus(path("/test/hadoop/file"));
@@ -186,7 +207,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
       // expected
     }
   }
-  
+
   public void testListStatus() throws Exception {
     Path[] testDirs = { path("/test/hadoop/a"),
                         path("/test/hadoop/b"),
@@ -210,7 +231,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     paths = fs.listStatus(path("/test/hadoop/a"));
     assertEquals(0, paths.length);
   }
-  
+
   public void testWriteReadAndDeleteEmptyFile() throws Exception {
     writeReadAndDelete(0);
   }
@@ -222,7 +243,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   public void testWriteReadAndDeleteOneBlock() throws Exception {
     writeReadAndDelete(getBlockSize());
   }
-  
+
   public void testWriteReadAndDeleteOneAndAHalfBlocks() throws Exception {
     writeReadAndDelete(getBlockSize() + (getBlockSize() / 2));
   }
@@ -365,8 +386,7 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     Path dst = path("/test/new/newdir");
     fs.mkdirs(dst);
     rename(src, dst, true, false, true);
-    assertTrue("Destination changed",
-        fs.exists(path("/test/new/newdir/file")));
+    assertIsFile(path("/test/new/newdir/file"));
   }
   
   public void testRenameDirectoryMoveToNonExistentDirectory() 
@@ -466,9 +486,9 @@ public abstract class FileSystemContractBaseTest extends TestCase {
   
   private void rename(Path src, Path dst, boolean renameSucceeded,
       boolean srcExists, boolean dstExists) throws IOException {
-    assertEquals("Rename result", renameSucceeded, fs.rename(src, dst));
-    assertEquals("Source exists", srcExists, fs.exists(src));
-    assertEquals("Destination exists", dstExists, fs.exists(dst));
+    assertEquals("mv " + src + " " + dst,renameSucceeded, fs.rename(src, dst));
+    assertEquals("Source exists: " + src, srcExists, fs.exists(src));
+    assertEquals("Destination exists" + dst, dstExists, fs.exists(dst));
   }
 
   /**
@@ -494,6 +514,253 @@ public abstract class FileSystemContractBaseTest extends TestCase {
     writeAndRead(path, filedata2, blockSize * 2, true, false);
   }
 
+  /**
+   * Assert that a filesystem is case sensitive.
+   * This is done by creating a mixed-case filename and asserting that
+   * its lower case version is not there.
+   * @throws Exception
+   */
+  public void testFilesystemIsCaseSensitive() throws Exception {
+    if (!filesystemIsCaseSensitive()) {
+      LOG.info("Skipping test");
+      return;
+    }
+    String mixedCaseFilename = "/test/UPPER.TXT";
+    Path upper = path(mixedCaseFilename);
+    Path lower = path(mixedCaseFilename.toLowerCase(Locale.ENGLISH));
+    assertFalse("File exists" + upper, fs.exists(upper));
+    assertFalse("File exists" + lower, fs.exists(lower));
+    FSDataOutputStream out = fs.create(upper);
+    out.writeUTF("UPPER");
+    out.close();
+    FileStatus upperStatus = fs.getFileStatus(upper);
+    assertTrue("File does not exist" + upper, fs.exists(upper));
+    //verify the lower-case version of the filename doesn't exist
+    assertFalse("File exists" + lower, fs.exists(lower));
+    //now overwrite the lower case version of the filename with a
+    //new version.
+    out = fs.create(lower);
+    out.writeUTF("l");
+    out.close();
+    assertTrue("File does not exist" + lower, fs.exists(lower));
+    //verify the length of the upper file hasn't changed
+    FileStatus newStatus = fs.getFileStatus(upper);
+    assertEquals("Expected status:" + upperStatus
+                 + " actual status " + newStatus,
+                 upperStatus.getLen(),
+                 newStatus.getLen()); }
+
+  /**
+   * Asserts that a zero byte file has a status of file and not
+   * directory or symlink
+   * @throws Exception on failures
+   */
+  public void testZeroByteFilesAreFiles() throws Exception {
+    Path src = path("/test/testZeroByteFilesAreFiles");
+    //create a zero byte file
+    FSDataOutputStream out = fs.create(src);
+    out.close();
+    assertIsFile(src);
+  }
+
+  /**
+   * Asserts that a zero byte file has a status of file and not
+   * directory or symlink
+   * @throws Exception on failures
+   */
+  public void testMultiByteFilesAreFiles() throws Exception {
+    Path src = path("/test/testMultiByteFilesAreFiles");
+    FSDataOutputStream out = fs.create(src);
+    out.writeUTF("testMultiByteFilesAreFiles");
+    out.close();
+    assertIsFile(src);
+  }
+
+  /**
+   * Assert that root directory renames are not allowed
+   * @throws Exception on failures
+   */
+  public void testRootDirAlwaysExists() throws Exception {
+    //this will throw an exception if the path is not found
+    fs.getFileStatus(path("/"));
+    //this catches overrides of the base exists() method that don't
+    //use getFileStatus() as an existence probe
+    assertTrue("FileSystem.exists() fails for root", fs.exists(path("/")));
+  }
+
+  /**
+   * Assert that root directory renames are not allowed
+   * @throws Exception on failures
+   */
+  public void testRenameRootDirForbidden() throws Exception {
+    if (!renameSupported()) return;
+
+    rename(path("/"),
+           path("/test/newRootDir"),
+           false, true, false);
+  }
+
+  /**
+   * Assert that renaming a parent directory to be a child
+   * of itself is forbidden
+   * @throws Exception on failures
+   */
+  public void testRenameChildDirForbidden() throws Exception {
+    if (!renameSupported()) return;
+    LOG.info("testRenameChildDirForbidden");
+    Path parentdir = path("/test/parentdir");
+    fs.mkdirs(parentdir);
+    Path childFile = new Path(parentdir, "childfile");
+    createFile(childFile);
+    //verify one level down
+    Path childdir = new Path(parentdir, "childdir");
+    rename(parentdir, childdir, false, true, false);
+    //now another level
+    fs.mkdirs(childdir);
+    Path childchilddir = new Path(childdir, "childdir");
+    rename(parentdir, childchilddir, false, true, false);
+  }
+
+  /**
+   * This a sanity check to make sure that any filesystem's handling of
+   * renames doesn't cause any regressions
+   */
+  public void testRenameToDirWithSamePrefixAllowed() throws Throwable {
+    if (!renameSupported()) return;
+    Path parentdir = path("test/parentdir");
+    fs.mkdirs(parentdir);
+    Path dest = path("test/parentdirdest");
+    rename(parentdir, dest, true, false, true);
+  }
+
+  /**
+   * trying to rename a directory onto itself should fail,
+   * preserving everything underneath.
+   */
+  public void testRenameDirToSelf() throws Throwable {
+    if (!renameSupported()) {
+      return;
+    }
+    Path parentdir = path("test/parentdir");
+    fs.mkdirs(parentdir);
+    Path child = new Path(parentdir, "child");
+    createFile(child);
+
+    rename(parentdir, parentdir, false, true, true);
+    //verify the child is still there
+    assertIsFile(child);
+  }
+
+  /**
+   * trying to rename a directory onto its parent dir will build
+   * a destination path of its original name, which should then fail.
+   * The source path and the destination path should still exist afterwards
+   */
+  public void testMoveDirUnderParent() throws Throwable {
+    if (!renameSupported()) {
+      return;
+    }
+    Path testdir = path("test/dir");
+    fs.mkdirs(testdir);
+    Path parent = testdir.getParent();
+    //the outcome here is ambiguous, so is not checked
+    fs.rename(testdir, parent);
+    assertEquals("Source exists: " + testdir, true, fs.exists(testdir));
+    assertEquals("Destination exists" + parent, true, fs.exists(parent));
+  }
+
+  /**
+   * trying to rename a file onto itself should succeed (it's a no-op)
+   *
+   */
+  public void testRenameFileToSelf() throws Throwable {
+    if (!renameSupported()) return;
+    Path filepath = path("test/file");
+    createFile(filepath);
+    //HDFS expects rename src, src -> true
+    rename(filepath, filepath, true, true, true);
+    //verify the file is still there
+    assertIsFile(filepath);
+  }
+
+  /**
+   * trying to move a file into it's parent dir should succeed
+   * again: no-op
+   */
+  public void testMoveFileUnderParent() throws Throwable {
+    if (!renameSupported()) return;
+    Path filepath = path("test/file");
+    createFile(filepath);
+    //HDFS expects rename src, src -> true
+    rename(filepath, filepath, true, true, true);
+    //verify the file is still there
+    assertIsFile(filepath);
+  }
+
+  public void testLSRootDir() throws Throwable {
+    Path dir = path("/");
+    Path child = path("/test");
+    createFile(child);
+    assertListFilesFinds(dir, child);
+  }
+
+  public void testListStatusRootDir() throws Throwable {
+    Path dir = path("/");
+    Path child  = path("/test");
+    createFile(child);
+    assertListStatusFinds(dir, child);
+  }
+
+  private void assertListFilesFinds(Path dir, Path subdir) throws IOException {
+    RemoteIterator<LocatedFileStatus> iterator =
+      fs.listFiles(dir, true);
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    while (iterator.hasNext()) {
+      LocatedFileStatus next =  iterator.next();
+      builder.append(next.toString()).append('\n');
+      if (next.getPath().equals(subdir)) {
+        found = true;
+      }
+    }
+    assertTrue("Path " + subdir
+               + " not found in directory " + dir + ":" + builder,
+               found);
+  }
+
+  private void assertListStatusFinds(Path dir, Path subdir) throws IOException {
+    FileStatus[] stats = fs.listStatus(dir);
+    boolean found = false;
+    StringBuilder builder = new StringBuilder();
+    for (FileStatus stat : stats) {
+      builder.append(stat.toString()).append('\n');
+      if (stat.getPath().equals(subdir)) {
+        found = true;
+      }
+    }
+    assertTrue("Path " + subdir
+               + " not found in directory " + dir + ":" + builder,
+               found);
+  }
+
+
+  /**
+   * Assert that a file exists and whose {@link FileStatus} entry
+   * declares that this is a file and not a symlink or directory.
+   * @param filename name of the file
+   * @throws IOException IO problems during file operations
+   */
+  private void assertIsFile(Path filename) throws IOException {
+    assertTrue("Does not exist: " + filename, fs.exists(filename));
+    FileStatus status = fs.getFileStatus(filename);
+    String fileInfo = filename + "  " + status;
+    assertTrue("Not a file " + fileInfo, status.isFile());
+    assertFalse("File claims to be a symlink " + fileInfo,
+                status.isSymlink());
+    assertFalse("File claims to be a directory " + fileInfo,
+                status.isDirectory());
+  }
+
   /**
    *
    * Write a file and read it in, validating the result. Optional flags control

+ 1 - 0
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/s3/InMemoryFileSystemStore.java

@@ -50,6 +50,7 @@ class InMemoryFileSystemStore implements FileSystemStore {
   @Override
   public void initialize(URI uri, Configuration conf) {
     this.conf = conf;
+    inodes.put(new Path("/"), INode.DIRECTORY_INODE);
   }
   
   @Override

+ 1 - 1
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/s3native/NativeS3FileSystemContractBaseTest.java

@@ -51,7 +51,7 @@ public abstract class NativeS3FileSystemContractBaseTest
   
   public void testListStatusForRoot() throws Exception {
     FileStatus[] paths = fs.listStatus(path("/"));
-    assertEquals(0, paths.length);
+    assertEquals("Root directory is not empty; ", 0, paths.length);
     
     Path testDir = path("/test");
     assertTrue(fs.mkdirs(testDir));