Browse Source

HDFS-16990. HttpFS Add Support getFileLinkStatus API (#5602). Contributed by Hualong Zhang.

Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Ayush Saxena <ayushsaxena@apache.org>
zhtttylz 2 years ago
parent
commit
5084e881ef

+ 13 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java

@@ -277,6 +277,7 @@ public class HttpFSFileSystem extends FileSystem
     GETSERVERDEFAULTS(HTTP_GET),
     CHECKACCESS(HTTP_GET), SETECPOLICY(HTTP_PUT), GETECPOLICY(HTTP_GET), UNSETECPOLICY(
         HTTP_POST), SATISFYSTORAGEPOLICY(HTTP_PUT), GETSNAPSHOTDIFFLISTING(HTTP_GET),
+    GETFILELINKSTATUS(HTTP_GET),
     GET_BLOCK_LOCATIONS(HTTP_GET);
 
     private String httpMethod;
@@ -1743,6 +1744,18 @@ public class HttpFSFileSystem extends FileSystem
     return getFileBlockLocations(status.getPath(), offset, length);
   }
 
+  @Override
+  public FileStatus getFileLinkStatus(final Path path) throws IOException {
+    Map<String, String> params = new HashMap<>();
+    params.put(OP_PARAM, Operation.GETFILELINKSTATUS.toString());
+    HttpURLConnection conn =
+        getConnection(Operation.GETFILELINKSTATUS.getMethod(), params, path, true);
+    HttpExceptionUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
+    JSONObject json = (JSONObject) HttpFSUtils.jsonParse(conn);
+    HdfsFileStatus status = JsonUtilClient.toFileStatus(json, true);
+    return status.makeQualified(getUri(), path);
+  }
+
   @VisibleForTesting
   static BlockLocation[] toBlockLocations(JSONObject json) throws IOException {
     ObjectMapper mapper = new ObjectMapper();

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/FSOperations.java

@@ -2265,4 +2265,39 @@ public final class FSOperations {
           "because the file system is not DistributedFileSystem.");
     }
   }
+
+  /**
+   * Executor that performs a linkFile-status FileSystemAccess files
+   * system operation.
+   */
+  @InterfaceAudience.Private
+  @SuppressWarnings("rawtypes")
+  public static class FSFileLinkStatus
+      implements FileSystemAccess.FileSystemExecutor<Map> {
+    final private Path path;
+
+    /**
+     * Creates a linkFile-status executor.
+     *
+     * @param path the path to retrieve the status.
+     */
+    public FSFileLinkStatus(String path) {
+      this.path = new Path(path);
+    }
+
+    /**
+     * Executes the filesystem getFileLinkStatus operation and returns the
+     * result in a JSON Map.
+     *
+     * @param fs filesystem instance to use.
+     * @return a Map object (JSON friendly) with the file status.
+     * @throws IOException thrown if an IO error occurred.
+     */
+    @Override
+    public Map execute(FileSystem fs) throws IOException {
+      FileStatus status = fs.getFileLinkStatus(path);
+      HttpFSServerWebApp.get().getMetrics().incrOpsStat();
+      return toJson(status);
+    }
+  }
 }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -128,6 +128,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
     PARAMS_DEF.put(Operation.GETECPOLICY, new Class[] {});
     PARAMS_DEF.put(Operation.UNSETECPOLICY, new Class[] {});
     PARAMS_DEF.put(Operation.SATISFYSTORAGEPOLICY, new Class[] {});
+    PARAMS_DEF.put(Operation.GETFILELINKSTATUS, new Class[]{});
     PARAMS_DEF.put(Operation.GET_BLOCK_LOCATIONS, new Class[] {OffsetParam.class, LenParam.class});
   }
 

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSServer.java

@@ -546,6 +546,14 @@ public class HttpFSServer {
       response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
       break;
     }
+    case GETFILELINKSTATUS: {
+      FSOperations.FSFileLinkStatus command =
+          new FSOperations.FSFileLinkStatus(path);
+      @SuppressWarnings("rawtypes") Map js = fsExecute(user, command);
+      AUDIT_LOG.info("[{}]", path);
+      response = Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+      break;
+    }
     default: {
       throw new IOException(
           MessageFormat.format("Invalid HTTP GET operation [{0}]", op.value()));

+ 26 - 1
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -1215,7 +1215,8 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     ALLOW_SNAPSHOT, DISALLOW_SNAPSHOT, DISALLOW_SNAPSHOT_EXCEPTION,
     FILE_STATUS_ATTR, GET_SNAPSHOT_DIFF, GET_SNAPSHOTTABLE_DIRECTORY_LIST,
     GET_SNAPSHOT_LIST, GET_SERVERDEFAULTS, CHECKACCESS, SETECPOLICY,
-    SATISFYSTORAGEPOLICY, GET_SNAPSHOT_DIFF_LISTING, GETFILEBLOCKLOCATIONS
+    SATISFYSTORAGEPOLICY, GET_SNAPSHOT_DIFF_LISTING, GETFILEBLOCKLOCATIONS,
+    GETFILELINKSTATUS
   }
 
   private void operation(Operation op) throws Exception {
@@ -1358,6 +1359,9 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     case GETFILEBLOCKLOCATIONS:
       testGetFileBlockLocations();
       break;
+    case GETFILELINKSTATUS:
+      testGetFileLinkStatus();
+      break;
     }
 
   }
@@ -2056,6 +2060,27 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     }
   }
 
+  private void testGetFileLinkStatus() throws Exception {
+    if (isLocalFS()) {
+      // do not test the symlink for local FS.
+      return;
+    }
+    FileSystem fs = FileSystem.get(getProxiedFSConf());
+
+    Path root = new Path(getProxiedFSTestDir(), "httpFSTest");
+    Path file = new Path(root, "file");
+    Path linkToFile = new Path(root, "linkToFile");
+
+    OutputStream os = fs.create(file);
+    os.write(1);
+    fs.createSymlink(file, linkToFile, false);
+
+    fs = this.getHttpFSFileSystem();
+
+    assertFalse(fs.getFileLinkStatus(file).isSymlink());
+    assertTrue(fs.getFileLinkStatus(linkToFile).isSymlink());
+  }
+
   private void assertHttpFsReportListingWithDfsClient(SnapshotDiffReportListing diffReportListing,
       SnapshotDiffReportListing dfsDiffReportListing) {
     Assert.assertEquals(diffReportListing.getCreateList().size(),