[
https://issues.apache.org/jira/browse/MAPREDUCE-5038?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13663210#comment-13663210
]
Sandy Ryza commented on MAPREDUCE-5038:
---------------------------------------
To elaborate on that a little further, the relevant change on the MR side was:
{code}
- Path p = new Path(paths[i].toUri().getPath());
+ FileSystem fs = paths[i].getFileSystem(conf);
+ Path p = fs.makeQualified(paths[i]);
{code}
so a path that was previously not examined now is, which makes me think that
the change exposed a preexisting bug. Not having had success at running the
Hive tests on top of a patched MR, I can't speak with 100% confidence, but if
anyone knowledgeable about the Hive side wants to take a look, I'd be happy to
help investigate on or offline.
> old API CombineFileInputFormat missing fixes that are in new API
> -----------------------------------------------------------------
>
> Key: MAPREDUCE-5038
> URL: https://issues.apache.org/jira/browse/MAPREDUCE-5038
> Project: Hadoop Map/Reduce
> Issue Type: Bug
> Affects Versions: 1.1.1
> Reporter: Sandy Ryza
> Assignee: Sandy Ryza
> Fix For: 1.3.0
>
> Attachments: MAPREDUCE-5038-1.patch, MAPREDUCE-5038.patch,
> MAPREDUCE-5038-revised-1.patch, MAPREDUCE-5038-revised-1.patch,
> MAPREDUCE-5038-revised.patch
>
>
> The following changes patched the CombineFileInputFormat in mapreduce, but
> neglected the one in mapred
> MAPREDUCE-1597 enabled the CombineFileInputFormat to work on splittable files
> MAPREDUCE-2021 solved returning duplicate hostnames in split locations
> MAPREDUCE-1806 CombineFileInputFormat does not work with paths not on default
> FS
> In trunk this is not an issue as the one in mapred extends the one in
> mapreduce.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira