Skip to content

Commit

Permalink
HDFS-7811. Avoid recursive call getStoragePolicyID in INodeFile#compu…
Browse files Browse the repository at this point in the history
…teQuotaUsage. Contributed by Xiaoyu Yao and Jing Zhao.

(cherry picked from commit 72f6bd4)

Conflicts:
	hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
  • Loading branch information
Jing9 committed Apr 3, 2015
1 parent 405aead commit 2c1f33d
Show file tree
Hide file tree
Showing 9 changed files with 84 additions and 37 deletions.
3 changes: 3 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -571,6 +571,9 @@ Release 2.7.0 - UNRELEASED
HDFS-8035. Move checkBlocksProperlyReplicated() in FSNamesystem to
BlockManager. (wheat9)

HDFS-7811. Avoid recursive call getStoragePolicyID in
INodeFile#computeQuotaUsage. (Xiaoyu Yao and jing9)

OPTIMIZATIONS

HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -860,23 +860,27 @@ private long loadEdits(Iterable<EditLogInputStream> editStreams,
*/
static void updateCountForQuota(BlockStoragePolicySuite bsps,
INodeDirectory root) {
updateCountForQuotaRecursively(bsps, root, new QuotaCounts.Builder().build());
updateCountForQuotaRecursively(bsps, root.getStoragePolicyID(), root,
new QuotaCounts.Builder().build());
}

private static void updateCountForQuotaRecursively(BlockStoragePolicySuite bsps,
INodeDirectory dir, QuotaCounts counts) {
byte blockStoragePolicyId, INodeDirectory dir, QuotaCounts counts) {
final long parentNamespace = counts.getNameSpace();
final long parentStoragespace = counts.getStorageSpace();
final EnumCounters<StorageType> parentTypeSpaces = counts.getTypeSpaces();

dir.computeQuotaUsage4CurrentDirectory(bsps, counts);
dir.computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId, counts);

for (INode child : dir.getChildrenList(Snapshot.CURRENT_STATE_ID)) {
final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
if (child.isDirectory()) {
updateCountForQuotaRecursively(bsps, child.asDirectory(), counts);
updateCountForQuotaRecursively(bsps, childPolicyId,
child.asDirectory(), counts);
} else {
// file or symlink: count here to reduce recursive calls.
child.computeQuotaUsage(bsps, counts, false);
child.computeQuotaUsage(bsps, childPolicyId, counts, false,
Snapshot.CURRENT_STATE_ID);
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -505,9 +505,14 @@ public final boolean isQuotaSet() {

/**
* Count subtree {@link Quota#NAMESPACE} and {@link Quota#STORAGESPACE} usages.
* Entry point for FSDirectory where blockStoragePolicyId is given its initial
* value.
*/
public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
return computeQuotaUsage(bsps, new QuotaCounts.Builder().build(), true);
final byte storagePolicyId = isSymlink() ?
BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
return computeQuotaUsage(bsps, storagePolicyId,
new QuotaCounts.Builder().build(), true, Snapshot.CURRENT_STATE_ID);
}

/**
Expand All @@ -532,6 +537,7 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
* <pre>
*
* @param bsps Block storage policy suite to calculate intended storage type usage
* @param blockStoragePolicyId block storage policy id of the current INode
* @param counts The subtree counts for returning.
* @param useCache Whether to use cached quota usage. Note that
* {@link WithName} node never uses cache for its subtree.
Expand All @@ -542,12 +548,15 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps) {
* @return The same objects as the counts parameter.
*/
public abstract QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps,
BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
QuotaCounts counts, boolean useCache, int lastSnapshotId);

public final QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache) {
return computeQuotaUsage(bsps, counts, useCache, Snapshot.CURRENT_STATE_ID);
final byte storagePolicyId = isSymlink() ?
BlockStoragePolicySuite.ID_UNSPECIFIED : getStoragePolicyID();
return computeQuotaUsage(bsps, storagePolicyId, counts,
useCache, Snapshot.CURRENT_STATE_ID);
}

/**
Expand Down Expand Up @@ -707,6 +716,20 @@ public final INode setAccessTime(long accessTime, int latestSnapshotId) {
*/
public abstract byte getLocalStoragePolicyID();

/**
* Get the storage policy ID while computing quota usage
* @param parentStoragePolicyId the storage policy ID of the parent directory
* @return the storage policy ID of this INode. Note that for an
* {@link INodeSymlink} we return {@link BlockStoragePolicySuite#ID_UNSPECIFIED}
* instead of throwing Exception
*/
public byte getStoragePolicyIDForQuota(byte parentStoragePolicyId) {
byte localId = isSymlink() ?
BlockStoragePolicySuite.ID_UNSPECIFIED : getLocalStoragePolicyID();
return localId != BlockStoragePolicySuite.ID_UNSPECIFIED ?
localId : parentStoragePolicyId;
}

/**
* Breaks {@code path} into components.
* @return array of byte arrays each of which represents
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,8 @@
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;

import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;

/**
* Directory INode class.
*/
Expand Down Expand Up @@ -123,18 +125,18 @@ public byte getLocalStoragePolicyID() {
return (xattr.getValue())[0];
}
}
return BlockStoragePolicySuite.ID_UNSPECIFIED;
return ID_UNSPECIFIED;
}

