瀏覽代碼

HDFS-17760. Fix MoveToTrash throws ParentNotDirectoryException when there is a file inode with the same name in the trash (#7514). Contributed by liuguanghua.

Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
LiuGuH 3 周之前
父節點
當前提交
37151167dd

+ 1 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicyDefault.java

@@ -162,7 +162,7 @@ public class TrashPolicyDefault extends TrashPolicy {
           LOG.warn("Can't create(mkdir) trash directory: " + baseTrashPath);
           LOG.warn("Can't create(mkdir) trash directory: " + baseTrashPath);
           return false;
           return false;
         }
         }
-      } catch (FileAlreadyExistsException e) {
+      } catch (FileAlreadyExistsException | ParentNotDirectoryException e) {
         // find the path which is not a directory, and modify baseTrashPath
         // find the path which is not a directory, and modify baseTrashPath
         // & trashPath, then mkdirs
         // & trashPath, then mkdirs
         Path existsFilePath = baseTrashPath;
         Path existsFilePath = baseTrashPath;

+ 23 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSTrash.java

@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.TestTrash;
 import org.apache.hadoop.fs.TestTrash;
@@ -190,4 +191,26 @@ public class TestHDFSTrash {
         .thenReturn(userTrashRoot);
         .thenReturn(userTrashRoot);
     return new Trash(spyUserFs, config);
     return new Trash(spyUserFs, config);
   }
   }
+
+
+  @Test
+  public void testDeleteToTrashWhenInodeNameDuplicate() throws Exception {
+    Configuration testConf = new Configuration(conf);
+    testConf.set(CommonConfigurationKeys.FS_TRASH_INTERVAL_KEY, "600");
+
+    Path file = new Path(TEST_ROOT, "subdir0");
+    Path dir = new Path(TEST_ROOT, "subdir0/subdir1/subdir2");
+
+    fs = DFSTestUtil.login(fs, testConf, user1);
+
+    FSDataOutputStream out = fs.create(file);
+    out.writeBytes("This is a file");
+    out.close();
+
+    Trash trash = new Trash(testConf);
+    assertTrue(trash.moveToTrash(file));
+
+    fs.mkdirs(dir);
+    assertTrue(trash.moveToTrash(dir));
+  }
 }
 }