Michael Luckey created BEAM-1569:
------------------------------------
Summary: HDFSFileSource: Unable to read from filePattern with
spaces in path
Key: BEAM-1569
URL: https://issues.apache.org/jira/browse/BEAM-1569
Project: Beam
Issue Type: Bug
Components: sdk-java-core
Reporter: Michael Luckey
Assignee: Davor Bonaci
After the merge of the changes introduced with
https://issues.apache.org/jira/browse/BEAM-1497 we are unable to read from
files containing spaces in path. We encounter following stack trace
{noformat}
java.lang.reflect.UndeclaredThrowableException
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1713)
at
org.apache.beam.sdk.io.hdfs.HDFSFileSource.validate(HDFSFileSource.java:337)
at
org.apache.beam.sdk.io.hdfs.HDFSFileSource.createReader(HDFSFileSource.java:329)
at
org.apache.beam.sdk.testing.SourceTestUtils.readFromSource(SourceTestUtils.java:138)
Caused by: java.net.URISyntaxException: Illegal character in path at index 77:
/var/folders/1t/s9pcmfj50nxbt68h3_2z_5wc0000gn/T/junit6887354597440386901/tmp
data.seq
at java.net.URI$Parser.fail(URI.java:2848)
at java.net.URI$Parser.checkChars(URI.java:3021)
at java.net.URI$Parser.parseHierarchical(URI.java:3105)
at java.net.URI$Parser.parse(URI.java:3063)
at java.net.URI.<init>(URI.java:588)
at
org.apache.beam.sdk.io.hdfs.HDFSFileSource$7.run(HDFSFileSource.java:340)
at
org.apache.beam.sdk.io.hdfs.HDFSFileSource$7.run(HDFSFileSource.java:337)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:422)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1698)
... 40 more
{noformat}
This can be reproduced for instance by
{noformat}
// shameless copy of existing test case
@Test
public void testFullyReadSingleFileWithSpaces() throws Exception {
PipelineOptions options = PipelineOptionsFactory.create();
List<KV<IntWritable, Text>> expectedResults = createRandomRecords(3, 10, 0);
File file = createFileWithData("tmp data.seq", expectedResults);
HDFSFileSource<KV<IntWritable, Text>, IntWritable, Text> source =
HDFSFileSource.from(
file.toString(), SequenceFileInputFormat.class,
IntWritable.class, Text.class);
assertEquals(file.length(), source.getEstimatedSizeBytes(null));
assertThat(expectedResults, containsInAnyOrder(readFromSource(source,
options).toArray()));
}
{noformat}
Changing the implementation slightly to
{noformat}
diff --git
a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
index 2a731fb..df72643 100644
---
a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
+++
b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java
@@ -30,7 +30,6 @@ import java.io.ObjectInput;
import java.io.ObjectOutput;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
-import java.net.URI;
import java.security.PrivilegedExceptionAction;
import java.util.List;
import java.util.ListIterator;
@@ -337,9 +336,10 @@ public abstract class HDFSFileSource<T, K, V> extends
BoundedSource<T> {
UGIHelper.getBestUGI(username()).doAs(new
PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
- FileSystem fs = FileSystem.get(new URI(filepattern()),
+ final Path pathPattern = new Path(filepattern());
+ FileSystem fs = FileSystem.get(pathPattern.toUri(),
SerializableConfiguration.newConfiguration(serializableConfiguration()));
- FileStatus[] fileStatuses = fs.globStatus(new
Path(filepattern()));
+ FileStatus[] fileStatuses = fs.globStatus(pathPattern);
checkState(
fileStatuses != null && fileStatuses.length > 0,
"Unable to find any files matching %s", filepattern());
{noformat}
seems to be fixing the issue for us.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)