You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-user@hadoop.apache.org by Steve Lewis <lo...@gmail.com> on 2011/09/12 04:27:40 UTC

Does anyone have sample code for forcing a custom InputFormat to use a small split

I have a problem where there is a single, relatively small (10-20 MB) input
file. (It happens it is a fasta file which will have meaning if you are a
biologist.)  I am already using a custom  InputFormat  and a custom reader
to force a custom parsing. The file may generate tens or hundreds of
millions of key value pairs and the mapper does a fair amount of work on
each record.
The standard implementation of
 * public List<InputSplit> getSplits(JobContext job) throws IOException {*

uses fs.getFileBlockLocations(file, 0, length); to determine the blocks and
for a file of this size will come up with a single InputSplit and a single
mapper.

I am looking for a good example of forcing the generation of multiple
InputSplits for a small file. In this case I am  happy if every Mapper
instance is required to read and parse the entire file    as long as I can
guarantee that every record is processed by only a single mapper.
While I think I see how I might modify*  getSplits(JobContext job)  *I am
not sure how and when the code is called when the job is running on the
cluster.

-- 
Steven M. Lewis PhD
4221 105th Ave NE
Kirkland, WA 98033
206-384-1340 (cell)
Skype lordjoe_com

Re: Does anyone have sample code for forcing a custom InputFormat to use a small split

Posted by Harsh J <ha...@cloudera.com>.
Steve,

I second using Bz2 if you are able to. For files that are small, the
slow decompression speeds should not matter as much so its a good
tradeoff to have splits.

Regarding your specific question of:

>> In the case of a small file with enough work in the mapper it may be
>> useful to split even a zipped file -
>> even if it means reading from the beginning to reach a specific index in
>> the unzipped stream -
>> ever seen that done??

Yep, I've been following Niels Basjes' work on a similar thing around
this (with gzip). You can see it in action here:
https://issues.apache.org/jira/browse/HADOOP-7076 (It isn't gone into
trunk yet, though). Feel free to add your comments into that issue, or
for his patches!

On Tue, Sep 13, 2011 at 10:25 AM, Sudharsan Sampath <su...@gmail.com> wrote:
> Hi,
> Which version of Hadoop are u using. With v0.21 hadoop supports split bzip2
> compressed files(HADOOP-4012). So you dont even have to read from beginning
> to end.
> This patch is also available in cdh3 distribution which I would recommend as
> 0.21 is not declared suitable for production.
> Also the following link summarizes the comparison of diff compression
> formats.
> http://tukaani.org/lzma/benchmarks.html
> Thanks
> Sudhan S
> On Tue, Sep 13, 2011 at 6:41 AM, Steve Lewis <lo...@gmail.com> wrote:
>>
>> Thanks - what NLineInputFormat is pretty close to what I want.
>> In most cases the file is text and quite splittable
>>  although it raises another issue - sometimes the file is compressed -
>> even though it may
>> only be tens of megs compression is useful to speed transport
>> In the case of a small file with enough work in the mapper it may be
>> useful to split even a zipped file -
>> even if it means reading from the beginning to reach a specific index in
>> the unzipped stream -
>> ever seen that done??
>>
>> On Mon, Sep 12, 2011 at 1:36 AM, Harsh J <ha...@cloudera.com> wrote:
>>>
>>> Hello Steve,
>>>
>>> On Mon, Sep 12, 2011 at 7:57 AM, Steve Lewis <lo...@gmail.com>
>>> wrote:
>>> > I have a problem where there is a single, relatively small (10-20 MB)
>>> > input
>>> > file. (It happens it is a fasta file which will have meaning if you are
>>> > a
>>> > biologist.)  I am already using a custom  InputFormat  and a custom
>>> > reader
>>> > to force a custom parsing. The file may generate tens or hundreds of
>>> > millions of key value pairs and the mapper does a fair amount of work
>>> > on
>>> > each record.
>>> > The standard implementation of
>>> >   public List<InputSplit> getSplits(JobContext job) throws IOException
>>> > {
>>> >
>>> > uses fs.getFileBlockLocations(file, 0, length); to determine the blocks
>>> > and
>>> > for a file of this size will come up with a single InputSplit and a
>>> > single
>>> > mapper.
>>> > I am looking for a good example of forcing the generation of multiple
>>> > InputSplits for a small file. In this case I am  happy if every Mapper
>>> > instance is required to read and parse the entire file    as long as I
>>> > can
>>> > guarantee that every record is processed by only a single mapper.
>>>
>>> Is the file splittable?
>>>
>>> You may look at the FileInputFormat's "mapred.min.split.size"
>>> property. See
>>> http://hadoop.apache.org/common/docs/r0.20.2/api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html#setMinInputSplitSize(org.apache.hadoop.mapreduce.Job,
>>> long)
>>>
>>> Perhaps the 'NLineInputFormat' may also be what you're really looking
>>> for, which lets you limit no. of records per mapper instead of
>>> fiddling around with byte sizes with the above.
>>>
>>> > While I think I see how I might modify  getSplits(JobContext job)  I am
>>> > not
>>> > sure how and when the code is called when the job is running on the
>>> > cluster.
>>>
>>> The method is called in the client-end, at the job-submission point.
>>>
>>> --
>>> Harsh J
>>
>>
>>
>> --
>> Steven M. Lewis PhD
>> 4221 105th Ave NE
>> Kirkland, WA 98033
>> 206-384-1340 (cell)
>> Skype lordjoe_com
>>
>>
>
>



