[
https://issues.apache.org/jira/browse/CRUNCH-622?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Josh Wills resolved CRUNCH-622.
-------------------------------
Resolution: Fixed
Pushed to master.
> From.avroFile fails if path not on default filesystem
> -----------------------------------------------------
>
> Key: CRUNCH-622
> URL: https://issues.apache.org/jira/browse/CRUNCH-622
> Project: Crunch
> Issue Type: Bug
> Components: Core
> Affects Versions: 0.13.0, 0.14.0
> Reporter: Tom De Leu
> Assignee: Micah Whitacre
> Fix For: 0.15.0
>
> Attachments: CRUNCH-622.patch
>
>
> {noformat}
> MemPipeline.getInstance().read(From.avroFile(new
> Path("s3:///something")));
> {noformat}
> Fails with:
> {noformat}
> java.lang.IllegalArgumentException: Wrong FS: s3:/something, expected:
> file:///
> at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:643)
> at
> org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:80)
> at
> org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:519)
> at
> org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:737)
> at
> org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:514)
> at
> org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:397)
> at org.apache.hadoop.fs.FileSystem.isFile(FileSystem.java:1424)
> at org.apache.crunch.io.From.getSchemaFromPath(From.java:351)
> at org.apache.crunch.io.From.avroFile(From.java:306)
> at org.apache.crunch.io.From.avroFile(From.java:280)
> {noformat}
> I noticed this in the From class, method getSchemaFromPath:
> {noformat}
> FileSystem fs = FileSystem.get(conf);
> {noformat}
> Shouldn't that be changed to this?
> {noformat}
> FileSystem fs = path.getFileSystem(conf);
> {noformat}
> We ran into this in a usecase where the file was on a valid path on S3 but
> the Configuration was pointing to HDFS, which I believe should just work.
>
> After some googling, I also found CRUNCH-47 which seems related, but the
> patch there couldn't fix the From/At/To helpers as they were introduced
> later...
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)