HDFS-8581. ContentSummary on / skips further counts on yielding lock (contributed by J.Andreina)
(cherry picked from commit 4014ce5990bff9b0ecb3d38a633d40eaf6cf07a7) (cherry picked from commit 8854cdd9eefd05c10d0518528a3bff6a7348f37e) Conflicts: hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
This commit is contained in:
parent
a2d960b6dc
commit
867b29be9c
@ -27,6 +27,9 @@ Release 2.6.5 - UNRELEASED
|
||||
HDFS-10271. Extra bytes are getting released from reservedSpace for append
|
||||
(Brahma Reddy Battula via vinayakumarb)
|
||||
|
||||
HDFS-8581. ContentSummary on / skips further counts on yielding lock
|
||||
(J.Andreina via vinayakumarb)
|
||||
|
||||
Release 2.6.4 - 2016-02-11
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -638,7 +638,7 @@ ContentSummaryComputationContext computeDirectoryContentSummary(
|
||||
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;
|
||||
}
|
||||
|
@ -21,11 +21,13 @@
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
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.hdfs.protocol.DSQuotaExceededException;
|
||||
@ -921,4 +923,33 @@ public void testMultipleFilesSmallerThanOneBlock() throws Exception {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user