-- 
Harsh J

Re: Does anyone have sample code for forcing a custom InputFormat to use a small split

Posted by Sudharsan Sampath <su...@gmail.com>.
Hi,

Which version of Hadoop are u using. With v0.21 hadoop supports split bzip2
compressed files(HADOOP-4012). So you dont even have to read from beginning
to end.

This patch is also available in cdh3 distribution which I would recommend as
0.21 is not declared suitable for production.

Also the following link summarizes the comparison of diff compression
formats.

http://tukaani.org/lzma/benchmarks.html

Thanks
Sudhan S

On Tue, Sep 13, 2011 at 6:41 AM, Steve Lewis <lo...@gmail.com> wrote:

> Thanks - what NLineInputFormat is pretty close to what I want.
> In most cases the file is text and quite splittable
>  although it raises another issue - sometimes the file is compressed - even
> though it may
> only be tens of megs compression is useful to speed transport
> In the case of a small file with enough work in the mapper it may be useful
> to split even a zipped file -
> even if it means reading from the beginning to reach a specific index in
> the unzipped stream -
> ever seen that done??
>
>
> On Mon, Sep 12, 2011 at 1:36 AM, Harsh J <ha...@cloudera.com> wrote:
>
>> Hello Steve,
>>
>> On Mon, Sep 12, 2011 at 7:57 AM, Steve Lewis <lo...@gmail.com>
>> wrote:
>> > I have a problem where there is a single, relatively small (10-20 MB)
>> input
>> > file. (It happens it is a fasta file which will have meaning if you are
>> a
>> > biologist.)  I am already using a custom  InputFormat  and a custom
>> reader
>> > to force a custom parsing. The file may generate tens or hundreds of
>> > millions of key value pairs and the mapper does a fair amount of work on
>> > each record.
>> > The standard implementation of
>> >   public List<InputSplit> getSplits(JobContext job) throws IOException {
>> >
>> > uses fs.getFileBlockLocations(file, 0, length); to determine the blocks
>> and
>> > for a file of this size will come up with a single InputSplit and a
>> single
>> > mapper.
>> > I am looking for a good example of forcing the generation of multiple
>> > InputSplits for a small file. In this case I am  happy if every Mapper
>> > instance is required to read and parse the entire file    as long as I
>> can
>> > guarantee that every record is processed by only a single mapper.
>>
>> Is the file splittable?
>>
>> You may look at the FileInputFormat's "mapred.min.split.size"
>> property. See
>> http://hadoop.apache.org/common/docs/r0.20.2/api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html#setMinInputSplitSize(org.apache.hadoop.mapreduce.Job
>> ,
>> long)
>>
>> Perhaps the 'NLineInputFormat' may also be what you're really looking
>> for, which lets you limit no. of records per mapper instead of
>> fiddling around with byte sizes with the above.
>>
>> > While I think I see how I might modify  getSplits(JobContext job)  I am
>> not
>> > sure how and when the code is called when the job is running on the
>> cluster.
>>
>> The method is called in the client-end, at the job-submission point.
>>
>> --
>> Harsh J
>>
>
>
>
> --
> Steven M. Lewis PhD
> 4221 105th Ave NE
> Kirkland, WA 98033
> 206-384-1340 (cell)
> Skype lordjoe_com
>
>
>

