Sfoglia il codice sorgente

HDFS-17168. Support getTrashRoots API in WebHDFS (#5992) Contributed by Hualong Zhang.

Reviewed-by: Shilun Fan <slfan1989@apache.org>
Signed-off-by: Shilun Fan <slfan1989@apache.org>
zhtttylz 1 anno fa
parent
commit
9857fbcd61

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOpsCountStatistics.java

@@ -75,6 +75,7 @@ public class DFSOpsCountStatistics extends StorageStatistics {
     GET_STORAGE_POLICIES("op_get_storage_policies"),
     GET_STORAGE_POLICY("op_get_storage_policy"),
     GET_TRASH_ROOT("op_get_trash_root"),
+    GET_TRASH_ROOTS("op_get_trash_roots"),
     GET_XATTR("op_get_xattr"),
     LIST_CACHE_DIRECTIVE("op_list_cache_directive"),
     LIST_CACHE_POOL("op_list_cache_pool"),

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -3612,6 +3612,8 @@ public class DistributedFileSystem extends FileSystem
    */
   @Override
   public Collection<FileStatus> getTrashRoots(boolean allUsers) {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_TRASH_ROOTS);
     Set<FileStatus> ret = new HashSet<>();
     // Get normal trash roots
     ret.addAll(super.getTrashRoots(allUsers));

+ 35 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java

@@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectReader;
 
 import org.apache.hadoop.classification.VisibleForTesting;
 import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 import org.apache.hadoop.fs.ContentSummary;
@@ -877,6 +878,40 @@ public class JsonUtilClient {
     return map;
   }
 
+  public static Collection<FileStatus> getTrashRoots(Map<?, ?> json) {
+    List<?> objs = (List<?>) json.get("Paths");
+    if (objs != null) {
+      FileStatus[] trashRoots = new FileStatus[objs.size()];
+      for (int i = 0; i < objs.size(); i++) {
+        Map<?, ?> m = (Map<?, ?>) objs.get(i);
+        trashRoots[i] = toFileStatus(m);
+      }
+      return Arrays.asList(trashRoots);
+    }
+    return new ArrayList<FileStatus>(0);
+  }
+
+  public static FileStatus toFileStatus(Map<?, ?> json) {
+    Path path = new Path(getString(json, "path", ""));
+    long length = getLong(json, "length", 0);
+    boolean isdir = getBoolean(json, "isdir", false);
+    short replication = (short) getInt(json, "block_replication", -1);
+    long blockSize = getLong(json, "blocksize", 256);
+    long modificationTime = getLong(json, "modification_time", 0);
+    long accessTime = getLong(json, "access_time", 0);
+    String permString = getString(json, "permission", null);
+    FsPermission permission = toFsPermission(permString);
+    String owner = getString(json, "owner", null);
+    String group = getString(json, "group", null);
+    if (json.get("symlink") != null) {
+      Path symlink = new Path((String) json.get("symlink"));
+      return new FileStatus(length, isdir, replication, blockSize, modificationTime,
+          accessTime, permission, owner, group, symlink, path);
+    }
+    return new FileStatus(length, isdir, replication, blockSize, modificationTime,
+        accessTime, permission, owner, group, path);
+  }
+
   private static List<SnapshotDiffReport.DiffReportEntry> toDiffList(
       List<?> objs) {
     if (objs == null) {

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -1956,6 +1956,26 @@ public class WebHdfsFileSystem extends FileSystem
     }
   }
 
+  public Collection<FileStatus> getTrashRoots(boolean allUsers) {
+    statistics.incrementReadOps(1);
+    storageStatistics.incrementOpCounter(OpType.GET_TRASH_ROOTS);
+
+    final HttpOpParam.Op op = GetOpParam.Op.GETTRASHROOTS;
+    try {
+      Collection<FileStatus> trashRoots =
+          new FsPathResponseRunner<Collection<FileStatus>>(op, null,
+              new AllUsersParam(allUsers)) {
+            @Override
+            Collection<FileStatus> decodeResponse(Map<?, ?> json) throws IOException {
+              return JsonUtilClient.getTrashRoots(json);
+            }
+          }.run();
+      return trashRoots;
+    } catch (IOException e) {
+      return super.getTrashRoots(allUsers);
+    }
+  }
+
   @Override
   public void access(final Path path, final FsAction mode) throws IOException {
     final HttpOpParam.Op op = GetOpParam.Op.CHECKACCESS;

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/AllUsersParam.java

@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.web.resources;
+
+/** AllUsers parameter. */
+public class AllUsersParam extends BooleanParam {
+  /** Parameter name. */
+  public static final String NAME = "allusers";
+  /** Default parameter value. */
+  public static final String DEFAULT = FALSE;
+
+  private static final Domain DOMAIN = new Domain(NAME);
+
+  /**
+   * Constructor.
+   * @param value the parameter value.
+   */
+  public AllUsersParam(final Boolean value) {
+    super(DOMAIN, value);
+  }
+
+  /**
+   * Constructor.
+   * @param str a string representation of the parameter value.
+   */
+  public AllUsersParam(final String str) {
+    this(DOMAIN.parse(str));
+  }
+
+  @Override
+  public String getName() {
+    return NAME;
+  }
+}

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java

@@ -69,6 +69,7 @@ public class GetOpParam extends HttpOpParam<GetOpParam.Op> {
     GETSTATUS(false, HttpURLConnection.HTTP_OK),
     GETECPOLICIES(false, HttpURLConnection.HTTP_OK),
     GETECCODECS(false, HttpURLConnection.HTTP_OK),
+    GETTRASHROOTS(false, HttpURLConnection.HTTP_OK),
     GETSNAPSHOTLIST(false, HttpURLConnection.HTTP_OK);
 
     final boolean redirect;

+ 4 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.hdfs.web.ParamFilter;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.AclPermissionParam;
+import org.apache.hadoop.hdfs.web.resources.AllUsersParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
 import org.apache.hadoop.hdfs.web.resources.ConcatSourcesParam;
@@ -344,7 +345,8 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final NoRedirectParam noredirectParam,
-      final StartAfterParam startAfter
+      final StartAfterParam startAfter,
+      final AllUsersParam allUsers
   ) throws IOException, URISyntaxException {
     try {
       final Router router = getRouter();
@@ -393,7 +395,7 @@ public class RouterWebHdfsMethods extends NamenodeWebHdfsMethods {
             offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             snapshotDiffStartPath, snapshotDiffIndex,
-            tokenKind, tokenService, noredirectParam, startAfter);
+            tokenKind, tokenService, noredirectParam, startAfter, allUsers);
       }
       default:
         throw new UnsupportedOperationException(op + " is not supported");

+ 25 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -30,6 +30,7 @@ import java.security.Principal;
 import java.security.PrivilegedExceptionAction;
 import java.util.Base64;
 import java.util.Base64.Encoder;
+import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.List;
@@ -56,6 +57,7 @@ import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
 
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.InvalidPathException;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
@@ -1067,14 +1069,16 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
           final NoRedirectParam noredirect,
       @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
-          final StartAfterParam startAfter
+          final StartAfterParam startAfter,
+      @QueryParam(AllUsersParam.NAME) @DefaultValue(AllUsersParam.DEFAULT)
+          final AllUsersParam allUsers
       ) throws IOException, InterruptedException {
     return get(ugi, delegation, username, doAsUser, ROOT, op, offset, length,
         renewer, bufferSize, xattrNames, xattrEncoding, excludeDatanodes,
         fsAction, snapshotName, oldSnapshotName,
         snapshotDiffStartPath, snapshotDiffIndex,
         tokenKind, tokenService,
-        noredirect, startAfter);
+        noredirect, startAfter, allUsers);
   }
 
   /** Handle HTTP GET request. */
@@ -1124,12 +1128,14 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(NoRedirectParam.NAME) @DefaultValue(NoRedirectParam.DEFAULT)
           final NoRedirectParam noredirect,
       @QueryParam(StartAfterParam.NAME) @DefaultValue(StartAfterParam.DEFAULT)
-          final StartAfterParam startAfter
+          final StartAfterParam startAfter,
+      @QueryParam(AllUsersParam.NAME) @DefaultValue(AllUsersParam.DEFAULT)
+          final AllUsersParam allUsers
       ) throws IOException, InterruptedException {
 
     init(ugi, delegation, username, doAsUser, path, op, offset, length,
         renewer, bufferSize, xattrEncoding, excludeDatanodes, fsAction,
-        snapshotName, oldSnapshotName, tokenKind, tokenService, startAfter);
+        snapshotName, oldSnapshotName, tokenKind, tokenService, startAfter, allUsers);
 
     return doAs(ugi, new PrivilegedExceptionAction<Response>() {
       @Override
@@ -1138,7 +1144,7 @@ public class NamenodeWebHdfsMethods {
             op, offset, length, renewer, bufferSize, xattrNames, xattrEncoding,
             excludeDatanodes, fsAction, snapshotName, oldSnapshotName,
             snapshotDiffStartPath, snapshotDiffIndex,
-            tokenKind, tokenService, noredirect, startAfter);
+            tokenKind, tokenService, noredirect, startAfter, allUsers);
       }
     });
   }
