浏览代码

HADOOP-14726. Mark FileStatus::isDir as final

Chris Douglas 7 年之前
父节点
当前提交
645a8f2a4d
共有 20 个文件被更改,包括 46 次插入66 次删除
  1. 9 10
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java
  2. 1 7
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java
  3. 0 6
      hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java
  4. 1 1
      hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java
  5. 4 2
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java
  6. 4 4
      hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java
  7. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java
  8. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
  9. 1 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java
  10. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java
  11. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java
  12. 2 2
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java
  13. 3 3
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java
  14. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java
  15. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java
  16. 4 4
      hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java
  17. 4 12
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java
  18. 2 2
      hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java
  19. 2 2
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java
  20. 1 1
      hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java

+ 9 - 10
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java

@@ -172,7 +172,7 @@ public class FileStatus implements Writable, Comparable<Object>,
    * @return true if this is a file
    * @return true if this is a file
    */
    */
   public boolean isFile() {
   public boolean isFile() {
-    return !isdir && !isSymlink();
+    return !isDirectory() && !isSymlink();
   }
   }
 
 
   /**
   /**
@@ -182,20 +182,20 @@ public class FileStatus implements Writable, Comparable<Object>,
   public boolean isDirectory() {
   public boolean isDirectory() {
     return isdir;
     return isdir;
   }
   }
-  
+
   /**
   /**
-   * Old interface, instead use the explicit {@link FileStatus#isFile()}, 
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * Old interface, instead use the explicit {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * @return true if this is a directory.
    * @return true if this is a directory.
-   * @deprecated Use {@link FileStatus#isFile()},  
-   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()} 
+   * @deprecated Use {@link FileStatus#isFile()},
+   * {@link FileStatus#isDirectory()}, and {@link FileStatus#isSymlink()}
    * instead.
    * instead.
    */
    */
   @Deprecated
   @Deprecated
-  public boolean isDir() {
-    return isdir;
+  public final boolean isDir() {
+    return isDirectory();
   }
   }
-  
+
   /**
   /**
    * Is this a symbolic link?
    * Is this a symbolic link?
    * @return true if this is a symbolic link
    * @return true if this is a symbolic link
@@ -448,7 +448,6 @@ public class FileStatus implements Writable, Comparable<Object>,
     FileStatus other = PBHelper.convert(proto);
     FileStatus other = PBHelper.convert(proto);
     isdir = other.isDirectory();
     isdir = other.isDirectory();
     length = other.getLen();
     length = other.getLen();
-    isdir = other.isDirectory();
     block_replication = other.getReplication();
     block_replication = other.getReplication();
     blocksize = other.getBlockSize();
     blocksize = other.getBlockSize();
     modification_time = other.getModificationTime();
     modification_time = other.getModificationTime();

+ 1 - 7
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsFileStatus.java

@@ -61,13 +61,7 @@ class ViewFsFileStatus extends FileStatus {
    public boolean isDirectory() {
    public boolean isDirectory() {
      return  myFs.isDirectory();
      return  myFs.isDirectory();
    }
    }
-   
-   @Override
-   @SuppressWarnings("deprecation")
-   public boolean isDir() {
-     return myFs.isDirectory();
-   }
-   
+
    @Override
    @Override
    public boolean isSymlink() {
    public boolean isSymlink() {
      return myFs.isSymlink();
      return myFs.isSymlink();

+ 0 - 6
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFsLocatedFileStatus.java

@@ -49,12 +49,6 @@ class ViewFsLocatedFileStatus extends LocatedFileStatus {
     return myFs.isDirectory();
     return myFs.isDirectory();
   }
   }
 
 
-  @Override
-  @SuppressWarnings("deprecation")
-  public boolean isDir() {
-    return myFs.isDirectory();
-  }
-
   @Override
   @Override
   public boolean isSymlink() {
   public boolean isSymlink() {
     return myFs.isSymlink();
     return myFs.isSymlink();

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java

@@ -2095,7 +2095,7 @@ public class PBHelperClient {
     if (fs == null)
     if (fs == null)
       return null;
       return null;
     FileType fType = FileType.IS_FILE;
     FileType fType = FileType.IS_FILE;
-    if (fs.isDir()) {
+    if (fs.isDirectory()) {
       fType = FileType.IS_DIR;
       fType = FileType.IS_DIR;
     } else if (fs.isSymlink()) {
     } else if (fs.isSymlink()) {
       fType = FileType.IS_SYMLINK;
       fType = FileType.IS_SYMLINK;

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3Utils.java

@@ -65,7 +65,9 @@ public class Nfs3Utils {
      * client takes only the lower 32bit of the fileId and treats it as signed
      * client takes only the lower 32bit of the fileId and treats it as signed
      * int. When the 32th bit is 1, the client considers it invalid.
      * int. When the 32th bit is 1, the client considers it invalid.
      */
      */