Re: Does anyone have sample code for forcing a custom InputFormat to use a small split

Posted by Steve Lewis <lo...@gmail.com>.
Thanks - what NLineInputFormat is pretty close to what I want.
In most cases the file is text and quite splittable
 although it raises another issue - sometimes the file is compressed - even
though it may
only be tens of megs compression is useful to speed transport
In the case of a small file with enough work in the mapper it may be useful
to split even a zipped file -
even if it means reading from the beginning to reach a specific index in the
unzipped stream -
ever seen that done??


On Mon, Sep 12, 2011 at 1:36 AM, Harsh J <ha...@cloudera.com> wrote:

> Hello Steve,
>
> On Mon, Sep 12, 2011 at 7:57 AM, Steve Lewis <lo...@gmail.com>
> wrote:
> > I have a problem where there is a single, relatively small (10-20 MB)
> input
> > file. (It happens it is a fasta file which will have meaning if you are a
> > biologist.)  I am already using a custom  InputFormat  and a custom
> reader
> > to force a custom parsing. The file may generate tens or hundreds of
> > millions of key value pairs and the mapper does a fair amount of work on
> > each record.
> > The standard implementation of
> >   public List<InputSplit> getSplits(JobContext job) throws IOException {
> >
> > uses fs.getFileBlockLocations(file, 0, length); to determine the blocks
> and
> > for a file of this size will come up with a single InputSplit and a
> single
> > mapper.
> > I am looking for a good example of forcing the generation of multiple
> > InputSplits for a small file. In this case I am  happy if every Mapper
> > instance is required to read and parse the entire file    as long as I
> can
> > guarantee that every record is processed by only a single mapper.
>
> Is the file splittable?
>
> You may look at the FileInputFormat's "mapred.min.split.size"
> property. See
> http://hadoop.apache.org/common/docs/r0.20.2/api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html#setMinInputSplitSize(org.apache.hadoop.mapreduce.Job
> ,
> long)
>
> Perhaps the 'NLineInputFormat' may also be what you're really looking
> for, which lets you limit no. of records per mapper instead of
> fiddling around with byte sizes with the above.
>
> > While I think I see how I might modify  getSplits(JobContext job)  I am
> not
> > sure how and when the code is called when the job is running on the
> cluster.
>
> The method is called in the client-end, at the job-submission point.
>
> --
> Harsh J
>



-- 
Steven M. Lewis PhD
4221 105th Ave NE
Kirkland, WA 98033
206-384-1340 (cell)
Skype lordjoe_com

Re: Does anyone have sample code for forcing a custom InputFormat to use a small split

Posted by Harsh J <ha...@cloudera.com>.
Hello Steve,

On Mon, Sep 12, 2011 at 7:57 AM, Steve Lewis <lo...@gmail.com> wrote:
> I have a problem where there is a single, relatively small (10-20 MB) input
> file. (It happens it is a fasta file which will have meaning if you are a
> biologist.)  I am already using a custom  InputFormat  and a custom reader
> to force a custom parsing. The file may generate tens or hundreds of
> millions of key value pairs and the mapper does a fair amount of work on
> each record.
> The standard implementation of
>   public List<InputSplit> getSplits(JobContext job) throws IOException {
>
> uses fs.getFileBlockLocations(file, 0, length); to determine the blocks and
> for a file of this size will come up with a single InputSplit and a single
> mapper.
> I am looking for a good example of forcing the generation of multiple
> InputSplits for a small file. In this case I am  happy if every Mapper
> instance is required to read and parse the entire file    as long as I can
> guarantee that every record is processed by only a single mapper.

Is the file splittable?

You may look at the FileInputFormat's "mapred.min.split.size"
property. See http://hadoop.apache.org/common/docs/r0.20.2/api/org/apache/hadoop/mapreduce/lib/input/FileInputFormat.html#setMinInputSplitSize(org.apache.hadoop.mapreduce.Job,
long)

Perhaps the 'NLineInputFormat' may also be what you're really looking
for, which lets you limit no. of records per mapper instead of
fiddling around with byte sizes with the above.

> While I think I see how I might modify  getSplits(JobContext job)  I am not
> sure how and when the code is called when the job is running on the cluster.

The method is called in the client-end, at the job-submission point.

-- 
Harsh J