You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-user@hadoop.apache.org by Jason Curtes <ja...@gmail.com> on 2008/03/29 03:34:08 UTC

small sized files - how to use MultiInputFileFormat

Hello,

I have been trying to run Hadoop on a set of small text files, not larger
than 10k each. The total input size is 15MB. If I try to run the example
word count application, it takes about 2000 seconds, more than half an hour
to complete. However, if I merge all the files into one large file, it takes
much less than a minute. I think using MultiInputFileFormat can be helpful
at this point. However, the API documentation is not really helpful. I
wonder if MultiInputFileFormat can really solve my problem, and if so, can
you suggest me a reference on how to use it, or a few lines to be added to
the word count example to make things more clear?

Thanks in advance.

Regards,

Jason Curtes

Re: small sized files - how to use MultiInputFileFormat

Posted by Enis Soztutar <en...@gmail.com>.
Hi,

An example extracting one record per file would be :

public class FooInputFormat extends MultiFileInputFormat {

   @Override
   public RecordReader getRecordReader(InputSplit split, JobConf job, 
Reporter reporter) throws IOException {
     return new FooRecordReader(job, (MultiFileSplit)split);
   }
 }


public static class FooRecordReader implements RecordReader {

   private MultiFileSplit split;
   private long offset;
   private long totLength;
   private FileSystem fs;
   private int count = 0;
   private Path[] paths;
     public FooRecordReader(Configuration conf, MultiFileSplit split)
   throws IOException {
     this.split = split;
     fs = FileSystem.get(conf);
     this.paths = split.getPaths();
     this.totLength = split.getLength();
     this.offset = 0;
   }

   public WritableComparable createKey() {
     ..
   }

   public Writable createValue() {
     ..
   }

   public void close() throws IOException { }

   public long getPos() throws IOException {
     return offset;
   }

   public float getProgress() throws IOException {
     return ((float)offset) / split.getLength();
   }

   public boolean next(Writable key, Writable value) throws IOException {
     if(offset >= totLength)
       return false;
     if(count >= split.numPaths())
       return false;
   
     Path file = paths[count];
     FSDataInputStream stream = fs.open(file);
     BufferedReader reader = new BufferedReader(new 
InputStreamReader(stream));
     Scanner scanner = new Scanner(reader.readLine());
           //read from file, fill in key and value
        reader.close();
     stream.close();
     offset += split.getLength(count);
     count++;
     return true;
   }
 }


I guess, I should add an example code to the mapred tutorial, and 
examples directory.

Jason Curtes wrote:
> Hello,
>
> I have been trying to run Hadoop on a set of small text files, not larger
> than 10k each. The total input size is 15MB. If I try to run the example
> word count application, it takes about 2000 seconds, more than half an hour
> to complete. However, if I merge all the files into one large file, it takes
> much less than a minute. I think using MultiInputFileFormat can be helpful
> at this point. However, the API documentation is not really helpful. I
> wonder if MultiInputFileFormat can really solve my problem, and if so, can
> you suggest me a reference on how to use it, or a few lines to be added to
> the word count example to make things more clear?
>
> Thanks in advance.
>
> Regards,
>
> Jason Curtes
>
>   

Re: small sized files - how to use MultiInputFileFormat

Posted by Alejandro Abdelnur <tu...@gmail.com>.
In principle I agree with you Ted.

However, in many cases we have multiple large jobs generating outputs
that are not that big and as result the number of small size files
(significantly smaller than a Hadoop block) is large, using the
default splitting logic there triggers jobs with a large amount of
tasks that inefficiently clogs the cluster.

The MultipleFileInputFormat helps to avoid that, but it has a problem,
if the file set is a mix of small and large files the splits are
uneven and it does not do split on single large files.

To address this we've written our own InputFormat (for Text and
SequenceFiles) that collapses small files into a splits up to the
block size and splits big files into the block size.

It has a  twist that you can you specify the max number of MAPs that
you want or the BLOCK size you want to use for the splits.

When a particular split contains multiple small files, the suggested
host for the splits is order based on the host that has most of the
data for those files.

We'll still have to do some clean up on the code and then we'll submit
it to Hadoop.

A

On Sat, Mar 29, 2008 at 10:20 PM, Ted Dunning <td...@veoh.com> wrote:
>
>  Small files are a bad idea for high throughput no matter what technology you
>  use.  The issue is that you need a larger file in order to avoid disk seeks.
>
>
>
>
>  On 3/28/08 7:34 PM, "Jason Curtes" <ja...@gmail.com> wrote:
>
>  > Hello,
>  >
>  > I have been trying to run Hadoop on a set of small text files, not larger
>  > than 10k each. The total input size is 15MB. If I try to run the example
>  > word count application, it takes about 2000 seconds, more than half an hour
>  > to complete. However, if I merge all the files into one large file, it takes
>  > much less than a minute. I think using MultiInputFileFormat can be helpful
>  > at this point. However, the API documentation is not really helpful. I
>  > wonder if MultiInputFileFormat can really solve my problem, and if so, can
>  > you suggest me a reference on how to use it, or a few lines to be added to
>  > the word count example to make things more clear?
>  >
>  > Thanks in advance.
>  >
>  > Regards,
>  >
>  > Jason Curtes
>
>

Re: small sized files - how to use MultiInputFileFormat

Posted by Ted Dunning <td...@veoh.com>.
Small files are a bad idea for high throughput no matter what technology you
use.  The issue is that you need a larger file in order to avoid disk seeks.


On 3/28/08 7:34 PM, "Jason Curtes" <ja...@gmail.com> wrote:

> Hello,
> 
> I have been trying to run Hadoop on a set of small text files, not larger
> than 10k each. The total input size is 15MB. If I try to run the example
> word count application, it takes about 2000 seconds, more than half an hour
> to complete. However, if I merge all the files into one large file, it takes
> much less than a minute. I think using MultiInputFileFormat can be helpful
> at this point. However, the API documentation is not really helpful. I
> wonder if MultiInputFileFormat can really solve my problem, and if so, can
> you suggest me a reference on how to use it, or a few lines to be added to
> the word count example to make things more clear?
> 
> Thanks in advance.
> 
> Regards,
> 
> Jason Curtes