-    NfsFileType fileType = fs.isDir() ? NfsFileType.NFSDIR : NfsFileType.NFSREG;
+    NfsFileType fileType = fs.isDirectory()
+        ? NfsFileType.NFSDIR
+        : NfsFileType.NFSREG;
     fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
     fileType = fs.isSymlink() ? NfsFileType.NFSLNK : fileType;
     int nlink = (fileType == NfsFileType.NFSDIR) ? fs.getChildrenNum() + 2 : 1;
     int nlink = (fileType == NfsFileType.NFSDIR) ? fs.getChildrenNum() + 2 : 1;
     long size = (fileType == NfsFileType.NFSDIR) ? getDirSize(fs
     long size = (fileType == NfsFileType.NFSDIR) ? getDirSize(fs
@@ -98,7 +100,7 @@ public class Nfs3Utils {
       return null;
       return null;
     }
     }
 
 
-    long size = fstat.isDir() ? getDirSize(fstat.getChildrenNum()) : fstat
+    long size = fstat.isDirectory() ? getDirSize(fstat.getChildrenNum()) : fstat
         .getLen();
         .getLen();
     return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
     return new WccAttr(size, new NfsTime(fstat.getModificationTime()),
         new NfsTime(fstat.getModificationTime()));
         new NfsTime(fstat.getModificationTime()));

+ 4 - 4
hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/RpcProgramNfs3.java

@@ -1208,7 +1208,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
       if (fstat == null) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
         return new REMOVE3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
       }
-      if (fstat.isDir()) {
+      if (fstat.isDirectory()) {
         return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, errWcc);
         return new REMOVE3Response(Nfs3Status.NFS3ERR_ISDIR, errWcc);
       }
       }
 
 
@@ -1289,7 +1289,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
       if (fstat == null) {
       if (fstat == null) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOENT, errWcc);
       }
       }
-      if (!fstat.isDir()) {
+      if (!fstat.isDirectory()) {
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, errWcc);
         return new RMDIR3Response(Nfs3Status.NFS3ERR_NOTDIR, errWcc);
       }
       }
 
 
@@ -1565,7 +1565,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
         return new READDIR3Response(Nfs3Status.NFS3ERR_STALE);
       }
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdir for regular file, fileId: "
         LOG.error("Can't readdir for regular file, fileId: "
             + handle.getFileId());
             + handle.getFileId());
         return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
         return new READDIR3Response(Nfs3Status.NFS3ERR_NOTDIR);
@@ -1732,7 +1732,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         LOG.info("Can't get path for fileId: " + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_STALE);
       }
       }
