Skip to content
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<QuotaCounts>, Optional<QuotaCounts>> verifyQuotaForRename(
FSDirectory fsd, INodesInPath src, INodesInPath dst) throws QuotaExceededException {
private static Triple<Boolean, Optional<QuotaCounts>, Optional<QuotaCounts>> verifyQuotaForRename(
FSDirectory fsd, INodesInPath src, INodesInPath dst, boolean overwrite)
throws QuotaExceededException {
Comment on lines +75 to +77
Copy link
Preview

Copilot AI Sep 23, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The method signature change from Pair to Triple with a Boolean parameter lacks clear documentation. The Boolean return value's meaning is not obvious from the method name or parameters. Consider renaming the method or adding clear JavaDoc to explain what the Boolean represents.

Copilot uses AI. Check for mistakes.

Copy link
Contributor Author

@huangzhaobo99 huangzhaobo99 Sep 23, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  /**
   * Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
   * dstInodes[dstInodes.length-1]
   *
   * @param fsd       FSDirectory instance
   * @param src       Source path inodes
   * @param dst       Destination path inodes
   * @param overwrite Whether destination will be overwritten
   * @return Triple containing:
   * - Boolean: true if quota was updated, false if skipped
   * - Optional<QuotaCounts>: Source path quotaCounts delta
   * - Optional<QuotaCounts>: Destination path quotaCounts delta
   * @throws QuotaExceededException if quota limit is exceeded
   */

Optional<QuotaCounts> srcDelta = Optional.empty();
Optional<QuotaCounts> 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.verifyPathWithoutValidFeature(src)
&& FSDirectory.verifyPathWithoutValidFeature(dst)) {
return Triple.of(false, srcDelta, dstDelta);
}

int i = 0;
while (src.getINode(i) == dst.getINode(i)) {
i++;
Expand Down Expand Up @@ -108,7 +118,7 @@ private static Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> 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);
}