@Override
public byte getStoragePolicyID() {
byte id = getLocalStoragePolicyID();
if (id != BlockStoragePolicySuite.ID_UNSPECIFIED) {
if (id != ID_UNSPECIFIED) {
return id;
}
// if it is unspecified, check its parent
return getParent() != null ? getParent().getStoragePolicyID() :
BlockStoragePolicySuite.ID_UNSPECIFIED;
ID_UNSPECIFIED;
}

void setQuota(BlockStoragePolicySuite bsps, long nsQuota, long ssQuota, StorageType type) {
Expand Down Expand Up @@ -568,18 +570,21 @@ private void addChild(final INode node, final int insertionPoint) {
}

@Override
public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
int lastSnapshotId) {
final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();

// we are computing the quota usage for a specific snapshot here, i.e., the
// computation only includes files/directories that exist at the time of the
// given snapshot
if (sf != null && lastSnapshotId != Snapshot.CURRENT_STATE_ID
&& !(useCache && isQuotaSet())) {
ReadOnlyList<INode> childrenList = getChildrenList(lastSnapshotId);
for (INode child : childrenList) {
child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
lastSnapshotId);
}
counts.addNameSpace(1);
return counts;
Expand All @@ -591,28 +596,33 @@ public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, QuotaCounts c
return q.AddCurrentSpaceUsage(counts);
} else {
useCache = q != null && !q.isQuotaSet() ? false : useCache;
return computeDirectoryQuotaUsage(bsps, counts, useCache, lastSnapshotId);
return computeDirectoryQuotaUsage(bsps, blockStoragePolicyId, counts,
useCache, lastSnapshotId);
}
}

private QuotaCounts computeDirectoryQuotaUsage(BlockStoragePolicySuite bsps,
QuotaCounts counts, boolean useCache, int lastSnapshotId) {
byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
int lastSnapshotId) {
if (children != null) {
for (INode child : children) {
child.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
final byte childPolicyId = child.getStoragePolicyIDForQuota(blockStoragePolicyId);
child.computeQuotaUsage(bsps, childPolicyId, counts, useCache,
lastSnapshotId);
}
}
return computeQuotaUsage4CurrentDirectory(bsps, counts);
return computeQuotaUsage4CurrentDirectory(bsps, blockStoragePolicyId,
counts);
}