-      if (!dirStatus.isDir()) {
+      if (!dirStatus.isDirectory()) {
         LOG.error("Can't readdirplus for regular file, fileId: "
         LOG.error("Can't readdirplus for regular file, fileId: "
             + handle.getFileId());
             + handle.getFileId());
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);
         return new READDIRPLUS3Response(Nfs3Status.NFS3ERR_NOTDIR);

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java

@@ -348,7 +348,7 @@ public class Mover {
     private void processRecursively(String parent, HdfsFileStatus status,
     private void processRecursively(String parent, HdfsFileStatus status,
         Result result) {
         Result result) {
       String fullPath = status.getFullName(parent);
       String fullPath = status.getFullName(parent);
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
         if (!fullPath.endsWith(Path.SEPARATOR)) {
           fullPath = fullPath + Path.SEPARATOR;
           fullPath = fullPath + Path.SEPARATOR;
         }
         }

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java

@@ -471,7 +471,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
   void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
   void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes)
       throws IOException {
       throws IOException {
     String path = file.getFullName(parent);
     String path = file.getFullName(parent);
-    if (file.isDir()) {
+    if (file.isDirectory()) {
       checkDir(path, replRes, ecRes);
       checkDir(path, replRes, ecRes);
       return;
       return;
     }
     }
@@ -1115,7 +1115,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       if (lfStatus == null) { // not exists
       if (lfStatus == null) { // not exists
         lfInitedOk = dfs.mkdirs(lfName, null, true);
         lfInitedOk = dfs.mkdirs(lfName, null, true);
         lostFound = lfName;
         lostFound = lfName;
-      } else if (!lfStatus.isDir()) { // exists but not a directory
+      } else if (!lfStatus.isDirectory()) { // exists but not a directory
         LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
         LOG.warn("Cannot use /lost+found : a regular file with this name exists.");
         lfInitedOk = false;
         lfInitedOk = false;
       }  else { // exists and is a directory
       }  else { // exists and is a directory

+ 1 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgradeFromImage.java

@@ -51,7 +51,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.apache.log4j.Logger;
 import org.apache.log4j.Logger;
 import org.junit.Test;
 import org.junit.Test;
 
 
-import static org.apache.hadoop.hdfs.inotify.Event.CreateEvent;
 import static org.junit.Assert.*;
 import static org.junit.Assert.*;
 
 
 /**
 /**
@@ -572,7 +571,7 @@ public class TestDFSUpgradeFromImage {
       Path path) throws IOException {
       Path path) throws IOException {
     String pathStr = path.toString();
     String pathStr = path.toString();
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
     HdfsFileStatus status = dfs.getFileInfo(pathStr);
-    if (!status.isDir()) {
+    if (!status.isDirectory()) {
       for (int retries = 10; retries > 0; retries--) {
       for (int retries = 10; retries > 0; retries--) {
         if (dfs.recoverLease(pathStr)) {
         if (dfs.recoverLease(pathStr)) {
           return;
           return;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestStorageMover.java

@@ -283,7 +283,7 @@ public class TestStorageMover {
 
 
     private void verifyRecursively(final Path parent,
     private void verifyRecursively(final Path parent,
         final HdfsFileStatus status) throws Exception {
         final HdfsFileStatus status) throws Exception {
-      if (status.isDir()) {
+      if (status.isDirectory()) {
         Path fullPath = parent == null ?
         Path fullPath = parent == null ?
             new Path("/") : status.getFullPath(parent);
             new Path("/") : status.getFullPath(parent);
         DirectoryListing children = dfs.getClient().listPaths(
         DirectoryListing children = dfs.getClient().listPaths(

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStartup.java

@@ -450,7 +450,7 @@ public class TestStartup {
     namenode.getNamesystem().mkdirs("/test",
     namenode.getNamesystem().mkdirs("/test",
         new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
         new PermissionStatus("hairong", null, FsPermission.getDefault()), true);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();
     namenode.stop();
@@ -481,7 +481,7 @@ public class TestStartup {
   private void checkNameSpace(Configuration conf) throws IOException {
   private void checkNameSpace(Configuration conf) throws IOException {
     NameNode namenode = new NameNode(conf);
     NameNode namenode = new NameNode(conf);
     NamenodeProtocols nnRpc = namenode.getRpcServer();
     NamenodeProtocols nnRpc = namenode.getRpcServer();
-    assertTrue(nnRpc.getFileInfo("/test").isDir());
+    assertTrue(nnRpc.getFileInfo("/test").isDirectory());
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false);
     nnRpc.saveNamespace(0, 0);
     nnRpc.saveNamespace(0, 0);
     namenode.stop();
     namenode.stop();

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestEditLogTailer.java

@@ -124,7 +124,7 @@ public class TestEditLogTailer {
       
       
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
       for (int i = 0; i < DIRS_TO_MAKE / 2; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
       }
       
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
@@ -137,7 +137,7 @@ public class TestEditLogTailer {
       
       
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
       for (int i = DIRS_TO_MAKE / 2; i < DIRS_TO_MAKE; i++) {
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
         assertTrue(NameNodeAdapter.getFileInfo(nn2,
-            getDirPath(i), false).isDir());
+            getDirPath(i), false).isDirectory());
       }
       }
     } finally {
     } finally {
       cluster.shutdown();
       cluster.shutdown();

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestFailureToReadEdits.java

@@ -205,7 +205,7 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
         TEST_DIR1, false));
     // Should have been successfully created.
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Null because it hasn't been created yet.
     // Null because it hasn't been created yet.
     assertNull(NameNodeAdapter.getFileInfo(nn1,
     assertNull(NameNodeAdapter.getFileInfo(nn1,
         TEST_DIR3, false));
         TEST_DIR3, false));
@@ -219,10 +219,10 @@ public class TestFailureToReadEdits {
         TEST_DIR1, false));
         TEST_DIR1, false));
     // Should have been successfully created.
     // Should have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR2, false).isDir());
+        TEST_DIR2, false).isDirectory());
     // Should now have been successfully created.
     // Should now have been successfully created.
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
     assertTrue(NameNodeAdapter.getFileInfo(nn1,
-        TEST_DIR3, false).isDir());
+        TEST_DIR3, false).isDirectory());
   }
   }
   
   
   /**
   /**

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestInitializeSharedEdits.java

@@ -128,7 +128,7 @@ public class TestInitializeSharedEdits {
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
       HATestUtil.waitForStandbyToCatchUp(cluster.getNameNode(0),
           cluster.getNameNode(1));
           cluster.getNameNode(1));
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
       assertTrue(NameNodeAdapter.getFileInfo(cluster.getNameNode(1),
-          newPath.toString(), false).isDir());
+          newPath.toString(), false).isDirectory());
     } finally {
     } finally {
       if (fs != null) {
       if (fs != null) {
         fs.close();
         fs.close();

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestCombineFileInputFormat.java

@@ -154,7 +154,7 @@ public class TestCombineFileInputFormat {
     @Override
     @Override
     public BlockLocation[] getFileBlockLocations(
     public BlockLocation[] getFileBlockLocations(
         FileStatus stat, long start, long len) throws IOException {
         FileStatus stat, long start, long len) throws IOException {
-      if (stat.isDir()) {
+      if (stat.isDirectory()) {
         return null;
         return null;
       }
       }
       System.out.println("File " + stat.getPath());
       System.out.println("File " + stat.getPath());

+ 4 - 4
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/TestOutOfBandAzureBlobOperations.java

@@ -73,20 +73,20 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
     FileStatus[] obtained = fs.listStatus(new Path("/root/b"));
     assertNotNull(obtained);
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
 
     // List the directory
     // List the directory
     obtained = fs.listStatus(new Path("/root"));
     obtained = fs.listStatus(new Path("/root"));
     assertNotNull(obtained);
     assertNotNull(obtained);
     assertEquals(1, obtained.length);
     assertEquals(1, obtained.length);
-    assertFalse(obtained[0].isDir());
+    assertFalse(obtained[0].isDirectory());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
     assertEquals("/root/b", obtained[0].getPath().toUri().getPath());
 
 
     // Get the directory's file status
     // Get the directory's file status
     FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
     FileStatus dirStatus = fs.getFileStatus(new Path("/root"));
     assertNotNull(dirStatus);
     assertNotNull(dirStatus);
-    assertTrue(dirStatus.isDir());
+    assertTrue(dirStatus.isDirectory());
     assertEquals("/root", dirStatus.getPath().toUri().getPath());
     assertEquals("/root", dirStatus.getPath().toUri().getPath());
   }
   }
 
 
@@ -114,7 +114,7 @@ public class TestOutOfBandAzureBlobOperations {
     FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
     FileStatus[] listResult = fs.listStatus(new Path("/root/b"));
     // File should win.
     // File should win.
     assertEquals(1, listResult.length);
     assertEquals(1, listResult.length);
-    assertFalse(listResult[0].isDir());
+    assertFalse(listResult[0].isDirectory());
     try {
     try {
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // Trying to delete root/b/c would cause a dilemma for WASB, so
       // it should throw.
       // it should throw.

+ 4 - 12
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftFileStatus.java

@@ -71,7 +71,7 @@ public class SwiftFileStatus extends FileStatus {
    * @return true if the status is considered to be a file
    * @return true if the status is considered to be a file
    */
    */
   @Override
   @Override
-  public boolean isDir() {
+  public boolean isDirectory() {
     return super.isDirectory() || getLen() == 0;
     return super.isDirectory() || getLen() == 0;
   }
   }
 
 
@@ -79,19 +79,11 @@ public class SwiftFileStatus extends FileStatus {
    * A entry is a file if it is not a directory.
    * A entry is a file if it is not a directory.
    * By implementing it <i>and not marking as an override</i> this
    * By implementing it <i>and not marking as an override</i> this
    * subclass builds and runs in both Hadoop versions.
    * subclass builds and runs in both Hadoop versions.
-   * @return the opposite value to {@link #isDir()}
+   * @return the opposite value to {@link #isDirectory()}
    */
    */
   @Override
   @Override
   public boolean isFile() {
   public boolean isFile() {
-    return !isDir();
-  }
-
-  /**
-   * Directory test
-   * @return true if the file is considered to be a directory
-   */
-  public boolean isDirectory() {
-    return isDir();
+    return !this.isDirectory();
   }
   }
 
 
   @Override
   @Override
@@ -100,7 +92,7 @@ public class SwiftFileStatus extends FileStatus {
     sb.append(getClass().getSimpleName());
     sb.append(getClass().getSimpleName());
     sb.append("{ ");
     sb.append("{ ");
     sb.append("path=").append(getPath());
     sb.append("path=").append(getPath());
-    sb.append("; isDirectory=").append(isDir());
+    sb.append("; isDirectory=").append(isDirectory());
     sb.append("; length=").append(getLen());
     sb.append("; length=").append(getLen());
     sb.append("; blocksize=").append(getBlockSize());
     sb.append("; blocksize=").append(getBlockSize());
     sb.append("; modification_time=").append(getModificationTime());
     sb.append("; modification_time=").append(getModificationTime());

+ 2 - 2
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java

@@ -578,7 +578,7 @@ public class SwiftNativeFileSystemStore {
 
 
     //enum the child entries and everything underneath
     //enum the child entries and everything underneath
     List<FileStatus> childStats = listDirectory(srcObject, true, true);
     List<FileStatus> childStats = listDirectory(srcObject, true, true);
-    boolean srcIsFile = !srcMetadata.isDir();
+    boolean srcIsFile = !srcMetadata.isDirectory();
     if (srcIsFile) {
     if (srcIsFile) {
 
 
       //source is a simple file OR a partitioned file
       //source is a simple file OR a partitioned file
@@ -945,7 +945,7 @@ public class SwiftNativeFileSystemStore {
     //>1 entry implies directory with children. Run through them,
     //>1 entry implies directory with children. Run through them,
     // but first check for the recursive flag and reject it *unless it looks
     // but first check for the recursive flag and reject it *unless it looks
     // like a partitioned file (len > 0 && has children)
     // like a partitioned file (len > 0 && has children)
-    if (!fileStatus.isDir()) {
+    if (!fileStatus.isDirectory()) {
       LOG.debug("Multiple child entries but entry has data: assume partitioned");
       LOG.debug("Multiple child entries but entry has data: assume partitioned");
     } else if (!recursive) {
     } else if (!recursive) {
       //if there are children, unless this is a recursive operation, fail immediately
       //if there are children, unless this is a recursive operation, fail immediately

+ 2 - 2
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemDirectories.java

@@ -87,7 +87,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     assertEquals("Wrong number of elements in file status " + statusString, 1,
     assertEquals("Wrong number of elements in file status " + statusString, 1,
                  statuses.length);
                  statuses.length);
     SwiftFileStatus stat = (SwiftFileStatus) statuses[0];
     SwiftFileStatus stat = (SwiftFileStatus) statuses[0];
-    assertTrue("isDir(): Not a directory: " + stat, stat.isDir());
+    assertTrue("isDir(): Not a directory: " + stat, stat.isDirectory());
     extraStatusAssertions(stat);
     extraStatusAssertions(stat);
   }
   }
 
 
@@ -135,7 +135,7 @@ public class TestSwiftFileSystemDirectories extends SwiftFileSystemBaseTest {
     SwiftTestUtils.writeTextFile(fs, src, "testMultiByteFilesAreFiles", false);
     SwiftTestUtils.writeTextFile(fs, src, "testMultiByteFilesAreFiles", false);
     assertIsFile(src);
     assertIsFile(src);
     FileStatus status = fs.getFileStatus(src);
     FileStatus status = fs.getFileStatus(src);
-    assertFalse(status.isDir());
+    assertFalse(status.isDirectory());
   }
   }
 
 
 }
 }

+ 1 - 1
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java

@@ -228,7 +228,7 @@ public class TestSwiftFileSystemPartitionedUploads extends
                  status.getLen());
                  status.getLen());
     String fileInfo = qualifiedPath + "  " + status;
     String fileInfo = qualifiedPath + "  " + status;
     assertFalse("File claims to be a directory " + fileInfo,
     assertFalse("File claims to be a directory " + fileInfo,
-                status.isDir());
+                status.isDirectory());
 
 
     FileStatus listedFileStat = resolveChild(parentDirListing, qualifiedPath);
     FileStatus listedFileStat = resolveChild(parentDirListing, qualifiedPath);
     assertNotNull("Did not find " + path + " in " + parentDirLS,
     assertNotNull("Did not find " + path + " in " + parentDirLS,