You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by xy...@apache.org on 2017/07/19 22:35:38 UTC

[46/50] [abbrv] hadoop git commit: HDFS-12139. HTTPFS liststatus returns incorrect pathSuffix for path of file. Contributed by Yongjun Zhang.

HDFS-12139. HTTPFS liststatus returns incorrect pathSuffix for path of file. Contributed by Yongjun Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/3556e36b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/3556e36b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/3556e36b

Branch: refs/heads/HDFS-7240
Commit: 3556e36be30211f46ac38899ce11a4d4cac6d635
Parents: 413b23e
Author: Yongjun Zhang <yz...@cloudera.com>
Authored: Wed Jul 19 10:54:13 2017 -0700
Committer: Yongjun Zhang <yz...@cloudera.com>
Committed: Wed Jul 19 10:56:50 2017 -0700

----------------------------------------------------------------------
 .../hadoop/fs/http/server/FSOperations.java     | 15 ++++++-----
 .../fs/http/client/BaseTestHttpFSWith.java      | 26 +++++++++++++++++++-
 2 files changed, 34 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/3556e36b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
index 0fb665a..f1615c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java
@@ -75,15 +75,17 @@ public class FSOperations {
 
   /**
    * @param fileStatuses list of FileStatus objects
+   * @param isFile is the fileStatuses from a file path
    * @return JSON map suitable for wire transport
    */
   @SuppressWarnings({"unchecked"})
-  private static Map<String, Object> toJson(FileStatus[] fileStatuses) {
+  private static Map<String, Object> toJson(FileStatus[] fileStatuses,
+      boolean isFile) {
     Map<String, Object> json = new LinkedHashMap<>();
     Map<String, Object> inner = new LinkedHashMap<>();
     JSONArray statuses = new JSONArray();
     for (FileStatus f : fileStatuses) {
-      statuses.add(toJsonInner(f, false));
+      statuses.add(toJsonInner(f, isFile));
     }
     inner.put(HttpFSFileSystem.FILE_STATUS_JSON, statuses);
     json.put(HttpFSFileSystem.FILE_STATUSES_JSON, inner);
@@ -129,13 +131,14 @@ public class FSOperations {
    * These two classes are slightly different, due to the impedance
    * mismatches between the WebHDFS and FileSystem APIs.
    * @param entries
+   * @param isFile is the entries from a file path
    * @return json
    */
   private static Map<String, Object> toJson(FileSystem.DirectoryEntries
-      entries) {
+      entries, boolean isFile) {
     Map<String, Object> json = new LinkedHashMap<>();
     Map<String, Object> inner = new LinkedHashMap<>();
-    Map<String, Object> fileStatuses = toJson(entries.getEntries());
+    Map<String, Object> fileStatuses = toJson(entries.getEntries(), isFile);
     inner.put(HttpFSFileSystem.PARTIAL_LISTING_JSON, fileStatuses);
     inner.put(HttpFSFileSystem.REMAINING_ENTRIES_JSON, entries.hasMore() ? 1
         : 0);
@@ -690,7 +693,7 @@ public class FSOperations {
     @Override
     public Map execute(FileSystem fs) throws IOException {
       FileStatus[] fileStatuses = fs.listStatus(path, filter);
-      return toJson(fileStatuses);
+      return toJson(fileStatuses, fs.getFileStatus(path).isFile());
     }
 
     @Override
@@ -735,7 +738,7 @@ public class FSOperations {
       WrappedFileSystem wrappedFS = new WrappedFileSystem(fs);
       FileSystem.DirectoryEntries entries =
           wrappedFS.listStatusBatch(path, token);
-      return toJson(entries);
+      return toJson(entries, wrappedFS.getFileStatus(path).isFile());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/3556e36b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
index 0fd3f91..e23093e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java
@@ -364,8 +364,15 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     assertEquals(status2.getLen(), status1.getLen());
 
     FileStatus[] stati = fs.listStatus(path.getParent());
-    assertEquals(stati.length, 1);
+    assertEquals(1, stati.length);
     assertEquals(stati[0].getPath().getName(), path.getName());
+
+    // The full path should be the path to the file. See HDFS-12139
+    FileStatus[] statl = fs.listStatus(path);
+    Assert.assertEquals(1, statl.length);
+    Assert.assertEquals(status2.getPath(), statl[0].getPath());
+    Assert.assertEquals(statl[0].getPath().getName(), path.getName());
+    Assert.assertEquals(stati[0].getPath(), statl[0].getPath());
   }
 
   private static void assertSameListing(FileSystem expected, FileSystem
@@ -411,6 +418,23 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       proxyFs.create(new Path(dir, "file" + i)).close();
       assertSameListing(proxyFs, httpFs, dir);
     }
+
+    // Test for HDFS-12139
+    Path dir1 = new Path(getProxiedFSTestDir(), "dir1");
+    proxyFs.mkdirs(dir1);
+    Path file1 = new Path(dir1, "file1");
+    proxyFs.create(file1).close();
+
+    RemoteIterator<FileStatus> si = proxyFs.listStatusIterator(dir1);
+    FileStatus statusl = si.next();
+    FileStatus status = proxyFs.getFileStatus(file1);
+    Assert.assertEquals(file1.getName(), statusl.getPath().getName());
+    Assert.assertEquals(status.getPath(), statusl.getPath());
+
+    si = proxyFs.listStatusIterator(file1);
+    statusl = si.next();
+    Assert.assertEquals(file1.getName(), statusl.getPath().getName());
+    Assert.assertEquals(status.getPath(), statusl.getPath());
   }
 
   private void testWorkingdirectory() throws Exception {


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org