HDFS-8581. ContentSummary on / skips further counts on yielding lock (contributed by J.Andreina)

This commit is contained in:
Vinayakumar B 2015-09-10 00:08:19 +05:30
parent 0f0e897bf1
commit 4014ce5990
3 changed files with 36 additions and 1 deletions

View File

@ -1326,6 +1326,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8939. Test(S)WebHdfsFileContextMainOperations failing on branch-2.
(Chris Nauroth via jghoman)
HDFS-8581. ContentSummary on / skips further counts on yielding lock
(J.Andreina via vinayakumarb)
Release 2.7.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -660,7 +660,7 @@ public class INodeDirectory extends INodeWithAdditionalFields
continue;
}
// The locks were released and reacquired. Check parent first.
if (getParent() == null) {
if (!isRoot() && getParent() == null) {
// Stop further counting and return whatever we have so far.
break;
}

View File

@ -24,12 +24,14 @@ import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.io.PrintStream;
import java.security.PrivilegedExceptionAction;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
@ -1007,4 +1009,34 @@ public class TestQuota {
assertTrue(errOutput.contains(StorageType.getTypesSupportingQuota()
.toString()));
}
/**
* File count on root , should return total value of files in Filesystem
* when one folder contains files more than "dfs.content-summary.limit".
*/
@Test
public void testHugeFileCount() throws IOException {
MiniDFSCluster cluster = null;
Configuration conf = new Configuration();
conf.setInt("dfs.content-summary.limit", 4);
try {
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(0).build();
DistributedFileSystem dfs = cluster.getFileSystem();
for (int i = 1; i <= 5; i++) {
FSDataOutputStream out =
dfs.create(new Path("/Folder1/" + "file" + i),(short)1);
out.close();
}
FSDataOutputStream out = dfs.create(new Path("/Folder2/file6"),(short)1);
out.close();
ContentSummary contentSummary = dfs.getContentSummary(new Path("/"));
assertEquals(6, contentSummary.getFileCount());
} finally {
if (cluster != null) {
cluster.shutdown();
cluster = null;
}
}
}
}