Repository: beam Updated Branches: refs/heads/master 884ac517b -> b0fb5b24d
[BEAM-2369] HadoopFileSystem: prevent NPE on match of non existing file Project: http://git-wip-us.apache.org/repos/asf/beam/repo Commit: http://git-wip-us.apache.org/repos/asf/beam/commit/4d86cece Tree: http://git-wip-us.apache.org/repos/asf/beam/tree/4d86cece Diff: http://git-wip-us.apache.org/repos/asf/beam/diff/4d86cece Branch: refs/heads/master Commit: 4d86cece9b27c7db13bb7c3ae3f0249e16273ac2 Parents: 884ac51 Author: Michael Luckey <[email protected]> Authored: Fri May 26 15:52:42 2017 +0200 Committer: Luke Cwik <[email protected]> Committed: Fri May 26 16:25:17 2017 -0700 ---------------------------------------------------------------------- .../beam/sdk/io/hdfs/HadoopFileSystem.java | 7 +++++ .../beam/sdk/io/hdfs/HadoopFileSystemTest.java | 33 ++++++++++++++++++++ 2 files changed, 40 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/beam/blob/4d86cece/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java ---------------------------------------------------------------------- diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java index d519a8c..803ddb6 100644 --- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java +++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystem.java @@ -28,6 +28,7 @@ import java.nio.channels.SeekableByteChannel; import java.nio.channels.WritableByteChannel; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.List; import org.apache.beam.sdk.io.FileSystem; import org.apache.beam.sdk.io.fs.CreateOptions; @@ -78,6 +79,12 @@ class HadoopFileSystem extends FileSystem<HadoopResourceId> { for (String spec : specs) { try { FileStatus[] fileStatuses = fileSystem.globStatus(new Path(spec)); + if (fileStatuses == null) { + resultsBuilder.add(MatchResult.create(Status.NOT_FOUND, + Collections.<Metadata>emptyList())); + continue; + } + List<Metadata> metadata = new ArrayList<>(); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isFile()) { http://git-wip-us.apache.org/repos/asf/beam/blob/4d86cece/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java ---------------------------------------------------------------------- diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java index 88275f4..d06142d 100644 --- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java +++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java @@ -19,6 +19,8 @@ package org.apache.beam.sdk.io.hdfs; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -158,6 +160,37 @@ public class HadoopFileSystemTest { } @Test + public void testMatchForNonExistentFile() throws Exception { + create("testFileAA", "testDataAA".getBytes()); + create("testFileBB", "testDataBB".getBytes()); + + // ensure files exist + assertArrayEquals("testDataAA".getBytes(), read("testFileAA")); + assertArrayEquals("testDataBB".getBytes(), read("testFileBB")); + + List<MatchResult> matchResults = fileSystem.match(ImmutableList.of( + testPath("testFileAA").toString(), + testPath("testFileA").toString(), + testPath("testFileBB").toString())); + + assertThat(matchResults, hasSize(3)); + + final List<MatchResult> expected = ImmutableList.of( + MatchResult.create(Status.OK, ImmutableList.of(Metadata.builder() + .setResourceId(testPath("testFileAA")) + .setIsReadSeekEfficient(true) + .setSizeBytes("testDataAA".getBytes().length) + .build())), + MatchResult.create(Status.NOT_FOUND, ImmutableList.<Metadata>of()), + MatchResult.create(Status.OK, ImmutableList.of(Metadata.builder() + .setResourceId(testPath("testFileBB")) + .setIsReadSeekEfficient(true) + .setSizeBytes("testDataBB".getBytes().length) + .build()))); + assertThat(matchResults, equalTo(expected)); + } + + @Test public void testRename() throws Exception { create("testFileA", "testDataA".getBytes()); create("testFileB", "testDataB".getBytes());
