HDFS-5228. The RemoteIterator returned by DistributedFileSystem.listFiles may throw NullPointerException. Contributed by szetszwo and cnauroth

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1525828 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Tsz-wo Sze 2013-09-24 10:16:14 +00:00
parent b838ac89a6
commit 5827d1667c
3 changed files with 33 additions and 4 deletions

View File

@ -952,6 +952,9 @@ Release 2.1.0-beta - 2013-08-22
HDFS-5016. Deadlock in pipeline recovery causes Datanode to be marked dead.
(suresh)
HDFS-5228. The RemoteIterator returned by DistributedFileSystem.listFiles
may throw NullPointerException. (szetszwo and cnauroth via szetszwo)
BREAKDOWN OF HDFS-347 SUBTASKS AND RELATED JIRAS
HDFS-4353. Encapsulate connections to peers in Peer and PeerServer classes.

View File

@ -713,6 +713,7 @@ public class DistributedFileSystem extends FileSystem {
protected RemoteIterator<LocatedFileStatus> listLocatedStatus(final Path p,
final PathFilter filter)
throws IOException {
final Path absF = fixRelativePart(p);
return new RemoteIterator<LocatedFileStatus>() {
private DirectoryListing thisListing;
private int i;
@ -722,7 +723,7 @@ public class DistributedFileSystem extends FileSystem {
{ // initializer
// Fully resolve symlinks in path first to avoid additional resolution
// round-trips as we fetch more batches of listings
src = getPathName(resolvePath(p));
src = getPathName(resolvePath(absF));
// fetch the first batch of entries in the directory
thisListing = dfs.listPaths(src, HdfsFileStatus.EMPTY_NAME, true);
statistics.incrementReadOps(1);
@ -736,7 +737,7 @@ public class DistributedFileSystem extends FileSystem {
while (curStat == null && hasNextNoFilter()) {
LocatedFileStatus next =
((HdfsLocatedFileStatus)thisListing.getPartialListing()[i++])
.makeQualifiedLocated(getUri(), p);
.makeQualifiedLocated(getUri(), absF);
if (filter.accept(next.getPath())) {
curStat = next;
}

View File

@ -31,8 +31,10 @@ import java.io.FileNotFoundException;
import java.io.IOException;
import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.List;
import java.util.Random;
import org.apache.commons.lang.ArrayUtils;
@ -47,9 +49,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
import org.apache.hadoop.fs.Options.ChecksumOpt;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.VolumeId;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
@ -226,7 +230,7 @@ public class TestDistributedFileSystem {
final long millis = Time.now();
{
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
final DistributedFileSystem dfs = cluster.getFileSystem();
dfs.dfs.getLeaseRenewer().setGraceSleepPeriod(grace);
assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
@ -326,7 +330,7 @@ public class TestDistributedFileSystem {
}
{
DistributedFileSystem dfs = (DistributedFileSystem)cluster.getFileSystem();
final DistributedFileSystem dfs = cluster.getFileSystem();
assertFalse(dfs.dfs.getLeaseRenewer().isRunning());
//open and check the file
@ -835,4 +839,25 @@ public class TestDistributedFileSystem {
}
}
@Test(timeout=60000)
public void testListFiles() throws IOException {
Configuration conf = new HdfsConfiguration();
MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf).build();
try {
DistributedFileSystem fs = cluster.getFileSystem();
final Path relative = new Path("relative");
fs.create(new Path(relative, "foo")).close();
final List<LocatedFileStatus> retVal = new ArrayList<LocatedFileStatus>();
final RemoteIterator<LocatedFileStatus> iter = fs.listFiles(relative, true);
while (iter.hasNext()) {
retVal.add(iter.next());
}
System.out.println("retVal = " + retVal);
} finally {
cluster.shutdown();
}
}
}