Skip to content

Commit

Permalink
HDFS-9063. Correctly handle snapshot path for getContentSummary. Cont…
Browse files Browse the repository at this point in the history
…ributed by Jing Zhao.
  • Loading branch information
Jing9 committed Sep 18, 2015
1 parent e690a32 commit 3f42753
Show file tree
Hide file tree
Showing 12 changed files with 166 additions and 48 deletions.
2 changes: 2 additions & 0 deletions hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
Expand Up @@ -1367,6 +1367,8 @@ Release 2.8.0 - UNRELEASED
HDFS-9073. Fix failures in TestLazyPersistLockedMemory
testReleaseOnEviction(). (J.Andreina via stevel)

HDFS-9063. Correctly handle snapshot path for getContentSummary. (jing9)

Release 2.7.2 - UNRELEASED

INCOMPATIBLE CHANGES
Expand Down
Expand Up @@ -571,7 +571,8 @@ private static ContentSummary getContentSummaryInt(FSDirectory fsd,
ContentSummaryComputationContext cscc =
new ContentSummaryComputationContext(fsd, fsd.getFSNamesystem(),
fsd.getContentCountLimit(), fsd.getContentSleepMicroSec());
ContentSummary cs = targetNode.computeAndConvertContentSummary(cscc);
ContentSummary cs = targetNode.computeAndConvertContentSummary(
iip.getPathSnapshotId(), cscc);
fsd.addYieldCount(cscc.getYieldCount());
return cs;
}
Expand Down
Expand Up @@ -419,16 +419,17 @@ public abstract void cleanSubtree(ReclaimContext reclaimContext,

/** Compute {@link ContentSummary}. Blocking call */
public final ContentSummary computeContentSummary(BlockStoragePolicySuite bsps) {
return computeAndConvertContentSummary(
return computeAndConvertContentSummary(Snapshot.CURRENT_STATE_ID,
new ContentSummaryComputationContext(bsps));
}

/**
* Compute {@link ContentSummary}.
*/
public final ContentSummary computeAndConvertContentSummary(
public final ContentSummary computeAndConvertContentSummary(int snapshotId,
ContentSummaryComputationContext summary) {
ContentCounts counts = computeContentSummary(summary).getCounts();
ContentCounts counts = computeContentSummary(snapshotId, summary)
.getCounts();
final QuotaCounts q = getQuotaCounts();
return new ContentSummary.Builder().
length(counts.getLength()).
Expand All @@ -445,11 +446,16 @@ public final ContentSummary computeAndConvertContentSummary(
/**
* Count subtree content summary with a {@link ContentCounts}.
*
* @param snapshotId Specify the time range for the calculation. If this
* parameter equals to {@link Snapshot#CURRENT_STATE_ID},
* the result covers both the current states and all the
* snapshots. Otherwise the result only covers all the
* files/directories contained in the specific snapshot.
* @param summary the context object holding counts for the subtree.
* @return The same objects as summary.
*/
public abstract ContentSummaryComputationContext computeContentSummary(
ContentSummaryComputationContext summary);
int snapshotId, ContentSummaryComputationContext summary);


/**
Expand Down
Expand Up @@ -626,18 +626,20 @@ public QuotaCounts computeQuotaUsage4CurrentDirectory(
}

@Override
public ContentSummaryComputationContext computeContentSummary(
public ContentSummaryComputationContext computeContentSummary(int snapshotId,
ContentSummaryComputationContext summary) {
final DirectoryWithSnapshotFeature sf = getDirectoryWithSnapshotFeature();
if (sf != null) {
if (sf != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
// if the getContentSummary call is against a non-snapshot path, the
// computation should include all the deleted files/directories
sf.computeContentSummary4Snapshot(summary.getBlockStoragePolicySuite(),
summary.getCounts());
}
final DirectoryWithQuotaFeature q = getDirectoryWithQuotaFeature();
if (q != null) {
if (q != null && snapshotId == Snapshot.CURRENT_STATE_ID) {
return q.computeContentSummary(this, summary);
} else {
return computeDirectoryContentSummary(summary, Snapshot.CURRENT_STATE_ID);
return computeDirectoryContentSummary(summary, snapshotId);
}
}

Expand All @@ -651,7 +653,7 @@ protected ContentSummaryComputationContext computeDirectoryContentSummary(
byte[] childName = child.getLocalNameBytes();

long lastYieldCount = summary.getYieldCount();
child.computeContentSummary(summary);
child.computeContentSummary(snapshotId, summary);

// Check whether the computation was paused in the subtree.
// The counts may be off, but traversing the rest of children
Expand Down
Expand Up @@ -610,23 +610,10 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,

@Override
public final ContentSummaryComputationContext computeContentSummary(
final ContentSummaryComputationContext summary) {
int snapshotId, final ContentSummaryComputationContext summary) {
final ContentCounts counts = summary.getCounts();
FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
final long fileLen;
if (sf == null) {
fileLen = computeFileSize();
counts.addContent(Content.FILE, 1);
} else {
final FileDiffList diffs = sf.getDiffs();
final int n = diffs.asList().size();
counts.addContent(Content.FILE, n);
if (n > 0 && sf.isCurrentFileDeleted()) {
fileLen = diffs.getLast().getFileSize();
} else {
fileLen = computeFileSize();
}
}
counts.addContent(Content.FILE, 1);
final long fileLen = computeFileSize(snapshotId);
counts.addContent(Content.LENGTH, fileLen);
counts.addContent(Content.DISKSPACE, storagespaceConsumed(null)
.getStorageSpace());
Expand Down
Expand Up @@ -109,7 +109,7 @@ public QuotaCounts computeQuotaUsage(

@Override
public ContentSummaryComputationContext computeContentSummary(
ContentSummaryComputationContext summary) {
int snapshotId, ContentSummaryComputationContext summary) {
return null;
}

Expand Down
Expand Up @@ -313,9 +313,9 @@ public void destroyAndCollectBlocks(ReclaimContext reclaimContext) {
}

@Override
public ContentSummaryComputationContext computeContentSummary(
public ContentSummaryComputationContext computeContentSummary(int snapshotId,
ContentSummaryComputationContext summary) {
return referred.computeContentSummary(summary);
return referred.computeContentSummary(snapshotId, summary);
}

@Override
Expand Down Expand Up @@ -502,11 +502,11 @@ public int getLastSnapshotId() {

@Override
public final ContentSummaryComputationContext computeContentSummary(
ContentSummaryComputationContext summary) {
int snapshotId, ContentSummaryComputationContext summary) {
final int s = snapshotId < lastSnapshotId ? snapshotId : lastSnapshotId;
// only count storagespace for WithName
final QuotaCounts q = computeQuotaUsage(
summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false,
lastSnapshotId);
summary.getBlockStoragePolicySuite(), getStoragePolicyID(), false, s);
summary.getCounts().addContent(Content.DISKSPACE, q.getStorageSpace());
summary.getCounts().addTypeSpaces(q.getTypeSpaces());
return summary;
Expand Down
Expand Up @@ -94,7 +94,7 @@ public QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps,
}

@Override
public ContentSummaryComputationContext computeContentSummary(
public ContentSummaryComputationContext computeContentSummary(int snapshotId,
final ContentSummaryComputationContext summary) {
summary.getCounts().addContent(Content.SYMLINK, 1);
return summary;
Expand Down
Expand Up @@ -31,7 +31,7 @@
import org.apache.hadoop.hdfs.protocol.SnapshotException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.namenode.Content;
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
import org.apache.hadoop.hdfs.server.namenode.INode;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory;
import org.apache.hadoop.hdfs.server.namenode.INodeDirectory.SnapshotAndINode;
Expand Down Expand Up @@ -219,14 +219,12 @@ public Snapshot removeSnapshot(
}
}

public ContentSummaryComputationContext computeContentSummary(
final BlockStoragePolicySuite bsps,
final INodeDirectory snapshotRoot,
final ContentSummaryComputationContext summary) {
snapshotRoot.computeContentSummary(summary);
summary.getCounts().addContent(Content.SNAPSHOT, snapshotsByNames.size());
summary.getCounts().addContent(Content.SNAPSHOTTABLE_DIRECTORY, 1);
return summary;
@Override
public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
final ContentCounts counts) {
counts.addContent(Content.SNAPSHOT, snapshotsByNames.size());
counts.addContent(Content.SNAPSHOTTABLE_DIRECTORY, 1);
super.computeContentSummary4Snapshot(bsps, counts);
}

/**
Expand Down
Expand Up @@ -30,7 +30,6 @@
import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
import org.apache.hadoop.hdfs.server.namenode.AclStorage;
import org.apache.hadoop.hdfs.server.namenode.Content;
import org.apache.hadoop.hdfs.server.namenode.ContentCounts;
import org.apache.hadoop.hdfs.server.namenode.ContentSummaryComputationContext;
import org.apache.hadoop.hdfs.server.namenode.FSImageSerialization;
Expand Down Expand Up @@ -628,13 +627,11 @@ public void computeContentSummary4Snapshot(final BlockStoragePolicySuite bsps,
new ContentSummaryComputationContext(bsps);
for(DirectoryDiff d : diffs) {
for(INode deleted : d.getChildrenDiff().getList(ListType.DELETED)) {
deleted.computeContentSummary(summary);
deleted.computeContentSummary(Snapshot.CURRENT_STATE_ID, summary);
}
}
// Add the counts from deleted trees.
counts.addContents(summary.getCounts());
// Add the deleted directory count.
counts.addContent(Content.DIRECTORY, diffs.asList().size());
}

/**
Expand Down
Expand Up @@ -176,8 +176,7 @@ public INode getChild(byte[] name, int snapshotId) {

@Override
public ContentSummaryComputationContext computeContentSummary(
ContentSummaryComputationContext summary) {
int snapshotId = getParent().getSnapshot(getLocalNameBytes()).getId();
int snapshotId, ContentSummaryComputationContext summary) {
return computeDirectoryContentSummary(summary, snapshotId);
}

Expand Down
@@ -0,0 +1,126 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdfs.server.namenode.snapshot;

import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;

import java.io.FileNotFoundException;
import java.io.IOException;

/**
* Verify content summary is computed correctly when
* 1. There are snapshots taken under the directory
* 2. The given path is a snapshot path
*/
public class TestGetContentSummaryWithSnapshot {
protected static final short REPLICATION = 3;
protected static final long BLOCKSIZE = 1024;

protected Configuration conf;
protected MiniDFSCluster cluster;
protected FSNamesystem fsn;
protected FSDirectory fsdir;
protected DistributedFileSystem dfs;

@Rule
public ExpectedException exception = ExpectedException.none();

@Before
public void setUp() throws Exception {
conf = new Configuration();
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION).build();
cluster.waitActive();

fsn = cluster.getNamesystem();
fsdir = fsn.getFSDirectory();
dfs = cluster.getFileSystem();
}

@After
public void tearDown() throws Exception {
if (cluster != null) {
cluster.shutdown();
}
}

/**
* Calculate against a snapshot path.
* 1. create dirs /foo/bar
* 2. take snapshot s1 on /foo
* 3. create a 10 byte file /foo/bar/baz
* Make sure for "/foo/bar" and "/foo/.snapshot/s1/bar" have correct results:
* the 1 byte file is not included in snapshot s1.
*/
@Test
public void testGetContentSummary() throws IOException {
final Path foo = new Path("/foo");
final Path bar = new Path(foo, "bar");
final Path baz = new Path(bar, "baz");

dfs.mkdirs(bar);
dfs.allowSnapshot(foo);
dfs.createSnapshot(foo, "s1");

DFSTestUtil.createFile(dfs, baz, 10, REPLICATION, 0L);

ContentSummary summary = cluster.getNameNodeRpc().getContentSummary(
bar.toString());
Assert.assertEquals(1, summary.getDirectoryCount());
Assert.assertEquals(1, summary.getFileCount());
Assert.assertEquals(10, summary.getLength());

final Path barS1 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar");
summary = cluster.getNameNodeRpc().getContentSummary(barS1.toString());
Assert.assertEquals(1, summary.getDirectoryCount());
Assert.assertEquals(0, summary.getFileCount());
Assert.assertEquals(0, summary.getLength());

// also check /foo and /foo/.snapshot/s1
summary = cluster.getNameNodeRpc().getContentSummary(foo.toString());
Assert.assertEquals(2, summary.getDirectoryCount());
Assert.assertEquals(1, summary.getFileCount());
Assert.assertEquals(10, summary.getLength());

final Path fooS1 = SnapshotTestHelper.getSnapshotRoot(foo, "s1");
summary = cluster.getNameNodeRpc().getContentSummary(fooS1.toString());
Assert.assertEquals(2, summary.getDirectoryCount());
Assert.assertEquals(0, summary.getFileCount());
Assert.assertEquals(0, summary.getLength());

final Path bazS1 = SnapshotTestHelper.getSnapshotPath(foo, "s1", "bar/baz");
try {
cluster.getNameNodeRpc().getContentSummary(bazS1.toString());
Assert.fail("should get FileNotFoundException");
} catch (FileNotFoundException ignored) {}
}
}

0 comments on commit 3f42753

Please sign in to comment.