|
@@ -17,6 +17,8 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
+import org.apache.commons.lang3.tuple.Pair;
|
|
|
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
|
|
import org.apache.hadoop.util.Preconditions;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
@@ -43,6 +45,8 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
+import java.util.Optional;
|
|
|
+
|
|
|
import static org.apache.hadoop.hdfs.protocol.FSLimitException.MaxDirectoryItemsExceededException;
|
|
|
import static org.apache.hadoop.hdfs.protocol.FSLimitException.PathComponentTooLongException;
|
|
|
|
|
@@ -68,14 +72,18 @@ class FSDirRenameOp {
|
|
|
* Verify quota for rename operation where srcInodes[srcInodes.length-1] moves
|
|
|
* dstInodes[dstInodes.length-1]
|
|
|
*/
|
|
|
- private static void verifyQuotaForRename(FSDirectory fsd, INodesInPath src,
|
|
|
- INodesInPath dst) throws QuotaExceededException {
|
|
|
+ private static Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> verifyQuotaForRename(
|
|
|
+ FSDirectory fsd, INodesInPath src, INodesInPath dst) throws QuotaExceededException {
|
|
|
+ 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;
|
|
|
+ return Pair.of(srcDelta, dstDelta);
|
|
|
}
|
|
|
int i = 0;
|
|
|
- while(src.getINode(i) == dst.getINode(i)) { i++; }
|
|
|
+ while (src.getINode(i) == dst.getINode(i)) {
|
|
|
+ i++;
|
|
|
+ }
|
|
|
// src[i - 1] is the last common ancestor.
|
|
|
BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
|
|
|
// Assume dstParent existence check done by callers.
|
|
@@ -88,13 +96,19 @@ class FSDirRenameOp {
|
|
|
final QuotaCounts delta = src.getLastINode()
|
|
|
.computeQuotaUsage(bsps, storagePolicyID, false,
|
|
|
Snapshot.CURRENT_STATE_ID);
|
|
|
+ QuotaCounts srcQuota = new QuotaCounts.Builder().quotaCount(delta).build();
|
|
|
+ srcDelta = Optional.of(srcQuota);
|
|
|
|
|
|
// Reduce the required quota by dst that is being removed
|
|
|
final INode dstINode = dst.getLastINode();
|
|
|
if (dstINode != null) {
|
|
|
- delta.subtract(dstINode.computeQuotaUsage(bsps));
|
|
|
+ QuotaCounts counts = dstINode.computeQuotaUsage(bsps);
|
|
|
+ QuotaCounts dstQuota = new QuotaCounts.Builder().quotaCount(counts).build();
|
|
|
+ dstDelta = Optional.of(dstQuota);
|
|
|
+ delta.subtract(counts);
|
|
|
}
|
|
|
FSDirectory.verifyQuota(dst, dst.length() - 1, delta, src.getINode(i - 1));
|
|
|
+ return Pair.of(srcDelta, dstDelta);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -202,9 +216,10 @@ class FSDirRenameOp {
|
|
|
fsd.ezManager.checkMoveValidity(srcIIP, dstIIP);
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
|
|
|
- verifyQuotaForRename(fsd, srcIIP, dstIIP);
|
|
|
+ Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> countPair =
|
|
|
+ verifyQuotaForRename(fsd, srcIIP, dstIIP);
|
|
|
|
|
|
- RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP);
|
|
|
+ RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, countPair);
|
|
|
|
|
|
boolean added = false;
|
|
|
|
|
@@ -421,9 +436,10 @@ class FSDirRenameOp {
|
|
|
|
|
|
// Ensure dst has quota to accommodate rename
|
|
|
verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
|
|
|
- verifyQuotaForRename(fsd, srcIIP, dstIIP);
|
|
|
+ Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> quotaPair =
|
|
|
+ verifyQuotaForRename(fsd, srcIIP, dstIIP);
|
|
|
|
|
|
- RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP);
|
|
|
+ RenameOperation tx = new RenameOperation(fsd, srcIIP, dstIIP, quotaPair);
|
|
|
|
|
|
boolean undoRemoveSrc = true;
|
|
|
tx.removeSrc();
|
|
@@ -638,16 +654,22 @@ class FSDirRenameOp {
|
|
|
private final byte[] srcChildName;
|
|
|
private final boolean isSrcInSnapshot;
|
|
|
private final boolean srcChildIsReference;
|
|
|
- private final QuotaCounts oldSrcCounts;
|
|
|
+ private final QuotaCounts oldSrcCountsInSnapshot;
|
|
|
+ private final boolean sameStoragePolicy;
|
|
|
+ private final Optional<QuotaCounts> srcSubTreeCount;
|
|
|
+ private final Optional<QuotaCounts> dstSubTreeCount;
|
|
|
private INode srcChild;
|
|
|
private INode oldDstChild;
|
|
|
|
|
|
- RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP) {
|
|
|
+ RenameOperation(FSDirectory fsd, INodesInPath srcIIP, INodesInPath dstIIP,
|
|
|
+ Pair<Optional<QuotaCounts>, Optional<QuotaCounts>> quotaPair) {
|
|
|
this.fsd = fsd;
|
|
|
this.srcIIP = srcIIP;
|
|
|
this.dstIIP = dstIIP;
|
|
|
this.srcParentIIP = srcIIP.getParentINodesInPath();
|
|
|
this.dstParentIIP = dstIIP.getParentINodesInPath();
|
|
|
+ this.sameStoragePolicy = isSameStoragePolicy();
|
|
|
+
|
|
|
|
|
|
BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
|
|
|
srcChild = this.srcIIP.getLastINode();
|
|
@@ -672,7 +694,7 @@ class FSDirRenameOp {
|
|
|
// check srcChild for reference
|
|
|
srcRefDstSnapshot = srcChildIsReference ?
|
|
|
srcChild.asReference().getDstSnapshotId() : Snapshot.CURRENT_STATE_ID;
|
|
|
- oldSrcCounts = new QuotaCounts.Builder().build();
|
|
|
+ oldSrcCountsInSnapshot = new QuotaCounts.Builder().build();
|
|
|
if (isSrcInSnapshot) {
|
|
|
final INodeReference.WithName withName = srcParent
|
|
|
.replaceChild4ReferenceWithName(srcChild, srcLatestSnapshotId);
|
|
@@ -681,7 +703,7 @@ class FSDirRenameOp {
|
|
|
this.srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1,
|
|
|
srcChild);
|
|
|
// get the counts before rename
|
|
|
- oldSrcCounts.add(withCount.getReferredINode().computeQuotaUsage(bsps));
|
|
|
+ oldSrcCountsInSnapshot.add(withCount.getReferredINode().computeQuotaUsage(bsps));
|
|
|
} else if (srcChildIsReference) {
|
|
|
// srcChild is reference but srcChild is not in latest snapshot
|
|
|
withCount = (INodeReference.WithCount) srcChild.asReference()
|
|
@@ -689,6 +711,38 @@ class FSDirRenameOp {
|
|
|
} else {
|
|
|
withCount = null;
|
|
|
}
|
|
|
+ // Set quota for src and dst, ignore src is in Snapshot or is Reference
|
|
|
+ this.srcSubTreeCount = withCount == null ?
|
|
|
+ quotaPair.getLeft() : Optional.empty();
|
|
|
+ this.dstSubTreeCount = quotaPair.getRight();
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean isSameStoragePolicy() {
|
|
|
+ final INode src = srcIIP.getLastINode();
|
|
|
+ final INode dst = dstIIP.getLastINode();
|
|
|
+ // If the source INode has a storagePolicyID, we should use
|
|
|
+ // its storagePolicyId to update dst`s quota usage.
|
|
|
+ if (src.isSetStoragePolicy()) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ final byte srcSp;
|
|
|
+ final byte dstSp;
|
|
|
+ if (dst == null) {
|
|
|
+ dstSp = dstIIP.getINode(-2).getStoragePolicyID();
|
|
|
+ } else if (dst.isSymlink()) {
|
|
|
+ dstSp = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
+ } else {
|
|
|
+ dstSp = dst.getStoragePolicyID();
|
|
|
+ }
|
|
|
+
|
|
|
+ if (src.isSymlink()) {
|
|
|
+ srcSp = HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
|
|
|
+ } else {
|
|
|
+ // Update src should use src·s storage policyID
|
|
|
+ srcSp = src.getStoragePolicyID();
|
|
|
+ }
|
|
|
+ return srcSp == dstSp;
|
|
|
}
|
|
|
|
|
|
long removeSrc() throws IOException {
|
|
@@ -701,7 +755,9 @@ class FSDirRenameOp {
|
|
|
throw new IOException(error);
|
|
|
} else {
|
|
|
// update the quota count if necessary
|
|
|
- fsd.updateCountForDelete(srcChild, srcIIP);
|
|
|
+ Optional<QuotaCounts> countOp = sameStoragePolicy ?
|
|
|
+ srcSubTreeCount : Optional.empty();
|
|
|
+ fsd.updateCountForDelete(srcChild, srcIIP, countOp);
|
|
|
srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
|
|
|
return removedNum;
|
|
|
}
|
|
@@ -716,7 +772,9 @@ class FSDirRenameOp {
|
|
|
return false;
|
|
|
} else {
|
|
|
// update the quota count if necessary
|
|
|
- fsd.updateCountForDelete(srcChild, srcIIP);
|
|
|
+ Optional<QuotaCounts> countOp = sameStoragePolicy ?
|
|
|
+ srcSubTreeCount : Optional.empty();
|
|
|
+ fsd.updateCountForDelete(srcChild, srcIIP, countOp);
|
|
|
srcIIP = INodesInPath.replace(srcIIP, srcIIP.length() - 1, null);
|
|
|
return true;
|
|
|
}
|
|
@@ -727,7 +785,7 @@ class FSDirRenameOp {
|
|
|
if (removedNum != -1) {
|
|
|
oldDstChild = dstIIP.getLastINode();
|
|
|
// update the quota count if necessary
|
|
|
- fsd.updateCountForDelete(oldDstChild, dstIIP);
|
|
|
+ fsd.updateCountForDelete(oldDstChild, dstIIP, dstSubTreeCount);
|
|
|
dstIIP = INodesInPath.replace(dstIIP, dstIIP.length() - 1, null);
|
|
|
}
|
|
|
return removedNum;
|
|
@@ -745,7 +803,7 @@ class FSDirRenameOp {
|
|
|
toDst = new INodeReference.DstReference(dstParent.asDirectory(),
|
|
|
withCount, dstIIP.getLatestSnapshotId());
|
|
|
}
|
|
|
- return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst);
|
|
|
+ return fsd.addLastINodeNoQuotaCheck(dstParentIIP, toDst, srcSubTreeCount);
|
|
|
}
|
|
|
|
|
|
void updateMtimeAndLease(long timestamp) {
|
|
@@ -777,7 +835,9 @@ class FSDirRenameOp {
|
|
|
} else {
|
|
|
// srcParent is not an INodeDirectoryWithSnapshot, we only need to add
|
|
|
// the srcChild back
|
|
|
- fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild);
|
|
|
+ Optional<QuotaCounts> countOp = sameStoragePolicy ?
|
|
|
+ srcSubTreeCount : Optional.empty();
|
|
|
+ fsd.addLastINodeNoQuotaCheck(srcParentIIP, srcChild, countOp);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -787,7 +847,7 @@ class FSDirRenameOp {
|
|
|
if (dstParent.isWithSnapshot()) {
|
|
|
dstParent.undoRename4DstParent(bsps, oldDstChild, dstIIP.getLatestSnapshotId());
|
|
|
} else {
|
|
|
- fsd.addLastINodeNoQuotaCheck(dstParentIIP, oldDstChild);
|
|
|
+ fsd.addLastINodeNoQuotaCheck(dstParentIIP, oldDstChild, dstSubTreeCount);
|
|
|
}
|
|
|
if (oldDstChild != null && oldDstChild.isReference()) {
|
|
|
final INodeReference removedDstRef = oldDstChild.asReference();
|
|
@@ -826,7 +886,7 @@ class FSDirRenameOp {
|
|
|
if (isSrcInSnapshot) {
|
|
|
// get the counts after rename
|
|
|
QuotaCounts newSrcCounts = srcChild.computeQuotaUsage(bsps, false);
|
|
|
- newSrcCounts.subtract(oldSrcCounts);
|
|
|
+ newSrcCounts.subtract(oldSrcCountsInSnapshot);
|
|
|
srcParent.addSpaceConsumed(newSrcCounts);
|
|
|
}
|
|
|
}
|