/** Add quota usage for this inode excluding children. */
public QuotaCounts computeQuotaUsage4CurrentDirectory(
BlockStoragePolicySuite bsps, QuotaCounts counts) {
BlockStoragePolicySuite bsps, byte storagePolicyId, QuotaCounts counts) {
counts.addNameSpace(1);
// include the diff list
DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
if (sf != null) {
sf.computeQuotaUsage4CurrentDirectory(bsps, counts);
sf.computeQuotaUsage4CurrentDirectory(bsps, storagePolicyId, counts);
}
return counts;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,14 @@
*/
package org.apache.hadoop.hdfs.server.namenode;

import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintWriter;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
Expand Down Expand Up @@ -393,7 +393,7 @@ public byte getLocalStoragePolicyID() {
@Override
public byte getStoragePolicyID() {
byte id = getLocalStoragePolicyID();
if (id == BlockStoragePolicySuite.ID_UNSPECIFIED) {
if (id == ID_UNSPECIFIED) {
return this.getParent() != null ?
this.getParent().getStoragePolicyID() : id;
}
Expand Down Expand Up @@ -554,7 +554,8 @@ public String getName() {
// derive the intended storage type usage for quota by storage type
@Override
public final QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps, QuotaCounts counts, boolean useCache,
BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
QuotaCounts counts, boolean useCache,
int lastSnapshotId) {
long nsDelta = 1;
final long ssDeltaNoReplication;
Expand Down Expand Up @@ -583,8 +584,8 @@ public final QuotaCounts computeQuotaUsage(
counts.addNameSpace(nsDelta);
counts.addStorageSpace(ssDeltaNoReplication * replication);

if (getStoragePolicyID() != BlockStoragePolicySuite.ID_UNSPECIFIED){
BlockStoragePolicy bsp = bsps.getPolicy(getStoragePolicyID());
if (blockStoragePolicyId != ID_UNSPECIFIED){
BlockStoragePolicy bsp = bsps.getPolicy(blockStoragePolicyId);
List<StorageType> storageTypes = bsp.chooseStorageTypes(replication);
for (StorageType t : storageTypes) {
if (!t.supportTypeQuota()) {
Expand Down Expand Up @@ -618,7 +619,7 @@ public final ContentSummaryComputationContext computeContentSummary(
counts.addContent(Content.LENGTH, fileLen);
counts.addContent(Content.DISKSPACE, storagespaceConsumed());

if (getStoragePolicyID() != BlockStoragePolicySuite.ID_UNSPECIFIED){
if (getStoragePolicyID() != ID_UNSPECIFIED){
BlockStoragePolicy bsp = summary.getBlockStoragePolicySuite().
getPolicy(getStoragePolicyID());
List<StorageType> storageTypes = bsp.chooseStorageTypes(getFileReplication());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -103,8 +103,8 @@ public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,

@Override
public QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps, QuotaCounts counts,
boolean useCache, int lastSnapshotId) {
BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
QuotaCounts counts, boolean useCache, int lastSnapshotId) {
return null;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import java.util.Comparator;
import java.util.List;

import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.permission.PermissionStatus;
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
Expand Down Expand Up @@ -327,9 +326,10 @@ public ContentSummaryComputationContext computeContentSummary(

@Override
public QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps,
BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
QuotaCounts counts, boolean useCache, int lastSnapshotId) {
return referred.computeQuotaUsage(bsps, counts, useCache, lastSnapshotId);
return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
useCache, lastSnapshotId);
}

@Override
Expand Down Expand Up @@ -512,15 +512,17 @@ public final ContentSummaryComputationContext computeContentSummary(
ContentSummaryComputationContext summary) {
//only count storagespace for WithName
final QuotaCounts q = new QuotaCounts.Builder().build();
computeQuotaUsage(summary.getBlockStoragePolicySuite(), q, false, lastSnapshotId);
computeQuotaUsage(summary.getBlockStoragePolicySuite(),
getStoragePolicyID(), q, false, lastSnapshotId);
summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
summary.getCounts().addTypeSpaces(q.getTypeSpaces());
return summary;
}

@Override
public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
QuotaCounts counts, boolean useCache, int lastSnapshotId) {
byte blockStoragePolicyId, QuotaCounts counts, boolean useCache,
int lastSnapshotId) {
// if this.lastSnapshotId < lastSnapshotId, the rename of the referred
// node happened before the rename of its ancestor. This should be
// impossible since for WithName node we only count its children at the
Expand All @@ -535,7 +537,8 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
// been updated by changes in the current tree.
int id = lastSnapshotId != Snapshot.CURRENT_STATE_ID ?
lastSnapshotId : this.lastSnapshotId;
return referred.computeQuotaUsage(bsps, counts, false, id);
return referred.computeQuotaUsage(bsps, blockStoragePolicyId, counts,
false, id);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public void destroyAndCollectBlocks(final BlockStoragePolicySuite bsps,

@Override
public QuotaCounts computeQuotaUsage(
BlockStoragePolicySuite bsps,
BlockStoragePolicySuite bsps, byte blockStoragePolicyId,
QuotaCounts counts, boolean useCache, int lastSnapshotId) {
counts.addNameSpace(1);
return counts;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -641,10 +641,13 @@ public void clear(BlockStoragePolicySuite bsps, INodeDirectory currentINode,
}

public QuotaCounts computeQuotaUsage4CurrentDirectory(
BlockStoragePolicySuite bsps, QuotaCounts counts) {
BlockStoragePolicySuite bsps, byte storagePolicyId,
QuotaCounts counts) {
for(DirectoryDiff d : diffs) {
for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
deleted.computeQuotaUsage(bsps, counts, false, Snapshot.CURRENT_STATE_ID);
final byte childPolicyId = deleted.getStoragePolicyIDForQuota(storagePolicyId);
deleted.computeQuotaUsage(bsps, childPolicyId, counts, false,
Snapshot.CURRENT_STATE_ID);
}
}
return counts;
Expand Down

0 comments on commit 2c1f33d

Please sign in to comment.