diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java index 0d7f3b202a0b4..a0f2da2ce2292 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirMkdirOp.java @@ -224,7 +224,7 @@ private static INodesInPath unprotectedMkdir(FSDirectory fsd, long inodeId, timestamp); INodesInPath iip = fsd.addLastINode(parent, dir, permission.getPermission(), - true, Optional.empty()); + true, Optional.empty(), true); if (iip != null && aclEntries != null) { AclStorage.updateINodeAcl(dir, aclEntries, Snapshot.CURRENT_STATE_ID); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java index 0f6ceae82489b..ee6a069cf7c58 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java @@ -17,7 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode; -import org.apache.commons.lang3.tuple.Pair; +import org.apache.commons.lang3.tuple.Triple; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -72,14 +72,24 @@ static RenameResult renameToInt( * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves * dstInodes[dstInodes.length-1] */ - private static Pair, Optional> verifyQuotaForRename( - FSDirectory fsd, INodesInPath src, INodesInPath dst) throws QuotaExceededException { + private static Triple, Optional> verifyQuotaForRename( + FSDirectory fsd, INodesInPath src, INodesInPath dst, boolean overwrite) + throws QuotaExceededException { Optional srcDelta = Optional.empty(); Optional dstDelta = Optional.empty(); if (!fsd.getFSNamesystem().isImageLoaded() || fsd.shouldSkipQuotaChecks()) { // Do not check quota if edits log is still being processed - return Pair.of(srcDelta, dstDelta); + return Triple.of(false, srcDelta, dstDelta); } + + // Verify path without valid 'DirectoryWithQuotaFeature' + // Note: In overwrite scenarios, quota calculation is still required, + // overwrite operations delete existing content, which affects the root directory's quota. + if (!overwrite && FSDirectory.verifyPathWithoutValidQuotaFeature(src) + && FSDirectory.verifyPathWithoutValidQuotaFeature(dst)) { + return Triple.of(false, srcDelta, dstDelta); + } + int i = 0; while (src.getINode(i) == dst.getINode(i)) { i++; @@ -108,7 +118,7 @@ private static Pair, Optional> verifyQuotaFor delta.subtract(counts); } FSDirectory.verifyQuota(dst, dst.length() - 1, delta, src.getINode(i - 1)); - return Pair.of(srcDelta, dstDelta); + return Triple.of(true, srcDelta, dstDelta); } /** @@ -216,10 +226,10 @@ static INodesInPath unprotectedRenameTo(FSDirectory fsd, fsd.ezManager.checkMoveValidity(srcIIP, dstIIP); // Ensure dst has quota to accommodate rename verifyFsLimitsForRename(fsd, srcIIP, dstIIP); - Pair, Optional> countPair = - verifyQuotaForRename(fsd, srcIIP, dstIIP); + Triple, Optional> countTriple = + verifyQuotaForRename(fsd, srcIIP, dstIIP, false); - RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, countPair); + RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, countTriple); boolean added = false; @@ -436,10 +446,10 @@ static RenameResult unprotectedRenameTo(FSDirectory fsd, // Ensure dst has quota to accommodate rename verifyFsLimitsForRename(fsd, srcIIP, dstIIP); - Pair, Optional> quotaPair = - verifyQuotaForRename(fsd, srcIIP, dstIIP); + Triple, Optional> countTriple = + verifyQuotaForRename(fsd, srcIIP, dstIIP, overwrite); - RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, quotaPair); + RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, countTriple); boolean undoRemoveSrc = true; tx.removeSrc(); @@ -656,13 +666,14 @@ private static class RenameOperation { private final boolean srcChildIsReference; private final QuotaCounts oldSrcCountsInSnapshot; private final boolean sameStoragePolicy; + private final boolean updateQuota; private final Optional srcSubTreeCount; private final Optional dstSubTreeCount; private INode srcChild; private INode oldDstChild; RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP, - Pair, Optional> quotaPair) { + Triple, Optional> countTriple) { this.fsd = fsd; this.srcIIP = srcIIP; this.dstIIP = dstIIP; @@ -712,9 +723,10 @@ private static class RenameOperation { withCount = null; } // Set quota for src and dst, ignore src is in Snapshot or is Reference + this.updateQuota = countTriple.getLeft() || withCount != null; this.srcSubTreeCount = withCount == null ? - quotaPair.getLeft() : Optional.empty(); - this.dstSubTreeCount = quotaPair.getRight(); + countTriple.getMiddle() : Optional.empty(); + this.dstSubTreeCount = countTriple.getRight(); } boolean isSameStoragePolicy() { @@ -755,9 +767,11 @@ long removeSrc() throws IOException { throw new IOException(error); } else { // update the quota count if necessary - Optional countOp = sameStoragePolicy ? - srcSubTreeCount : Optional.empty(); - fsd.updateCountForDelete(srcChild, srcIIP, countOp); + if (updateQuota) { + Optional countOp = sameStoragePolicy ? + srcSubTreeCount : Optional.empty(); + fsd.updateCountForDelete(srcChild, srcIIP, countOp); + } srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null); return removedNum; } @@ -772,9 +786,11 @@ boolean removeSrc4OldRename() { return false; } else { // update the quota count if necessary - Optional countOp = sameStoragePolicy ? - srcSubTreeCount : Optional.empty(); - fsd.updateCountForDelete(srcChild, srcIIP, countOp); + if (updateQuota) { + Optional countOp = sameStoragePolicy ? + srcSubTreeCount : Optional.empty(); + fsd.updateCountForDelete(srcChild, srcIIP, countOp); + } srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null); return true; } @@ -785,7 +801,9 @@ long removeDst() { if (removedNum != -1) { oldDstChild = dstIIP.getLastINode(); // update the quota count if necessary - fsd.updateCountForDelete(oldDstChild, dstIIP, dstSubTreeCount); + if (updateQuota) { + fsd.updateCountForDelete(oldDstChild, dstIIP, dstSubTreeCount); + } dstIIP = INodesInPath.replace(dstIIP, dstIIP.length() - 1, null); } return removedNum; @@ -803,7 +821,7 @@ INodesInPath addSourceToDestination() { toDst = new INodeReference.DstReference(dstParent.asDirectory(), withCount, dstIIP.getLatestSnapshotId()); } - return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst, srcSubTreeCount); + return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst, srcSubTreeCount, updateQuota); } void updateMtimeAndLease(long timestamp) { @@ -837,7 +855,7 @@ void restoreSource() { // the srcChild back Optional countOp = sameStoragePolicy ? srcSubTreeCount : Optional.empty(); - fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild, countOp); + fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild, countOp, updateQuota); } } @@ -847,7 +865,7 @@ void restoreDst(BlockStoragePolicySuite bsps) { if (dstParent.isWithSnapshot()) { dstParent.undoRename4DstParent(bsps, oldDstChild, dstIIP.getLatestSnapshotId()); } else { - fsd.addLastINodeNoQuotaCheck(dstParentIIP, oldDstChild, dstSubTreeCount); + fsd.addLastINodeNoQuotaCheck(dstParentIIP, oldDstChild, dstSubTreeCount, updateQuota); } if (oldDstChild != null && oldDstChild.isReference()) { final INodeReference removedDstRef = oldDstChild.asReference(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index dd6e021a80d31..e0309bdc7cde5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -1196,7 +1196,7 @@ INodesInPath addINode(INodesInPath existing, INode child, cacheName(child); writeLock(); try { - return addLastINode(existing, child, modes, true, Optional.empty()); + return addLastINode(existing, child, modes, true, Optional.empty(), true); } finally { writeUnlock(); } @@ -1242,6 +1242,23 @@ static void verifyQuota(INodesInPath iip, int pos, QuotaCounts deltas, } } + /** + * Verify that the path from the specified position to the root + * (excluding the root itself) does not contain any valid quota feature. + * + * @param iip the INodesInPath instance containing all the INodes for the path. + * @return true if no valid quota feature, false if any directory in the path has quota feature. + */ + static boolean verifyPathWithoutValidQuotaFeature(INodesInPath iip) { + for (int i = iip.length() - 2; i >= 1; i--) { + INodeDirectory d = iip.getINode(i).asDirectory(); + if (d.isWithQuota()) { + return false; + } + } + return true; + } + /** Verify if the inode name is legal. */ void verifyINodeName(byte[] childName) throws HadoopIllegalArgumentException { if (Arrays.equals(HdfsServerConstants.DOT_SNAPSHOT_DIR_BYTES, childName)) { @@ -1343,11 +1360,12 @@ private void copyINodeDefaultAcl(INode child, FsPermission modes) { * @param inode the new INode to add * @param modes create modes * @param checkQuota whether to check quota + * @param updateQuota whether to update quota * @return an INodesInPath instance containing the new INode */ @VisibleForTesting - public INodesInPath addLastINode(INodesInPath existing, INode inode, - FsPermission modes, boolean checkQuota, Optional quotaCount) + public INodesInPath addLastINode(INodesInPath existing, INode inode, FsPermission modes, + boolean checkQuota, Optional quotaCount, boolean updateQuota) throws QuotaExceededException { assert existing.getLastINode() != null && existing.getLastINode().isDirectory(); @@ -1379,17 +1397,23 @@ public INodesInPath addLastINode(INodesInPath existing, INode inode, // always verify inode name verifyINodeName(inode.getLocalNameBytes()); - final QuotaCounts counts = quotaCount.orElseGet(() -> inode. - computeQuotaUsage(getBlockStoragePolicySuite(), - parent.getStoragePolicyID(), false, - Snapshot.CURRENT_STATE_ID)); - updateCount(existing, pos, counts, checkQuota); + QuotaCounts counts = null; + if (updateQuota) { + counts = quotaCount.orElseGet(() -> inode. + computeQuotaUsage(getBlockStoragePolicySuite(), + parent.getStoragePolicyID(), false, + Snapshot.CURRENT_STATE_ID)); + updateCount(existing, pos, counts, checkQuota); + } boolean isRename = (inode.getParent() != null); final boolean added = parent.addChild(inode, true, existing.getLatestSnapshotId()); if (!added) { - updateCountNoQuotaCheck(existing, pos, counts.negation()); + // When adding INode fails, if 'updateQuota' is true, rollback quota. + if (updateQuota) { + updateCountNoQuotaCheck(existing, pos, counts.negation()); + } return null; } else { if (!isRename) { @@ -1401,10 +1425,10 @@ public INodesInPath addLastINode(INodesInPath existing, INode inode, } INodesInPath addLastINodeNoQuotaCheck(INodesInPath existing, INode i, - Optional quotaCount) { + Optional quotaCount, boolean updateQuota) { try { // All callers do not have create modes to pass. - return addLastINode(existing, i, null, false, quotaCount); + return addLastINode(existing, i, null, false, quotaCount, updateQuota); } catch (QuotaExceededException e) { NameNode.LOG.warn("FSDirectory.addChildNoQuotaCheck - unexpected", e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorrectnessOfQuotaAfterRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorrectnessOfQuotaAfterRenameOp.java index 64df2e6f1147f..a9aa2bcfa6576 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorrectnessOfQuotaAfterRenameOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCorrectnessOfQuotaAfterRenameOp.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.QuotaUsage; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HdfsConfiguration; @@ -28,17 +29,23 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; +import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.mockito.Mockito; import java.io.IOException; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.HOT_STORAGE_POLICY_NAME; import static org.apache.hadoop.hdfs.protocol.HdfsConstants.ONESSD_STORAGE_POLICY_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyInt; public class TestCorrectnessOfQuotaAfterRenameOp { private static MiniDFSCluster cluster; @@ -79,24 +86,32 @@ public void testQuotaUsageWhenRenameWithSameStoragePolicy() throws Exception { DFSTestUtil.createFile(dfs, file2, fileLen, replication, 0); final Path dstDir1 = new Path(testParentDir2, "dst-dir"); + // If dstDir1 not exist, after the rename operation, + // the root dir's quota usage should remain unchanged. + QuotaUsage quotaUsage1 = dfs.getQuotaUsage(new Path("/")); ContentSummary cs1 = dfs.getContentSummary(testParentDir1); // srcDir=/root/test1/src/dir // dstDir1=/root/test2/dst-dir dstDir1 not exist boolean rename = dfs.rename(srcDir, dstDir1); assertEquals(true, rename); + QuotaUsage quotaUsage2 = dfs.getQuotaUsage(new Path("/")); ContentSummary cs2 = dfs.getContentSummary(testParentDir2); + assertEquals(quotaUsage1, quotaUsage2); assertTrue(cs1.equals(cs2)); final Path dstDir2 = new Path(testParentDir3, "dst-dir"); assertTrue(dfs.mkdirs(dstDir2)); + QuotaUsage quotaUsage3 = dfs.getQuotaUsage(testParentDir2); ContentSummary cs3 = dfs.getContentSummary(testParentDir2); //Src and dst must be same (all file or all dir) // dstDir1=/root/test2/dst-dir // dstDir2=/root/test3/dst-dir dfs.rename(dstDir1, dstDir2, Options.Rename.OVERWRITE); + QuotaUsage quotaUsage4 = dfs.getQuotaUsage(testParentDir3); ContentSummary cs4 = dfs.getContentSummary(testParentDir3); + assertEquals(quotaUsage3, quotaUsage4); assertTrue(cs3.equals(cs4)); } @@ -158,4 +173,186 @@ public void testQuotaUsageWhenRenameWithDifferStoragePolicy() throws Exception { QuotaCounts subtract = dstCountsAfterRename.subtract(dstCountsBeforeRename); assertTrue(subtract.equals(srcCounts)); } + + @Test + public void testRenameWithoutValidFeature() throws Exception { + final int fileLen = 1024; + final short replication = 3; + final Path root = new Path("/testRename"); + assertTrue(dfs.mkdirs(root)); + + Path testParentDir1 = new Path(root, "testDir1"); + assertTrue(dfs.mkdirs(testParentDir1)); + Path testParentDir2 = new Path(root, "testDir2"); + assertTrue(dfs.mkdirs(testParentDir2)); + Path testParentDir3 = new Path(root, "testDir3"); + assertTrue(dfs.mkdirs(testParentDir3)); + + final Path srcDir = new Path(testParentDir1, "src-dir"); + for (int i = 0; i < 2; i++) { + Path file1 = new Path(srcDir, "file" + i); + DFSTestUtil.createFile(dfs, file1, fileLen, replication, 0); + } + + // 1. Test rename1 + ContentSummary rootContentSummary1 = dfs.getContentSummary(new Path("/")); + QuotaUsage rootQuotaUsage1 = dfs.getQuotaUsage(new Path("/")); + ContentSummary contentSummary1 = dfs.getContentSummary(testParentDir1); + // srcDir=/testRename/testDir1/src-dir + // dstDir=/testRename/testDir2/dst-dir dstDir not exist + final Path dstDir2 = new Path(testParentDir2, "dst-dir"); + assertTrue(dfs.rename(srcDir, dstDir2)); + ContentSummary contentSummary2 = dfs.getContentSummary(testParentDir2); + assertEquals(contentSummary1, contentSummary2); + QuotaUsage rootQuotaUsage2 = dfs.getQuotaUsage(new Path("/")); + assertEquals(rootQuotaUsage1.getFileAndDirectoryCount(), + rootQuotaUsage2.getFileAndDirectoryCount()); + // The return values of the getContentSummary() and getQuotaUsage() should be consistent + assertEquals(rootContentSummary1.getFileAndDirectoryCount(), + rootQuotaUsage2.getFileAndDirectoryCount()); + + // 2. Test rename2 + final Path dstDir3 = new Path(testParentDir3, "dst-dir"); + assertTrue(dfs.mkdirs(dstDir3)); + long originDstDirUsage = dfs.getQuotaUsage(dstDir3).getFileAndDirectoryCount(); + // Overwrite the rename destination, the usage of dstDir3 should be excluded + long expectedCount = + dfs.getQuotaUsage(new Path("/")).getFileAndDirectoryCount() - originDstDirUsage; + ContentSummary contentSummary3 = dfs.getContentSummary(testParentDir2); + // Src and dst must be same + // dstDir2=/testRename/testDir2/dst-dir + // dstDir3=/testRename/testDir3/dst-dir + dfs.rename(dstDir2, dstDir3, Options.Rename.OVERWRITE); + long actualCount = dfs.getQuotaUsage(new Path("/")).getFileAndDirectoryCount(); + assertEquals(expectedCount, actualCount); + ContentSummary contentSummary4 = dfs.getContentSummary(testParentDir3); + assertEquals(contentSummary3, contentSummary4); + } + + @Test + public void testRenameUndoWithoutValidFeature() throws Exception { + final int fileLen = 1024; + final short replication = 3; + final Path root = new Path("/testRenameUndo"); + assertTrue(dfs.mkdirs(root)); + + Path testParentDir1 = new Path(root, "testDir1"); + assertTrue(dfs.mkdirs(testParentDir1)); + Path testParentDir2 = new Path(root, "testDir2"); + assertTrue(dfs.mkdirs(testParentDir2)); + Path testParentDir3 = new Path(root, "testDir3"); + assertTrue(dfs.mkdirs(testParentDir3)); + Path testParentDir4 = new Path(root, "testDir4"); + assertTrue(dfs.mkdirs(testParentDir4)); + + final Path srcDir1 = new Path(testParentDir1, "src-dir"); + for (int i = 0; i < 2; i++) { + Path file1 = new Path(srcDir1, "file" + i); + DFSTestUtil.createFile(dfs, file1, fileLen, replication, 0); + } + + final Path srcDir3 = new Path(testParentDir3, "src-dir"); + for (int i = 0; i < 2; i++) { + Path file1 = new Path(srcDir3, "file" + i); + DFSTestUtil.createFile(dfs, file1, fileLen, replication, 0); + } + + // Test rename1 + ContentSummary rootContentSummary1 = dfs.getContentSummary(new Path("/")); + QuotaUsage rootQuotaUsage1 = dfs.getQuotaUsage(new Path("/")); + ContentSummary contentSummary1 = dfs.getContentSummary(testParentDir1); + + FSNamesystem fsn = cluster.getNamesystem(); + FSDirectory fsDirectory = fsn.getFSDirectory(); + + // Replace INode, expected addChild return false + INodeDirectory dir = fsDirectory.getINode4Write(testParentDir2.toString()).asDirectory(); + INodeDirectory mockDir = Mockito.spy(dir); + INode srcInode = fsDirectory.getINode(srcDir3.toString()); + // Fail the rename but succeed in undo + Mockito.doReturn(false).when(mockDir).addChild(Mockito.eq(srcInode), anyBoolean(), anyInt()); + INodeDirectory rootDir = fsDirectory.getINode4Write(root.toString()).asDirectory(); + rootDir.replaceChild(dir, mockDir, fsDirectory.getINodeMap()); + mockDir.setParent(rootDir); + + // srcDir=/testRenameUndo/testDir1/src-dir + // dstDir=/testRenameUndo/testDir2/ + assertFalse(dfs.rename(srcDir3, testParentDir2)); + + ContentSummary rootContentSummary2 = dfs.getContentSummary(new Path("/")); + QuotaUsage rootQuotaUsage2 = dfs.getQuotaUsage(new Path("/")); + ContentSummary contentSummary2 = dfs.getContentSummary(testParentDir1); + assertEquals(rootContentSummary1.toString(), rootContentSummary2.toString()); + assertEquals(rootQuotaUsage1.toString(), rootQuotaUsage2.toString()); + assertEquals(contentSummary1.toString(), contentSummary2.toString()); + assertEquals(rootContentSummary1.getFileAndDirectoryCount(), + rootQuotaUsage2.getFileAndDirectoryCount()); + + // Test rename2 + final Path dstDir4 = new Path(testParentDir4, "src-dir"); + assertTrue(dfs.mkdirs(dstDir4)); + ContentSummary rootContentSummary3 = dfs.getContentSummary(new Path("/")); + QuotaUsage rootQuotaUsage3 = dfs.getQuotaUsage(new Path("/")); + ContentSummary contentSummary3 = dfs.getContentSummary(testParentDir3); + + // Replace INode, expected addChild return false + INodeDirectory dir4 = fsDirectory.getINode4Write(testParentDir4.toString()).asDirectory(); + INodeDirectory mockDir4 = Mockito.spy(dir4); + INode srcInode3 = fsDirectory.getINode(srcDir3.toString()); + Mockito.doReturn(false).when(mockDir4).addChild(Mockito.eq(srcInode3), anyBoolean(), anyInt()); + rootDir.replaceChild(dir4, mockDir4, fsDirectory.getINodeMap()); + mockDir4.setParent(rootDir); + + // srcDir=/testRenameUndo/testDir3/src-dir + // dstDir=/testRenameUndo/testDir4/src-dir dstDir exist + assertThrows(RemoteException.class, + () -> dfs.rename(srcDir3, dstDir4, Options.Rename.OVERWRITE)); + + ContentSummary rootContentSummary4 = dfs.getContentSummary(new Path("/")); + QuotaUsage rootQuotaUsage4 = dfs.getQuotaUsage(new Path("/")); + ContentSummary contentSummary4 = dfs.getContentSummary(testParentDir3); + assertEquals(rootContentSummary3.toString(), rootContentSummary4.toString()); + assertEquals(rootQuotaUsage3.toString(), rootQuotaUsage4.toString()); + assertEquals(contentSummary3.toString(), contentSummary4.toString()); + assertEquals(rootContentSummary3.getFileAndDirectoryCount(), + rootQuotaUsage4.getFileAndDirectoryCount()); + } + + @Test + public void testRenameFileInSnapshotDirWithoutValidFeature() throws Exception { + final int fileLen = 1024; + final short replication = 3; + final Path root = new Path("/testRenameFileInSnapshotDir"); + assertTrue(dfs.mkdirs(root)); + + Path testParentDir1 = new Path(root, "testDir1"); + assertTrue(dfs.mkdirs(testParentDir1)); + Path file = new Path(testParentDir1, "file1"); + DFSTestUtil.createFile(dfs, file, fileLen, replication, 0); + dfs.allowSnapshot(testParentDir1); + dfs.createSnapshot(testParentDir1, "snapshot1"); + + Path testParentDir2 = new Path(root, "testDir2"); + assertTrue(dfs.mkdirs(testParentDir2)); + + ContentSummary contentSummary1 = dfs.getContentSummary(new Path("/")); + QuotaUsage quotaUsage1 = dfs.getQuotaUsage(new Path("/")); + assertEquals(contentSummary1.getSpaceConsumed(), quotaUsage1.getSpaceConsumed()); + assertEquals(contentSummary1.getFileAndDirectoryCount(), + quotaUsage1.getFileAndDirectoryCount()); + + // The snapshot of file1 not be cleaned up + assertTrue(dfs.rename(new Path(testParentDir1, "file1"), testParentDir2)); + + ContentSummary contentSummary2 = dfs.getContentSummary(new Path("/")); + QuotaUsage quotaUsage2 = dfs.getQuotaUsage(new Path("/")); + assertEquals(quotaUsage1.getFileAndDirectoryCount() + 1, + quotaUsage2.getFileAndDirectoryCount()); + assertEquals(quotaUsage1.getSpaceConsumed() + fileLen * replication, + quotaUsage2.getSpaceConsumed()); + // Root directory's actual capacity must match quota usage + assertEquals(contentSummary2.getFileAndDirectoryCount(), + quotaUsage2.getFileAndDirectoryCount()); + assertEquals(contentSummary2.getSpaceConsumed(), quotaUsage2.getSpaceConsumed()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java index 907bd2ff03bbe..73f9e0e5cd9d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestRenameWithSnapshots.java @@ -1650,7 +1650,7 @@ public void testRenameUndo_5() throws Exception { final Path foo2 = new Path(subdir2, foo.getName()); FSDirectory fsdir2 = Mockito.spy(fsdir); Mockito.doThrow(new NSQuotaExceededException("fake exception")).when(fsdir2) - .addLastINode(any(), any(), any(), anyBoolean(), any()); + .addLastINode(any(), any(), any(), anyBoolean(), any(), anyBoolean()); Whitebox.setInternalState(fsn, "dir", fsdir2); // rename /test/dir1/foo to /test/dir2/subdir2/foo. // FSDirectory#verifyQuota4Rename will pass since the remaining quota is 2.