/**
Expand Down Expand Up @@ -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<QuotaCounts>, Optional<QuotaCounts>> countPair =
verifyQuotaForRename(fsd, srcIIP, dstIIP);
Triple<Boolean, Optional<QuotaCounts>, Optional<QuotaCounts>> 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;

Expand Down Expand Up @@ -436,10 +446,10 @@ static RenameResult unprotectedRenameTo(FSDirectory fsd,

// Ensure dst has quota to accommodate rename
verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> quotaPair =
verifyQuotaForRename(fsd, srcIIP, dstIIP);
Triple<Boolean, Optional<QuotaCounts>, Optional<QuotaCounts>> 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();
Expand Down Expand Up @@ -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<QuotaCounts> srcSubTreeCount;
private final Optional<QuotaCounts> dstSubTreeCount;
private INode srcChild;
private INode oldDstChild;

RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP,
Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> quotaPair) {
Triple<Boolean, Optional<QuotaCounts>, Optional<QuotaCounts>> quotaPair) {
this.fsd = fsd;
this.srcIIP = srcIIP;
this.dstIIP = dstIIP;
Expand Down Expand Up @@ -712,8 +723,9 @@ private static class RenameOperation {
withCount = null;
}
// Set quota for src and dst, ignore src is in Snapshot or is Reference
this.updateQuota = quotaPair.getLeft() || isSrcInSnapshot || srcChildIsReference;
this.srcSubTreeCount = withCount == null ?
quotaPair.getLeft() : Optional.empty();
quotaPair.getMiddle() : Optional.empty();
this.dstSubTreeCount = quotaPair.getRight();
}

Copy link
Preview

Copilot AI Sep 23, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The logic for determining updateQuota combines three different conditions without clear explanation. This complex boolean expression makes the code harder to understand and maintain. Consider extracting this logic into a separate method with descriptive naming.

Copilot uses AI. Check for mistakes.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ignore src is in Snapshot or is Reference

Expand Down Expand Up @@ -755,9 +767,11 @@ long removeSrc() throws IOException {
throw new IOException(error);
} else {
// update the quota count if necessary
Optional<QuotaCounts> countOp = sameStoragePolicy ?
srcSubTreeCount : Optional.empty();
fsd.updateCountForDelete(srcChild, srcIIP, countOp);
if (updateQuota) {
Optional<QuotaCounts> countOp = sameStoragePolicy ?
srcSubTreeCount : Optional.empty();
fsd.updateCountForDelete(srcChild, srcIIP, countOp);
}
srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
return removedNum;
}
Expand All @@ -772,9 +786,11 @@ boolean removeSrc4OldRename() {
return false;
} else {
// update the quota count if necessary
Optional<QuotaCounts> countOp = sameStoragePolicy ?
srcSubTreeCount : Optional.empty();
fsd.updateCountForDelete(srcChild, srcIIP, countOp);
if (updateQuota) {
Optional<QuotaCounts> countOp = sameStoragePolicy ?
srcSubTreeCount : Optional.empty();
fsd.updateCountForDelete(srcChild, srcIIP, countOp);
}
srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
return true;
}
Expand All @@ -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;
Expand All @@ -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) {
Expand Down Expand Up @@ -837,7 +855,7 @@ void restoreSource() {
// the srcChild back
Optional<QuotaCounts> countOp = sameStoragePolicy ?
srcSubTreeCount : Optional.empty();
fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild, countOp);
fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild, countOp, updateQuota);
}
}

Expand All @@ -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();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
}
Expand Down Expand Up @@ -1242,6 +1242,24 @@ 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 features.
*
* @param iip the INodesInPath containing all the ancestral INodes.
* @return true if no valid features are found along the path,
* false if any directory in the path has an active feature.
*/
static boolean verifyPathWithoutValidFeature(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)) {
Expand Down Expand Up @@ -1343,11 +1361,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<QuotaCounts> quotaCount)
public INodesInPath addLastINode(INodesInPath existing, INode inode, FsPermission modes,
boolean checkQuota, Optional<QuotaCounts> quotaCount, boolean updateQuota)
throws QuotaExceededException {
assert existing.getLastINode() != null &&
existing.getLastINode().isDirectory();
Expand Down Expand Up @@ -1379,16 +1398,22 @@ 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);
if (updateQuota) {
QuotaCounts 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) {
if (!added && updateQuota) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sorry for the late reply. Do you think we should keep the original logic of always returning null when added == false? Regardless of whether the quota is updated, the decision to roll back the quota update should probably depend only on updateQuota. I might be missing some context here—happy to hear your thoughts.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zhtttylz Sorry, the original rollback quotaUsage logic has been changed, fixed.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review. I will add an rename undo integration test here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@zhtttylz Hi sir, I have added the RenameUndo UT. The original logic had some issues, but it now functions as intended after being rectified.

QuotaCounts counts = quotaCount.orElseGet(() -> inode.
computeQuotaUsage(getBlockStoragePolicySuite(),
parent.getStoragePolicyID(), false,
Snapshot.CURRENT_STATE_ID));
Copy link
Preview

Copilot AI Sep 23, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The quota computation is duplicated - it's computed before line 1402 and again here. This results in unnecessary computation when updateQuota is true and the node fails to be added. Consider computing the counts once and reusing the result.

Copilot uses AI. Check for mistakes.

Copy link
Contributor Author

@huangzhaobo99 huangzhaobo99 Sep 23, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

When the add failed, it is to restore the quota usage.

If there was no update before, there is no need to update when it failed.

updateCountNoQuotaCheck(existing, pos, counts.negation());

updateCountNoQuotaCheck(existing, pos, counts.negation());
return null;
} else {
Expand All @@ -1401,10 +1426,10 @@ public INodesInPath addLastINode(INodesInPath existing, INode inode,
}

INodesInPath addLastINodeNoQuotaCheck(INodesInPath existing, INode i,
Optional<QuotaCounts> quotaCount) {
Optional<QuotaCounts> 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);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -79,24 +80,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));
}

Expand Down Expand Up @@ -158,4 +167,60 @@ 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 dstDir1 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 originDstDir2Usage = dfs.getQuotaUsage(dstDir3).getFileAndDirectoryCount();
// Exclude dstDir2 usage
long rootINodeCount1 =
dfs.getQuotaUsage(new Path("/")).getFileAndDirectoryCount() - originDstDir2Usage;
ContentSummary contentSummary3 = dfs.getContentSummary(testParentDir2);

// Src and dst must be same (all file or all dir)
// dstDir2=/testRename/testDir3/dst-dir
// dstDir3=/testRename/testDir3/dst-dir
dfs.rename(dstDir2, dstDir3, Options.Rename.OVERWRITE);
long rootINodeCount2 = dfs.getQuotaUsage(new Path("/")).getFileAndDirectoryCount();
assertEquals(rootINodeCount1, rootINodeCount2);
ContentSummary contentSummary4 = dfs.getContentSummary(testParentDir3);
assertEquals(contentSummary3, contentSummary4);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down