You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Niels Basjes (JIRA)" <ji...@apache.org> on 2014/07/27 13:17:39 UTC

[jira] [Commented] (MAPREDUCE-2094) org.apache.hadoop.mapreduce.lib.input.FileInputFormat: isSplitable implements unsafe default behaviour that is different from the documented behaviour.

    [ https://issues.apache.org/jira/browse/MAPREDUCE-2094?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14075606#comment-14075606 ] 

Niels Basjes commented on MAPREDUCE-2094:
-----------------------------------------

Question for [~gian], [~ggoodson] and others who have faced this problem.
Did you use the 'out of the box' LineRecordReader (or a subclass of that) as shown in https://developer.yahoo.com/hadoop/tutorial/module5.html#inputformat or did you write something 'completely new'?

When I ran into this I followed the tutorial.

I think the 'next best' spot to stop most problem scenarios (i.e. everyone who implements it like the tutorial shows) can be caught by letting the LineRecordReader fail the entire job when it is initialized with non splittable codec file and the provided split is not the entire file. 

What do you think?

> org.apache.hadoop.mapreduce.lib.input.FileInputFormat: isSplitable implements unsafe default behaviour that is different from the documented behaviour.
> -------------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: MAPREDUCE-2094
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-2094
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: task
>            Reporter: Niels Basjes
>            Assignee: Niels Basjes
>         Attachments: MAPREDUCE-2094-2011-05-19.patch, MAPREDUCE-2094-FileInputFormat-docs-v2.patch
>
>
> When implementing a custom derivative of FileInputFormat we ran into the effect that a large Gzipped input file would be processed several times. 
> A near 1GiB file would be processed around 36 times in its entirety. Thus producing garbage results and taking up a lot more CPU time than needed.
> It took a while to figure out and what we found is that the default implementation of the isSplittable method in [org.apache.hadoop.mapreduce.lib.input.FileInputFormat | http://svn.apache.org/viewvc/hadoop/mapreduce/trunk/src/java/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.java?view=markup ] is simply "return true;". 
> This is a very unsafe default and is in contradiction with the JavaDoc of the method which states: "Is the given filename splitable? Usually, true, but if the file is stream compressed, it will not be. " . The actual implementation effectively does "Is the given filename splitable? Always true, even if the file is stream compressed using an unsplittable compression codec. "
> For our situation (where we always have Gzipped input) we took the easy way out and simply implemented an isSplittable in our class that does "return false; "
> Now there are essentially 3 ways I can think of for fixing this (in order of what I would find preferable):
> # Implement something that looks at the used compression of the file (i.e. do migrate the implementation from TextInputFormat to FileInputFormat). This would make the method do what the JavaDoc describes.
> # "Force" developers to think about it and make this method abstract.
> # Use a "safe" default (i.e. return false)



--
This message was sent by Atlassian JIRA
(v6.2#6252)