Browse Source

Revert "HDFS-15982. Deleted data using HTTP API should be saved to the trash (#2927)"

This reverts commit 041488e8f533a27e9497e86bb779c0b82c2e1d71.
Ayush Saxena 3 years ago
parent
commit
b8ab19373d

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

@@ -1,50 +0,0 @@
-/*
- * 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;
-
-/**
- * SkipTrash param to be used by DELETE query.
- */
-public class DeleteSkipTrashParam extends BooleanParam {
-
-  public static final String NAME = "skiptrash";
-  public static final String DEFAULT = FALSE;
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   * @param value the parameter value.
-   */
-  public DeleteSkipTrashParam(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  /**
-   * Constructor.
-   * @param str a string representation of the parameter value.
-   */
-  public DeleteSkipTrashParam(final String str) {
-    this(DOMAIN.parse(str));
-  }
-
-  @Override
-  public String getName() {
-    return NAME;
-  }
-}

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

@@ -122,7 +122,6 @@ public class HttpFSFileSystem extends FileSystem
   public static final String ACLSPEC_PARAM = "aclspec";
   public static final String ACLSPEC_PARAM = "aclspec";
   public static final String DESTINATION_PARAM = "destination";
   public static final String DESTINATION_PARAM = "destination";
   public static final String RECURSIVE_PARAM = "recursive";
   public static final String RECURSIVE_PARAM = "recursive";
-  public static final String SKIP_TRASH_PARAM = "skiptrash";
   public static final String SOURCES_PARAM = "sources";
   public static final String SOURCES_PARAM = "sources";
   public static final String OWNER_PARAM = "owner";
   public static final String OWNER_PARAM = "owner";
   public static final String GROUP_PARAM = "group";
   public static final String GROUP_PARAM = "group";

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

@@ -31,7 +31,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
@@ -54,8 +53,6 @@ import org.apache.hadoop.util.StringUtils;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONObject;
 import org.apache.hadoop.fs.permission.FsCreateModes;
 import org.apache.hadoop.fs.permission.FsCreateModes;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
@@ -78,8 +75,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.HTTP_BUFFER_SIZE_DEFAULT;
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public final class FSOperations {
 public final class FSOperations {
 
 
-  private static final Logger LOG = LoggerFactory.getLogger(FSOperations.class);
-
   private static int bufferSize = 4096;
   private static int bufferSize = 4096;
 
 
   private FSOperations() {
   private FSOperations() {
@@ -722,22 +717,18 @@ public final class FSOperations {
    */
    */
   @InterfaceAudience.Private
   @InterfaceAudience.Private
   public static class FSDelete implements FileSystemAccess.FileSystemExecutor<JSONObject> {
   public static class FSDelete implements FileSystemAccess.FileSystemExecutor<JSONObject> {
-    private final Path path;
-    private final boolean recursive;
-    private final boolean skipTrash;
+    private Path path;
+    private boolean recursive;
 
 
     /**
     /**
      * Creates a Delete executor.
      * Creates a Delete executor.
      *
      *
      * @param path path to delete.
      * @param path path to delete.
      * @param recursive if the delete should be recursive or not.
      * @param recursive if the delete should be recursive or not.
-     * @param skipTrash if the file must be deleted and not kept in trash
-     *     regardless of fs.trash.interval config value.
      */
      */
-    public FSDelete(String path, boolean recursive, boolean skipTrash) {
+    public FSDelete(String path, boolean recursive) {
       this.path = new Path(path);
       this.path = new Path(path);
       this.recursive = recursive;
       this.recursive = recursive;
-      this.skipTrash = skipTrash;
     }
     }
 
 
     /**
     /**
@@ -752,19 +743,6 @@ public final class FSOperations {
      */
      */
     @Override
     @Override
     public JSONObject execute(FileSystem fs) throws IOException {
     public JSONObject execute(FileSystem fs) throws IOException {
-      if (!skipTrash) {
-        boolean movedToTrash = Trash.moveToAppropriateTrash(fs, path,
-            fs.getConf());
-        if (movedToTrash) {
-          HttpFSServerWebApp.getMetrics().incrOpsDelete();
-          return toJSON(
-              StringUtils.toLowerCase(HttpFSFileSystem.DELETE_JSON), true);
-        }
-        // Same is the behavior with Delete shell command.
-        // If moveToAppropriateTrash() returns false, file deletion
-        // is attempted rather than throwing Error.
-        LOG.debug("Could not move {} to Trash, attempting removal", path);
-      }
       boolean deleted = fs.delete(path, recursive);
       boolean deleted = fs.delete(path, recursive);
       HttpFSServerWebApp.get().getMetrics().incrOpsDelete();
       HttpFSServerWebApp.get().getMetrics().incrOpsDelete();
       return toJSON(
       return toJSON(

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

@@ -81,8 +81,7 @@ public class HttpFSParametersProvider extends ParametersProvider {
         new Class[]{ReplicationParam.class});
         new Class[]{ReplicationParam.class});
     PARAMS_DEF.put(Operation.SETTIMES,
     PARAMS_DEF.put(Operation.SETTIMES,
         new Class[]{ModifiedTimeParam.class, AccessTimeParam.class});
         new Class[]{ModifiedTimeParam.class, AccessTimeParam.class});
-    PARAMS_DEF.put(Operation.DELETE, new Class[]{RecursiveParam.class,
-        DeleteSkipTrashParam.class});
+    PARAMS_DEF.put(Operation.DELETE, new Class[]{RecursiveParam.class});
     PARAMS_DEF.put(Operation.SETACL, new Class[]{AclPermissionParam.class});
     PARAMS_DEF.put(Operation.SETACL, new Class[]{AclPermissionParam.class});
     PARAMS_DEF.put(Operation.REMOVEACL, new Class[]{});
     PARAMS_DEF.put(Operation.REMOVEACL, new Class[]{});
     PARAMS_DEF.put(Operation.MODIFYACLENTRIES,
     PARAMS_DEF.put(Operation.MODIFYACLENTRIES,
@@ -243,25 +242,6 @@ public class HttpFSParametersProvider extends ParametersProvider {
     }
     }
   }
   }
 
 
-  /**
-   * Class for delete's skipTrash parameter.
-   */
-  @InterfaceAudience.Private
-  public static class DeleteSkipTrashParam extends BooleanParam {
-
-    /**
-     * Parameter name.
-     */
-    public static final String NAME = HttpFSFileSystem.SKIP_TRASH_PARAM;
-
-    /**
-     * Constructor.
-     */
-    public DeleteSkipTrashParam() {
-      super(NAME, false);
-    }
-  }
-
   /**
   /**
    * Class for filter parameter.
    * Class for filter parameter.
    */
    */

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

@@ -30,7 +30,6 @@ import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AccessTimeParam
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.AclPermissionParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.BlockSizeParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DataParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DataParam;
-import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DeleteSkipTrashParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DestinationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.DestinationParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ECPolicyParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.ECPolicyParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.FilterParam;
 import org.apache.hadoop.fs.http.server.HttpFSParametersProvider.FilterParam;
@@ -549,13 +548,9 @@ public class HttpFSServer {
       case DELETE: {
       case DELETE: {
         Boolean recursive =
         Boolean recursive =
           params.get(RecursiveParam.NAME,  RecursiveParam.class);
           params.get(RecursiveParam.NAME,  RecursiveParam.class);
-        Boolean skipTrashParam = params.get(DeleteSkipTrashParam.NAME,
-            DeleteSkipTrashParam.class);
-        boolean skipTrash = skipTrashParam != null && skipTrashParam;
-        AUDIT_LOG.info("[{}] recursive [{}] skipTrash [{}]", path, recursive,
-            skipTrash);
+        AUDIT_LOG.info("[{}] recursive [{}]", path, recursive);
         FSOperations.FSDelete command =
         FSOperations.FSDelete command =
-          new FSOperations.FSDelete(path, recursive, skipTrash);
+          new FSOperations.FSDelete(path, recursive);
         JSONObject json = fsExecute(user, command);
         JSONObject json = fsExecute(user, command);
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         response = Response.ok(json).type(MediaType.APPLICATION_JSON).build();
         break;
         break;

+ 0 - 62
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -17,7 +17,6 @@
  */
  */
 package org.apache.hadoop.fs.http.server;
 package org.apache.hadoop.fs.http.server;
 
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
@@ -538,36 +537,6 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode());
     Assert.assertEquals(HttpURLConnection.HTTP_CREATED, conn.getResponseCode());
   }
   }
 
 
-  private void deleteWithHttp(String filename, String perms,
-      String unmaskedPerms, Boolean skipTrash) throws Exception {
-    String user = HadoopUsersConfTestHelper.getHadoopUsers()[0];
-    // Remove leading / from filename
-    if (filename.charAt(0) == '/') {
-      filename = filename.substring(1);
-    }
-    String pathOps;
-    if (perms == null) {
-      pathOps = MessageFormat.format("/webhdfs/v1/{0}?user.name={1}&op=DELETE",
-          filename, user);
-    } else {
-      pathOps = MessageFormat.format(
-          "/webhdfs/v1/{0}?user.name={1}&permission={2}&op=DELETE",
-          filename, user, perms);
-    }
-    if (unmaskedPerms != null) {
-      pathOps = pathOps + "&unmaskedpermission=" + unmaskedPerms;
-    }
-    if (skipTrash != null) {
-      pathOps = pathOps + "&skiptrash=" + skipTrash;
-    }
-    URL url = new URL(TestJettyHelper.getJettyURL(), pathOps);
-    HttpURLConnection conn = (HttpURLConnection) url.openConnection();
-    conn.addRequestProperty("Content-Type", "application/octet-stream");
-    conn.setRequestMethod("DELETE");
-    conn.connect();
-    Assert.assertEquals(HttpURLConnection.HTTP_OK, conn.getResponseCode());
-  }
-
   /**
   /**
    * Talks to the http interface to create a directory.
    * Talks to the http interface to create a directory.
    *
    *
@@ -813,37 +782,6 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertTrue("321".equals(getPerms(statusJson)));
     Assert.assertTrue("321".equals(getPerms(statusJson)));
   }
   }
 
 
-  /**
-   * Validate create and delete calls.
-   */
-  @Test
-  @TestDir
-  @TestJetty
-  @TestHdfs
-  public void testCreateDelete() throws Exception {
-    final String dir1 = "/testCreateDelete1";
-    final String path1 = dir1 + "/file1";
-    final String dir2 = "/testCreateDelete2";
-    final String path2 = dir2 + "/file2";
-
-    createHttpFSServer(false, false);
-    final Configuration conf = HttpFSServerWebApp.get()
-        .get(FileSystemAccess.class).getFileSystemConfiguration();
-    conf.setLong(FS_TRASH_INTERVAL_KEY, 5);
-    writeConf(conf, "hdfs-site.xml");
-
-    FileSystem fs = FileSystem.get(TestHdfsHelper.getHdfsConf());
-    fs.mkdirs(new Path(dir1));
-
-    createWithHttp(path1, null);
-    deleteWithHttp(path1, null, null, null);
-
-    fs.mkdirs(new Path(dir2));
-
-    createWithHttp(path2, null);
-    deleteWithHttp(path2, null, null, true);
-  }
-
   /**
   /**
    * Validate XAttr get/set/remove calls.
    * Validate XAttr get/set/remove calls.
    */
    */

+ 1 - 22
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/explorer.html

@@ -165,32 +165,11 @@
         <div class="modal-footer">
         <div class="modal-footer">
           <button type="button" class="btn" data-dismiss="modal">Cancel</button>
           <button type="button" class="btn" data-dismiss="modal">Cancel</button>
           <button type="button" class="btn btn-success" id="delete-button"
           <button type="button" class="btn btn-success" id="delete-button"
-                  data-complete-text="Deleting..." data-toggle="modal" data-target="#delete-trash-modal">Delete
-          </button>
+            data-complete-text="Deleting...">Delete</button>
         </div>
         </div>
       </div>
       </div>
     </div>
     </div>
   </div>
   </div>
-      <div class="modal" id="delete-trash-modal" tabindex="-1" role="dialog" aria-hidden="true">
-        <div class="modal-dialog">
-          <div class="modal-content">
-            <div class="modal-header">
-              <button type="button" class="close" data-dismiss="modal"
-                      aria-hidden="true">&times;</button>
-              <h4 class="modal-title" id="delete-trash-modal-title">Delete Trash</h4>
-            </div>
-            <div class="modal-body">
-              <div class="panel-body">
-                <div id="delete-trash-prompt"></div>
-              </div>
-            </div>
-            <div class="modal-footer">
-              <button type="button" class="btn" id="skip-trash-button">Yes (Skip Trash)</button>
-              <button type="button" class="btn btn-success" id="trash-button">No</button>
-            </div>
-          </div>
-        </div>
-      </div>
 
 
       <div class="row">
       <div class="row">
       <div class="col-xs-9 col-md-9">
       <div class="col-xs-9 col-md-9">

+ 13 - 37
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/webapps/router/explorer.js

@@ -82,47 +82,23 @@
   function delete_path(inode_name, absolute_file_path) {
   function delete_path(inode_name, absolute_file_path) {
     $('#delete-modal-title').text("Delete - " + inode_name);
     $('#delete-modal-title').text("Delete - " + inode_name);
     $('#delete-prompt').text("Are you sure you want to delete " + inode_name
     $('#delete-prompt').text("Are you sure you want to delete " + inode_name
-        + " ?");
-    $('#delete-trash-modal-title').text("Skip Trash - " + inode_name);
-    $('#delete-trash-prompt').text("Skipping Trash might delete file forever."
-        + " Do you want to skip-trash " + inode_name
-        + " ? (default behaviour - No)");
-
-    $('#skip-trash-button').click(function () {
-      // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>&skiptrash=true
-      var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
-          '?op=DELETE' + '&recursive=true&skiptrash=true';
-      $.ajax(url,
-          {
-            type: 'DELETE'
-          }).done(function (data) {
-        browse_directory(current_directory);
-      }).fail(network_error_handler(url)
-      ).always(function () {
-        $('#delete-modal').modal('hide');
-        $('#delete-button').button('reset');
-        $('#delete-trash-modal').modal('hide');
-        $('#skip-trash-button').button('reset');
-      });
-    })
-    $('#trash-button').click(function () {
+      + " ?");
+
+    $('#delete-button').click(function() {
       // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>
       // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>
       var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
       var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
-          '?op=DELETE' + '&recursive=true';
+        '?op=DELETE' + '&recursive=true';
+
       $.ajax(url,
       $.ajax(url,
-          {
-            type: 'DELETE'
-          }).done(function (data) {
-        browse_directory(current_directory);
-      }).fail(network_error_handler(url)
-      ).always(function () {
-        $('#delete-modal').modal('hide');
-        $('#delete-button').button('reset');
-        $('#delete-trash-modal').modal('hide');
-        $('#trash-button').button('reset');
-      });
+        { type: 'DELETE'
+        }).done(function(data) {
+          browse_directory(current_directory);
+        }).fail(network_error_handler(url)
+         ).always(function() {
+           $('#delete-modal').modal('hide');
+           $('#delete-button').button('reset');
+        });
     })
     })
-
     $('#delete-modal').modal();
     $('#delete-modal').modal();
   }
   }
 
 

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

@@ -55,10 +55,8 @@ import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.Response.Status;
 
 
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.Trash;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.slf4j.Logger;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
@@ -121,9 +119,6 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Charsets;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Lists;
 import com.sun.jersey.spi.container.ResourceFilters;
 import com.sun.jersey.spi.container.ResourceFilters;
 
 
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT;
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
-
 /** Web-hdfs NameNode implementation. */
 /** Web-hdfs NameNode implementation. */
 @Path("")
 @Path("")
 @ResourceFilters(ParamFilter.class)
 @ResourceFilters(ParamFilter.class)
@@ -1508,13 +1503,10 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
           final RecursiveParam recursive,
           final RecursiveParam recursive,
       @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
       @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
-          final SnapshotNameParam snapshotName,
-      @QueryParam(DeleteSkipTrashParam.NAME)
-      @DefaultValue(DeleteSkipTrashParam.DEFAULT)
-          final DeleteSkipTrashParam skiptrash
+          final SnapshotNameParam snapshotName
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
     return delete(ugi, delegation, username, doAsUser, ROOT, op, recursive,
     return delete(ugi, delegation, username, doAsUser, ROOT, op, recursive,
-        snapshotName, skiptrash);
+        snapshotName);
   }
   }
 
 
   /** Handle HTTP DELETE request. */
   /** Handle HTTP DELETE request. */
@@ -1535,53 +1527,34 @@ public class NamenodeWebHdfsMethods {
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
       @QueryParam(RecursiveParam.NAME) @DefaultValue(RecursiveParam.DEFAULT)
           final RecursiveParam recursive,
           final RecursiveParam recursive,
       @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
       @QueryParam(SnapshotNameParam.NAME) @DefaultValue(SnapshotNameParam.DEFAULT)
-          final SnapshotNameParam snapshotName,
-      @QueryParam(DeleteSkipTrashParam.NAME)
-      @DefaultValue(DeleteSkipTrashParam.DEFAULT)
-          final DeleteSkipTrashParam skiptrash
+          final SnapshotNameParam snapshotName
       ) throws IOException, InterruptedException {
       ) throws IOException, InterruptedException {
 
 
-    init(ugi, delegation, username, doAsUser, path, op, recursive,
-        snapshotName, skiptrash);
+    init(ugi, delegation, username, doAsUser, path, op, recursive, snapshotName);
 
 
-    return doAs(ugi, () -> delete(
-        path.getAbsolutePath(), op, recursive, snapshotName, skiptrash));
+    return doAs(ugi, new PrivilegedExceptionAction<Response>() {
+      @Override
+      public Response run() throws IOException {
+          return delete(ugi, delegation, username, doAsUser,
+              path.getAbsolutePath(), op, recursive, snapshotName);
+      }
+    });
   }
   }
 
 
   protected Response delete(
   protected Response delete(
+      final UserGroupInformation ugi,
+      final DelegationParam delegation,
+      final UserParam username,
+      final DoAsParam doAsUser,
       final String fullpath,
       final String fullpath,
       final DeleteOpParam op,
       final DeleteOpParam op,
       final RecursiveParam recursive,
       final RecursiveParam recursive,
-      final SnapshotNameParam snapshotName,
-      final DeleteSkipTrashParam skipTrash) throws IOException {
+      final SnapshotNameParam snapshotName
+      ) throws IOException {
     final ClientProtocol cp = getRpcClientProtocol();
     final ClientProtocol cp = getRpcClientProtocol();
 
 
     switch(op.getValue()) {
     switch(op.getValue()) {
     case DELETE: {
     case DELETE: {
-      Configuration conf =
-          (Configuration) context.getAttribute(JspHelper.CURRENT_CONF);
-      long trashInterval =
-          conf.getLong(FS_TRASH_INTERVAL_KEY, FS_TRASH_INTERVAL_DEFAULT);
-      if (trashInterval > 0 && !skipTrash.getValue()) {
-        LOG.info("{} is {} , trying to archive {} instead of removing",
-            FS_TRASH_INTERVAL_KEY, trashInterval, fullpath);
-        org.apache.hadoop.fs.Path path =
-            new org.apache.hadoop.fs.Path(fullpath);
-        Configuration clonedConf = new Configuration(conf);
-        // To avoid caching FS objects and prevent OOM issues
-        clonedConf.set("fs.hdfs.impl.disable.cache", "true");
-        FileSystem fs = FileSystem.get(clonedConf);
-        boolean movedToTrash = Trash.moveToAppropriateTrash(fs, path,
-            clonedConf);
-        if (movedToTrash) {
-          final String js = JsonUtil.toJsonString("boolean", true);
-          return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-        }
-        // Same is the behavior with Delete shell command.
-        // If moveToAppropriateTrash() returns false, file deletion
-        // is attempted rather than throwing Error.
-        LOG.debug("Could not move {} to Trash, attempting removal", fullpath);
-      }
       final boolean b = cp.delete(fullpath, recursive.getValue());
       final boolean b = cp.delete(fullpath, recursive.getValue());
       final String js = JsonUtil.toJsonString("boolean", b);
       final String js = JsonUtil.toJsonString("boolean", b);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();

+ 1 - 22
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.html

@@ -166,32 +166,11 @@
         <div class="modal-footer">
         <div class="modal-footer">
           <button type="button" class="btn" data-dismiss="modal">Cancel</button>
           <button type="button" class="btn" data-dismiss="modal">Cancel</button>
           <button type="button" class="btn btn-success" id="delete-button"
           <button type="button" class="btn btn-success" id="delete-button"
-                  data-complete-text="Deleting..." data-toggle="modal" data-target="#delete-trash-modal">Delete
-          </button>
+            data-complete-text="Deleting...">Delete</button>
         </div>
         </div>
       </div>
       </div>
     </div>
     </div>
   </div>
   </div>
-      <div class="modal" id="delete-trash-modal" tabindex="-1" role="dialog" aria-hidden="true">
-        <div class="modal-dialog">
-          <div class="modal-content">
-            <div class="modal-header">
-              <button type="button" class="close" data-dismiss="modal"
-                      aria-hidden="true">&times;</button>
-              <h4 class="modal-title" id="delete-trash-modal-title">Delete Trash</h4>
-            </div>
-            <div class="modal-body">
-              <div class="panel-body">
-                <div id="delete-trash-prompt"></div>
-              </div>
-            </div>
-            <div class="modal-footer">
-              <button type="button" class="btn" id="skip-trash-button">Yes (Skip Trash)</button>
-              <button type="button" class="btn btn-success" id="trash-button">No</button>
-            </div>
-          </div>
-        </div>
-      </div>
 
 
       <div class="row">
       <div class="row">
       <div class="col-xs-9 col-md-9">
       <div class="col-xs-9 col-md-9">

+ 11 - 35
hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/explorer.js

@@ -83,46 +83,22 @@
     $('#delete-modal-title').text("Delete - " + inode_name);
     $('#delete-modal-title').text("Delete - " + inode_name);
     $('#delete-prompt').text("Are you sure you want to delete " + inode_name
     $('#delete-prompt').text("Are you sure you want to delete " + inode_name
       + " ?");
       + " ?");
-    $('#delete-trash-modal-title').text("Skip Trash - " + inode_name);
-    $('#delete-trash-prompt').text("Skipping Trash might delete file forever."
-        + " Do you want to skip-trash " + inode_name
-        + " ? (default behaviour - No)");
 
 
-    $('#skip-trash-button').click(function () {
-      // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>&skiptrash=true
-      var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
-          '?op=DELETE' + '&recursive=true&skiptrash=true';
-      $.ajax(url,
-          {
-            type: 'DELETE'
-          }).done(function (data) {
-        browse_directory(current_directory);
-      }).fail(network_error_handler(url)
-      ).always(function () {
-        $('#delete-modal').modal('hide');
-        $('#delete-button').button('reset');
-        $('#delete-trash-modal').modal('hide');
-        $('#skip-trash-button').button('reset');
-      });
-    })
-    $('#trash-button').click(function () {
+    $('#delete-button').click(function() {
       // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>
       // DELETE /webhdfs/v1/<path>?op=DELETE&recursive=<true|false>
       var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
       var url = '/webhdfs/v1' + encode_path(absolute_file_path) +
-          '?op=DELETE' + '&recursive=true';
+        '?op=DELETE' + '&recursive=true';
+
       $.ajax(url,
       $.ajax(url,
-          {
-            type: 'DELETE'
-          }).done(function (data) {
-        browse_directory(current_directory);
-      }).fail(network_error_handler(url)
-      ).always(function () {
-        $('#delete-modal').modal('hide');
-        $('#delete-button').button('reset');
-        $('#delete-trash-modal').modal('hide');
-        $('#trash-button').button('reset');
-      });
+        { type: 'DELETE'
+        }).done(function(data) {
+          browse_directory(current_directory);
+        }).fail(network_error_handler(url)
+         ).always(function() {
+           $('#delete-modal').modal('hide');
+           $('#delete-button').button('reset');
+        });
     })
     })
-
     $('#delete-modal').modal();
     $('#delete-modal').modal();
   }
   }
 
 

+ 1 - 5
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/WebHDFS.md

@@ -462,11 +462,7 @@ See also: [`destination`](#Destination), [FileSystem](../../api/org/apache/hadoo
 * Submit a HTTP DELETE request.
 * Submit a HTTP DELETE request.
 
 
         curl -i -X DELETE "http://<host>:<port>/webhdfs/v1/<path>?op=DELETE
         curl -i -X DELETE "http://<host>:<port>/webhdfs/v1/<path>?op=DELETE
-                                      [&recursive=<true|false>][&skiptrash=<true|false>]"
-
-    Default values of queryparams if not provided:
-    1. recursive: false
-    2. skiptrash: false
+                                      [&recursive=<true |false>]"
 
 
     The client receives a response with a [`boolean` JSON object](#Boolean_JSON_Schema):
     The client receives a response with a [`boolean` JSON object](#Boolean_JSON_Schema):
 
 

+ 3 - 32
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestWebHDFS.java

@@ -18,8 +18,6 @@
 
 
 package org.apache.hadoop.hdfs.web;
 package org.apache.hadoop.hdfs.web;
 
 
-import static org.apache.hadoop.fs.CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER;
-import static org.apache.hadoop.fs.CommonConfigurationKeys.HADOOP_USER_GROUP_STATIC_OVERRIDES_DEFAULT;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
@@ -60,8 +58,6 @@ import java.util.Map;
 import java.util.NoSuchElementException;
 import java.util.NoSuchElementException;
 import java.util.Random;
 import java.util.Random;
 
 
-import org.apache.hadoop.hdfs.web.resources.DeleteSkipTrashParam;
-import org.apache.hadoop.hdfs.web.resources.RecursiveParam;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.fs.QuotaUsage;
 import org.apache.hadoop.fs.QuotaUsage;
@@ -1563,12 +1559,8 @@ public class TestWebHDFS {
       HttpURLConnection.HTTP_OK, conn.getResponseCode());
       HttpURLConnection.HTTP_OK, conn.getResponseCode());
 
 
     JSONObject responseJson = new JSONObject(response);
     JSONObject responseJson = new JSONObject(response);
-    if (!TYPE.equals("DELETE")) {
-      Assert.assertTrue("Response didn't give us a location. " + response,
-          responseJson.has("Location"));
-    } else {
-      Assert.assertTrue(responseJson.getBoolean("boolean"));
-    }
+    Assert.assertTrue("Response didn't give us a location. " + response,
+      responseJson.has("Location"));
 
 
     //Test that the DN allows CORS on Create
     //Test that the DN allows CORS on Create
     if(TYPE.equals("CREATE")) {
     if(TYPE.equals("CREATE")) {
@@ -1580,15 +1572,14 @@ public class TestWebHDFS {
     }
     }
   }
   }
 
 
+  @Test
   /**
   /**
    * Test that when "&noredirect=true" is added to operations CREATE, APPEND,
    * Test that when "&noredirect=true" is added to operations CREATE, APPEND,
    * OPEN, and GETFILECHECKSUM the response (which is usually a 307 temporary
    * OPEN, and GETFILECHECKSUM the response (which is usually a 307 temporary
    * redirect) is a 200 with JSON that contains the redirected location
    * redirect) is a 200 with JSON that contains the redirected location
    */
    */
-  @Test
   public void testWebHdfsNoRedirect() throws Exception {
   public void testWebHdfsNoRedirect() throws Exception {
     final Configuration conf = WebHdfsTestUtil.createConf();
     final Configuration conf = WebHdfsTestUtil.createConf();
-    conf.setLong(FS_TRASH_INTERVAL_KEY, 5);
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     cluster = new MiniDFSCluster.Builder(conf).numDataNodes(3).build();
     LOG.info("Started cluster");
     LOG.info("Started cluster");
     InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
     InetSocketAddress addr = cluster.getNameNode().getHttpAddress();
@@ -1627,26 +1618,6 @@ public class TestWebHDFS {
             + Param.toSortedString("&", new NoRedirectParam(true)));
             + Param.toSortedString("&", new NoRedirectParam(true)));
     LOG.info("Sending append request " + url);
     LOG.info("Sending append request " + url);
     checkResponseContainsLocation(url, "POST");
     checkResponseContainsLocation(url, "POST");
-
-    // setup some permission to allow moving file to .Trash location
-    cluster.getFileSystem().setPermission(new Path("/testWebHdfsNoRedirect"),
-        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
-    Path userDir = new Path(FileSystem.USER_HOME_PREFIX);
-    Path trashDir = new Path(FileSystem.USER_HOME_PREFIX, DEFAULT_HADOOP_HTTP_STATIC_USER);
-    Path trashPath = new Path(FileSystem.USER_HOME_PREFIX,
-        new Path(DEFAULT_HADOOP_HTTP_STATIC_USER, FileSystem.TRASH_PREFIX));
-    cluster.getFileSystem().mkdirs(userDir, FsPermission.getDirDefault());
-    cluster.getFileSystem().mkdir(trashDir, FsPermission.getDirDefault());
-    cluster.getFileSystem().mkdir(trashPath, FsPermission.getDirDefault());
-    cluster.getFileSystem().setOwner(trashPath, DEFAULT_HADOOP_HTTP_STATIC_USER, HADOOP_USER_GROUP_STATIC_OVERRIDES_DEFAULT);
-    cluster.getFileSystem().setPermission(new Path("/"), new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL));
-
-    url = new URL("http", addr.getHostString(), addr.getPort(),
-        WebHdfsFileSystem.PATH_PREFIX + "/testWebHdfsNoRedirect" + "?op=DELETE"
-            + Param.toSortedString("&", new RecursiveParam(true))
-            + Param.toSortedString("&", new DeleteSkipTrashParam(false)));
-    LOG.info("Sending append request " + url);
-    checkResponseContainsLocation(url, "DELETE");
   }
   }
 
 
   @Test
   @Test