@@ -1172,7 +1178,8 @@ public class NamenodeWebHdfsMethods {
       final TokenKindParam tokenKind,
       final TokenServiceParam tokenService,
       final NoRedirectParam noredirectParam,
-      final StartAfterParam startAfter
+      final StartAfterParam startAfter,
+      final AllUsersParam allUsers
       ) throws IOException, URISyntaxException {
     final Configuration conf = (Configuration) context
         .getAttribute(JspHelper.CURRENT_CONF);
@@ -1322,6 +1329,12 @@ public class NamenodeWebHdfsMethods {
       final String jsonStr = JsonUtil.toJsonString("Path", trashPath);
       return Response.ok(jsonStr).type(MediaType.APPLICATION_JSON).build();
     }
+    case GETTRASHROOTS: {
+      Boolean value = allUsers.getValue();
+      final Collection<FileStatus> trashPaths = getTrashRoots(conf, value);
+      final String js = JsonUtil.toJsonString(trashPaths);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
     case LISTSTATUS_BATCH:
     {
       byte[] start = HdfsFileStatus.EMPTY_NAME;
@@ -1558,6 +1571,12 @@ public class NamenodeWebHdfsMethods {
     };
   }
 
+  private Collection<FileStatus> getTrashRoots(Configuration conf, boolean allUsers)
+      throws IOException {
+    FileSystem fs = FileSystem.get(conf != null ? conf : new Configuration());
+    return fs.getTrashRoots(allUsers);
+  }
+
 
   /** Handle HTTP DELETE request for the root. */
   @DELETE

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/JsonUtil.java

@@ -752,6 +752,27 @@ public class JsonUtil {
     return m;
   }
 
+  public static Map<String, Object> toJsonMap(FileStatus status) throws IOException {
+    Map<String, Object> m = new HashMap<>();
+    m.put("path", status.getPath().toUri().getPath());
+    m.put("length", status.getLen());
+    m.put("isdir", status.isDirectory());
+    m.put("block_replication", status.getReplication());
+    m.put("blocksize", status.getBlockSize());
+    m.put("modification_time", status.getModificationTime());
+    m.put("access_time", status.getAccessTime());
+    FsPermission perm = status.getPermission();
+    if (perm != null) {
+      m.put("permission", toString(perm));
+    }
+    m.put("owner", status.getOwner());
+    m.put("group", status.getGroup());
+    if (status.isSymlink()) {
+      m.put("symlink", status.getSymlink());
+    }
+    return m;
+  }
+
   public static String toJsonString(ErasureCodingPolicyInfo[] ecPolicyInfos) {
     final Map<String, Object> erasureCodingPolicies = new HashMap<>();
     Object[] erasureCodingPolicyInfos = null;
@@ -764,4 +785,16 @@ public class JsonUtil {
     erasureCodingPolicies.put("ErasureCodingPolicyInfo", erasureCodingPolicyInfos);
     return toJsonString("ErasureCodingPolicies", erasureCodingPolicies);
   }
+
+  public static String toJsonString(Collection<FileStatus> trashPaths) throws IOException {
+    List<FileStatus> list = new ArrayList<>(trashPaths);
+    Object[] paths = null;
+    if(list.size() > 0){
+      paths = new Object[list.size()];
+      for(int i = 0; i < list.size(); i++){
+        paths[i] = toJsonMap(list.get(i));
+      }
+    }
+    return toJsonString("Paths", paths);
+  }
 }

+ 61 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -63,6 +63,7 @@ The HTTP REST API supports the complete [FileSystem](../../api/org/apache/hadoop
     * [`GETSTATUS`](#Get_Status) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStatus)
     * [`GETECPOLICIES`](#Get_EC_Policies)
     * [`GETECCODECS`](#Get_EC_Codecs)
+    * [`GETTRASHROOTS`](#Get_Trash_Roots) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getTrashRoots)
 *   HTTP PUT
     * [`CREATE`](#Create_and_Write_to_a_File) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).create)
     * [`MKDIRS`](#Make_a_Directory) (see [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).mkdirs)
@@ -1273,6 +1274,47 @@ See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getStatus
             }
         }
 
+### Get Trash Roots
+
+* Submit a HTTP GET request.
+
+        curl -i "http://<HOST>:<PORT>/webhdfs/v1/<PATH>?op=GETTRASHROOTS
+                                      &allusers=<true|false>"
+
+  The client receives a response with a [`Paths` JSON object](#Paths_JSON_Schema):
+
+        HTTP/1.1 200 OK
+        Content-Type: application/json
+        Transfer-Encoding: chunked
+
+        {
+            "Paths": [{
+                "blocksize": 0,
+                "owner": "hadoop",
+                "path": "/user/user0/.Trash",
+                "length": 0,
+                "permission": "755",
+                "modification_time": 1693050205747,
+                "isdir": true,
+                "block_replication": 0,
+                "access_time": 0,
+                "group": "supergroup"
+             }, {
+                "blocksize": 0,
+                "owner": "hadoop",
+                "path": "/user/user1/.Trash",
+                "length": 0,
+                "permission": "755",
+                "modification_time": 1693049382962,
+                "isdir": true,
+                "block_replication": 0,
+                "access_time": 0,
+                "group": "supergroup"
+             }]
+        }
+
+See also: [FileSystem](../../api/org/apache/hadoop/fs/FileSystem.html).getTrashRoots
+
 Storage Policy Operations
 -------------------------
 
@@ -3277,6 +3319,25 @@ var blockLocationProperties =
 }
 ```
 
+### Paths JSON Schema
+
+```json
+{
+  "Paths": [{
+    "blocksize": 0,
+    "owner": "hadoop",
+    "path": "/user/user0/.Trash",
+    "length": 0,
+    "permission": "755",
+    "modification_time": 1693050205747,
+    "isdir": true,
+    "block_replication": 0,
+    "access_time": 0,
+    "group": "supergroup"
+  }]
+}
+```
+
 HTTP Query Parameter Dictionary
 -------------------------------
 

+ 29 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -60,6 +60,7 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.Random;
 
+import org.apache.hadoop.hdfs.DFSUtilClient;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.QuotaUsage;
@@ -2342,6 +2343,34 @@ public class TestWebHDFS {
     }
   }
 
+  @Test
+  public void testGetTrashRoots() throws Exception {
+    final Configuration conf = WebHdfsTestUtil.createConf();
+    cluster = new MiniDFSCluster.Builder(conf).build();
+    final WebHdfsFileSystem webFS = WebHdfsTestUtil.getWebHdfsFileSystem(conf,
+        WebHdfsConstants.WEBHDFS_SCHEME);
+
+    // Create user trash
+    Path currUserHome = webFS.getHomeDirectory();
+    Path currUserTrash = new Path(currUserHome, FileSystem.TRASH_PREFIX);
+    webFS.mkdirs(currUserTrash);
+
+    Collection<FileStatus> webTrashRoots = webFS.getTrashRoots(true);
+    assertEquals(1, webTrashRoots.size());
+
+    // Create trash root for user0
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("user0");
+    String user0HomeStr = DFSUtilClient.getHomeDirectory(conf, ugi);
+    Path user0Trash = new Path(user0HomeStr, FileSystem.TRASH_PREFIX);
+    webFS.mkdirs(user0Trash);
+
+    webTrashRoots = webFS.getTrashRoots(true);
+    assertEquals(2, webTrashRoots.size());
+
+    webTrashRoots = webFS.getTrashRoots(false);
+    assertEquals(1,  webTrashRoots.size());
+  }
+
   /**
    * Get FileStatus JSONObject from ListStatus response.
    */