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 Sasha Dolgy <sa...@gmail.com> on 2009/05/06 01:34:00 UTC

large files vs many files

hi there,
working through a concept at the moment and was attempting to write lots of
data to few files as opposed to writing lots of data to lots of little
files.  what are the thoughts on this?

When I try and implement outputStream = hdfs.append(path); there doesn't
seem to be any locking mechanism in place ... or there is and it doesn't
work well enough for many writes per second?

i have read and seen that the property "dfs.support.append" is not meant for
production use.  still, if millions of little files are as good or better
--- or no difference -- to a few massive files then i suppose append isn't
something i really need.

I do see a lot of stack traces with messages like:

org.apache.hadoop.ipc.RemoteException:
org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to
create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on client
127.0.0.1 because current leaseholder is trying to recreate file.

i hope this make sense.  still a little bit confused.

thanks in advance
-sd

-- 
Sasha Dolgy
sasha.dolgy@gmail.com

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
Hi Tom,
Thanks for this.  I'll follow that up and see how I get on.  At issue is the
frequency of the data I have streaming in.  Even if I create a new file with
a name based on milliseconds I'm still running into the same problems.  My
thought is that using append, although it's not production ready, it's also
not the root of my problems.

cheers
-sd

On Wed, May 6, 2009 at 9:40 AM, Tom White <to...@cloudera.com> wrote:

> Hi Sasha,
>
> As you say, HDFS appends are not yet working reliably enough to be
> suitable for production use. On the other hand, having lots of little
> files is bad for the namenode, and inefficient for MapReduce (see
> http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/), so
> it's best to avoid this too.
>
> I would recommend using SequenceFile as a storage container for lots
> of small pieces of data. Each key-value pair would represent one of
> your little files (you can have a null key, if you only need to store
> the contents of the file). You can also enable compression (use block
> compression), and SequenceFiles are designed to work well with
> MapReduce.
>
> Cheers,
>
> Tom
>
> On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com>
> wrote:
> > hi there,
> > working through a concept at the moment and was attempting to write lots
> of
> > data to few files as opposed to writing lots of data to lots of little
> > files.  what are the thoughts on this?
> >
> > When I try and implement outputStream = hdfs.append(path); there doesn't
> > seem to be any locking mechanism in place ... or there is and it doesn't
> > work well enough for many writes per second?
> >
> > i have read and seen that the property "dfs.support.append" is not meant
> for
> > production use.  still, if millions of little files are as good or better
> > --- or no difference -- to a few massive files then i suppose append
> isn't
> > something i really need.
> >
> > I do see a lot of stack traces with messages like:
> >
> > org.apache.hadoop.ipc.RemoteException:
> > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to
> > create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on
> client
> > 127.0.0.1 because current leaseholder is trying to recreate file.
> >
> > i hope this make sense.  still a little bit confused.
> >
> > thanks in advance
> > -sd
> >
> > --
> > Sasha Dolgy
> > sasha.dolgy@gmail.com
>

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
2009-05-12 12:42:17,470 DEBUG [Thread-7] (FSStreamManager.java:28)
hdfs.HdfsQueueConsumer: Thread 19 getting an output stream
2009-05-12 12:42:17,470 DEBUG [Thread-7] (FSStreamManager.java:49)
hdfs.HdfsQueueConsumer: Re-using existing stream
2009-05-12 12:42:17,472 DEBUG [Thread-7] (FSStreamManager.java:63)
hdfs.HdfsQueueConsumer: Flushing stream, size = 1986
2009-05-12 12:42:17,472 DEBUG [Thread-7] (DFSClient.java:3013)
hdfs.DFSClient: DFSClient flush() : saveOffset 1613 bytesCurBlock 1986
lastFlushOffset 1731
2009-05-12 12:42:17,472 DEBUG [Thread-7] (FSStreamManager.java:66)
hdfs.HdfsQueueConsumer: Flushed stream, size = 1986
2009-05-12 12:42:19,586 DEBUG [Thread-7] (HdfsQueueConsumer.java:39)
hdfs.HdfsQueueConsumer: Consumer writing event
2009-05-12 12:42:19,587 DEBUG [Thread-7] (FSStreamManager.java:28)
hdfs.HdfsQueueConsumer: Thread 19 getting an output stream
2009-05-12 12:42:19,588 DEBUG [Thread-7] (FSStreamManager.java:49)
hdfs.HdfsQueueConsumer: Re-using existing stream
2009-05-12 12:42:19,589 DEBUG [Thread-7] (FSStreamManager.java:63)
hdfs.HdfsQueueConsumer: Flushing stream, size = 2235
2009-05-12 12:42:19,589 DEBUG [Thread-7] (DFSClient.java:3013)
hdfs.DFSClient: DFSClient flush() : saveOffset 2125 bytesCurBlock 2235
lastFlushOffset 1986
2009-05-12 12:42:19,590 DEBUG [Thread-7] (FSStreamManager.java:66)
hdfs.HdfsQueueConsumer: Flushed stream, size = 2235

So although the Offset is changing as expected, the output stream isn't
being flushed or cleared out and isn't being written to file...

On Tue, May 12, 2009 at 5:26 PM, Sasha Dolgy <sd...@gmail.com> wrote:

> Right now data is received in parallel and is written to a queue, then a
> single thread reads the queue and writes those messages to a
> FSDataOutputStream which is kept open, but the messages never get flushed.
>  Tried flush() and sync() with no joy.
> 1.
> outputStream.writeBytes(rawMessage.toString());
>
> 2.
>
> log.debug("Flushing stream, size = " + s.getOutputStream().size());
>  s.getOutputStream().sync();
> log.debug("Flushed stream, size = " + s.getOutputStream().size());
>
> or
>
> log.debug("Flushing stream, size = " + s.getOutputStream().size());
> s.getOutputStream().flush();
>  log.debug("Flushed stream, size = " + s.getOutputStream().size());
>
> Just see the size() remain the same after performing this action.
>
> This is using hadoop-0.20.0.
>
> -sd
>
>
> On Sun, May 10, 2009 at 4:45 PM, Stefan Podkowinski <sp...@gmail.com>wrote:
>
>> You just can't have many distributed jobs write into the same file
>> without locking/synchronizing these writes. Even with append(). Its
>> not different than using a regular file from multiple processes in
>> this respect.
>> Maybe you need to collect your data in front before processing them in
>> hadoop?
>> Have a look at Chukwa, http://wiki.apache.org/hadoop/Chukwa
>>
>>
>> On Sat, May 9, 2009 at 9:44 AM, Sasha Dolgy <sd...@gmail.com> wrote:
>> > Would WritableFactories not allow me to open one outputstream and
>> continue
>> > to write() and sync() ?
>> >
>> > Maybe I'm reading into that wrong.  Although UUID would be nice, it
>> would
>> > still leave me in the problem of having lots of little files instead of
>> a
>> > few large files.
>> >
>> > -sd
>> >
>> > On Sat, May 9, 2009 at 8:37 AM, jason hadoop <ja...@gmail.com>
>> wrote:
>> >
>> >> You must create unique file names, I don't believe (but I do not know)
>> that
>> >> the append could will allow multiple writers.
>> >>
>> >> Are you writing from within a task, or as an external application
>> writing
>> >> into hadoop.
>> >>
>> >> You may try using UUID,
>> >> http://java.sun.com/j2se/1.5.0/docs/api/java/util/UUID.html, as part
>> of
>> >> your
>> >> filename.
>> >> Without knowing more about your goals, environment and constraints it
>> is
>> >> hard to offer any more detailed suggestions.
>> >> You could also have an application aggregate the streams and write out
>> >> chunks, with one or more writers, one per output file.
>>
>


-- 
Sasha Dolgy
sasha.dolgy@gmail.com

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
Right now data is received in parallel and is written to a queue, then a
single thread reads the queue and writes those messages to a
FSDataOutputStream which is kept open, but the messages never get flushed.
 Tried flush() and sync() with no joy.
1.
outputStream.writeBytes(rawMessage.toString());

2.

log.debug("Flushing stream, size = " + s.getOutputStream().size());
s.getOutputStream().sync();
log.debug("Flushed stream, size = " + s.getOutputStream().size());

or

log.debug("Flushing stream, size = " + s.getOutputStream().size());
s.getOutputStream().flush();
log.debug("Flushed stream, size = " + s.getOutputStream().size());

Just see the size() remain the same after performing this action.

This is using hadoop-0.20.0.

-sd

On Sun, May 10, 2009 at 4:45 PM, Stefan Podkowinski <sp...@gmail.com>wrote:

> You just can't have many distributed jobs write into the same file
> without locking/synchronizing these writes. Even with append(). Its
> not different than using a regular file from multiple processes in
> this respect.
> Maybe you need to collect your data in front before processing them in
> hadoop?
> Have a look at Chukwa, http://wiki.apache.org/hadoop/Chukwa
>
>
> On Sat, May 9, 2009 at 9:44 AM, Sasha Dolgy <sd...@gmail.com> wrote:
> > Would WritableFactories not allow me to open one outputstream and
> continue
> > to write() and sync() ?
> >
> > Maybe I'm reading into that wrong.  Although UUID would be nice, it would
> > still leave me in the problem of having lots of little files instead of a
> > few large files.
> >
> > -sd
> >
> > On Sat, May 9, 2009 at 8:37 AM, jason hadoop <ja...@gmail.com>
> wrote:
> >
> >> You must create unique file names, I don't believe (but I do not know)
> that
> >> the append could will allow multiple writers.
> >>
> >> Are you writing from within a task, or as an external application
> writing
> >> into hadoop.
> >>
> >> You may try using UUID,
> >> http://java.sun.com/j2se/1.5.0/docs/api/java/util/UUID.html, as part of
> >> your
> >> filename.
> >> Without knowing more about your goals, environment and constraints it is
> >> hard to offer any more detailed suggestions.
> >> You could also have an application aggregate the streams and write out
> >> chunks, with one or more writers, one per output file.
>

Re: large files vs many files

Posted by Stefan Podkowinski <sp...@gmail.com>.
You just can't have many distributed jobs write into the same file
without locking/synchronizing these writes. Even with append(). Its
not different than using a regular file from multiple processes in
this respect.
Maybe you need to collect your data in front before processing them in hadoop?
Have a look at Chukwa, http://wiki.apache.org/hadoop/Chukwa


On Sat, May 9, 2009 at 9:44 AM, Sasha Dolgy <sd...@gmail.com> wrote:
> Would WritableFactories not allow me to open one outputstream and continue
> to write() and sync() ?
>
> Maybe I'm reading into that wrong.  Although UUID would be nice, it would
> still leave me in the problem of having lots of little files instead of a
> few large files.
>
> -sd
>
> On Sat, May 9, 2009 at 8:37 AM, jason hadoop <ja...@gmail.com> wrote:
>
>> You must create unique file names, I don't believe (but I do not know) that
>> the append could will allow multiple writers.
>>
>> Are you writing from within a task, or as an external application writing
>> into hadoop.
>>
>> You may try using UUID,
>> http://java.sun.com/j2se/1.5.0/docs/api/java/util/UUID.html, as part of
>> your
>> filename.
>> Without knowing more about your goals, environment and constraints it is
>> hard to offer any more detailed suggestions.
>> You could also have an application aggregate the streams and write out
>> chunks, with one or more writers, one per output file.
>>
>>
>

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
Would WritableFactories not allow me to open one outputstream and continue
to write() and sync() ?

Maybe I'm reading into that wrong.  Although UUID would be nice, it would
still leave me in the problem of having lots of little files instead of a
few large files.

-sd

On Sat, May 9, 2009 at 8:37 AM, jason hadoop <ja...@gmail.com> wrote:

> You must create unique file names, I don't believe (but I do not know) that
> the append could will allow multiple writers.
>
> Are you writing from within a task, or as an external application writing
> into hadoop.
>
> You may try using UUID,
> http://java.sun.com/j2se/1.5.0/docs/api/java/util/UUID.html, as part of
> your
> filename.
> Without knowing more about your goals, environment and constraints it is
> hard to offer any more detailed suggestions.
> You could also have an application aggregate the streams and write out
> chunks, with one or more writers, one per output file.
>
>

Re: large files vs many files

Posted by jason hadoop <ja...@gmail.com>.
You must create unique file names, I don't believe (but I do not know) that
the append could will allow multiple writers.

Are you writing from within a task, or as an external application writing
into hadoop.

You may try using UUID,
http://java.sun.com/j2se/1.5.0/docs/api/java/util/UUID.html, as part of your
filename.
Without knowing more about your goals, environment and constraints it is
hard to offer any more detailed suggestions.
You could also have an application aggregate the streams and write out
chunks, with one or more writers, one per output file.


On Sat, May 9, 2009 at 12:15 AM, Sasha Dolgy <sd...@gmail.com> wrote:

> yes, that is the problem.  two or hundreds...data streams in very quickly.
>
> On Fri, May 8, 2009 at 8:42 PM, jason hadoop <ja...@gmail.com>
> wrote:
>
> > Is it possible that two tasks are trying to write to the same file path?
> >
> >
> > On Fri, May 8, 2009 at 11:46 AM, Sasha Dolgy <sd...@gmail.com> wrote:
> >
> > > Hi Tom (or anyone else),
> > > Will SequenceFile allow me to avoid problems with concurrent writes to
> > the
> > > file?  I stll continue to get the following exceptions/errors in hdfs:
> > >
> > > org.apache.hadoop.ipc.RemoteException:
> > > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException:
> > > failed to create file /foo/bar/aaa.bbb.ccc.ddd.xxx for
> > > DFSClient_-1821265528
> > > on client 127.0.0.1 because current leaseholder is trying to recreate
> > file.
> > >
> > > Only happens when two processes are trying to write at the same time.
> >  Now
> > > ideally I don't want to buffer the data that's coming in and i want to
> > get
> > > it out and into the file asap to avoid any data loss...am i missing
> > > something here?  is there some sort of factory i can implement to help
> in
> > > writing a lot of simultaneous data streams?
> > >
> > > thanks in advance for any suggestions
> > > -sasha
> > >
> > > On Wed, May 6, 2009 at 9:40 AM, Tom White <to...@cloudera.com> wrote:
> > >
> > > > Hi Sasha,
> > > >
> > > > As you say, HDFS appends are not yet working reliably enough to be
> > > > suitable for production use. On the other hand, having lots of little
> > > > files is bad for the namenode, and inefficient for MapReduce (see
> > > > http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/),
> so
> > > > it's best to avoid this too.
> > > >
> > > > I would recommend using SequenceFile as a storage container for lots
> > > > of small pieces of data. Each key-value pair would represent one of
> > > > your little files (you can have a null key, if you only need to store
> > > > the contents of the file). You can also enable compression (use block
> > > > compression), and SequenceFiles are designed to work well with
> > > > MapReduce.
> > > >
> > > > Cheers,
> > > >
> > > > Tom
> > > >
> > > > On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com>
> > > > wrote:
> > > > > hi there,
> > > > > working through a concept at the moment and was attempting to write
> > > lots
> > > > of
> > > > > data to few files as opposed to writing lots of data to lots of
> > little
> > > > > files.  what are the thoughts on this?
> > > > >
> > > > > When I try and implement outputStream = hdfs.append(path); there
> > > doesn't
> > > > > seem to be any locking mechanism in place ... or there is and it
> > > doesn't
> > > > > work well enough for many writes per second?
> > > > >
> > > > > i have read and seen that the property "dfs.support.append" is not
> > > meant
> > > > for
> > > > > production use.  still, if millions of little files are as good or
> > > better
> > > > > --- or no difference -- to a few massive files then i suppose
> append
> > > > isn't
> > > > > something i really need.
> > > > >
> > > > > I do see a lot of stack traces with messages like:
> > > > >
> > > > > org.apache.hadoop.ipc.RemoteException:
> > > > > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException:
> failed
> > to
> > > > > create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528
> on
> > > > client
> > > > > 127.0.0.1 because current leaseholder is trying to recreate file.
> > > > >
> > > > > i hope this make sense.  still a little bit confused.
> > > > >
> > > > > thanks in advance
> > > > > -sd
> > > > >
> > > > > --
> > > > > Sasha Dolgy
> > > > > sasha.dolgy@gmail.com
> > > >
> > >
> >
> >
> >
> > --
> > Alpha Chapters of my book on Hadoop are available
> > http://www.apress.com/book/view/9781430219422
> > www.prohadoopbook.com a community for Hadoop Professionals
> >
>
>
>
> --
> Sasha Dolgy
> sasha.dolgy@gmail.com
>



-- 
Alpha Chapters of my book on Hadoop are available
http://www.apress.com/book/view/9781430219422
www.prohadoopbook.com a community for Hadoop Professionals

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
yes, that is the problem.  two or hundreds...data streams in very quickly.

On Fri, May 8, 2009 at 8:42 PM, jason hadoop <ja...@gmail.com> wrote:

> Is it possible that two tasks are trying to write to the same file path?
>
>
> On Fri, May 8, 2009 at 11:46 AM, Sasha Dolgy <sd...@gmail.com> wrote:
>
> > Hi Tom (or anyone else),
> > Will SequenceFile allow me to avoid problems with concurrent writes to
> the
> > file?  I stll continue to get the following exceptions/errors in hdfs:
> >
> > org.apache.hadoop.ipc.RemoteException:
> > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException:
> > failed to create file /foo/bar/aaa.bbb.ccc.ddd.xxx for
> > DFSClient_-1821265528
> > on client 127.0.0.1 because current leaseholder is trying to recreate
> file.
> >
> > Only happens when two processes are trying to write at the same time.
>  Now
> > ideally I don't want to buffer the data that's coming in and i want to
> get
> > it out and into the file asap to avoid any data loss...am i missing
> > something here?  is there some sort of factory i can implement to help in
> > writing a lot of simultaneous data streams?
> >
> > thanks in advance for any suggestions
> > -sasha
> >
> > On Wed, May 6, 2009 at 9:40 AM, Tom White <to...@cloudera.com> wrote:
> >
> > > Hi Sasha,
> > >
> > > As you say, HDFS appends are not yet working reliably enough to be
> > > suitable for production use. On the other hand, having lots of little
> > > files is bad for the namenode, and inefficient for MapReduce (see
> > > http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/), so
> > > it's best to avoid this too.
> > >
> > > I would recommend using SequenceFile as a storage container for lots
> > > of small pieces of data. Each key-value pair would represent one of
> > > your little files (you can have a null key, if you only need to store
> > > the contents of the file). You can also enable compression (use block
> > > compression), and SequenceFiles are designed to work well with
> > > MapReduce.
> > >
> > > Cheers,
> > >
> > > Tom
> > >
> > > On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com>
> > > wrote:
> > > > hi there,
> > > > working through a concept at the moment and was attempting to write
> > lots
> > > of
> > > > data to few files as opposed to writing lots of data to lots of
> little
> > > > files.  what are the thoughts on this?
> > > >
> > > > When I try and implement outputStream = hdfs.append(path); there
> > doesn't
> > > > seem to be any locking mechanism in place ... or there is and it
> > doesn't
> > > > work well enough for many writes per second?
> > > >
> > > > i have read and seen that the property "dfs.support.append" is not
> > meant
> > > for
> > > > production use.  still, if millions of little files are as good or
> > better
> > > > --- or no difference -- to a few massive files then i suppose append
> > > isn't
> > > > something i really need.
> > > >
> > > > I do see a lot of stack traces with messages like:
> > > >
> > > > org.apache.hadoop.ipc.RemoteException:
> > > > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed
> to
> > > > create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on
> > > client
> > > > 127.0.0.1 because current leaseholder is trying to recreate file.
> > > >
> > > > i hope this make sense.  still a little bit confused.
> > > >
> > > > thanks in advance
> > > > -sd
> > > >
> > > > --
> > > > Sasha Dolgy
> > > > sasha.dolgy@gmail.com
> > >
> >
>
>
>
> --
> Alpha Chapters of my book on Hadoop are available
> http://www.apress.com/book/view/9781430219422
> www.prohadoopbook.com a community for Hadoop Professionals
>



-- 
Sasha Dolgy
sasha.dolgy@gmail.com

Re: large files vs many files

Posted by jason hadoop <ja...@gmail.com>.
Is it possible that two tasks are trying to write to the same file path?


On Fri, May 8, 2009 at 11:46 AM, Sasha Dolgy <sd...@gmail.com> wrote:

> Hi Tom (or anyone else),
> Will SequenceFile allow me to avoid problems with concurrent writes to the
> file?  I stll continue to get the following exceptions/errors in hdfs:
>
> org.apache.hadoop.ipc.RemoteException:
> org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException:
> failed to create file /foo/bar/aaa.bbb.ccc.ddd.xxx for
> DFSClient_-1821265528
> on client 127.0.0.1 because current leaseholder is trying to recreate file.
>
> Only happens when two processes are trying to write at the same time.  Now
> ideally I don't want to buffer the data that's coming in and i want to get
> it out and into the file asap to avoid any data loss...am i missing
> something here?  is there some sort of factory i can implement to help in
> writing a lot of simultaneous data streams?
>
> thanks in advance for any suggestions
> -sasha
>
> On Wed, May 6, 2009 at 9:40 AM, Tom White <to...@cloudera.com> wrote:
>
> > Hi Sasha,
> >
> > As you say, HDFS appends are not yet working reliably enough to be
> > suitable for production use. On the other hand, having lots of little
> > files is bad for the namenode, and inefficient for MapReduce (see
> > http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/), so
> > it's best to avoid this too.
> >
> > I would recommend using SequenceFile as a storage container for lots
> > of small pieces of data. Each key-value pair would represent one of
> > your little files (you can have a null key, if you only need to store
> > the contents of the file). You can also enable compression (use block
> > compression), and SequenceFiles are designed to work well with
> > MapReduce.
> >
> > Cheers,
> >
> > Tom
> >
> > On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com>
> > wrote:
> > > hi there,
> > > working through a concept at the moment and was attempting to write
> lots
> > of
> > > data to few files as opposed to writing lots of data to lots of little
> > > files.  what are the thoughts on this?
> > >
> > > When I try and implement outputStream = hdfs.append(path); there
> doesn't
> > > seem to be any locking mechanism in place ... or there is and it
> doesn't
> > > work well enough for many writes per second?
> > >
> > > i have read and seen that the property "dfs.support.append" is not
> meant
> > for
> > > production use.  still, if millions of little files are as good or
> better
> > > --- or no difference -- to a few massive files then i suppose append
> > isn't
> > > something i really need.
> > >
> > > I do see a lot of stack traces with messages like:
> > >
> > > org.apache.hadoop.ipc.RemoteException:
> > > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to
> > > create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on
> > client
> > > 127.0.0.1 because current leaseholder is trying to recreate file.
> > >
> > > i hope this make sense.  still a little bit confused.
> > >
> > > thanks in advance
> > > -sd
> > >
> > > --
> > > Sasha Dolgy
> > > sasha.dolgy@gmail.com
> >
>



-- 
Alpha Chapters of my book on Hadoop are available
http://www.apress.com/book/view/9781430219422
www.prohadoopbook.com a community for Hadoop Professionals

Re: large files vs many files

Posted by Sasha Dolgy <sd...@gmail.com>.
Hi Tom (or anyone else),
Will SequenceFile allow me to avoid problems with concurrent writes to the
file?  I stll continue to get the following exceptions/errors in hdfs:

org.apache.hadoop.ipc.RemoteException:
org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException:
failed to create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528
on client 127.0.0.1 because current leaseholder is trying to recreate file.

Only happens when two processes are trying to write at the same time.  Now
ideally I don't want to buffer the data that's coming in and i want to get
it out and into the file asap to avoid any data loss...am i missing
something here?  is there some sort of factory i can implement to help in
writing a lot of simultaneous data streams?

thanks in advance for any suggestions
-sasha

On Wed, May 6, 2009 at 9:40 AM, Tom White <to...@cloudera.com> wrote:

> Hi Sasha,
>
> As you say, HDFS appends are not yet working reliably enough to be
> suitable for production use. On the other hand, having lots of little
> files is bad for the namenode, and inefficient for MapReduce (see
> http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/), so
> it's best to avoid this too.
>
> I would recommend using SequenceFile as a storage container for lots
> of small pieces of data. Each key-value pair would represent one of
> your little files (you can have a null key, if you only need to store
> the contents of the file). You can also enable compression (use block
> compression), and SequenceFiles are designed to work well with
> MapReduce.
>
> Cheers,
>
> Tom
>
> On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com>
> wrote:
> > hi there,
> > working through a concept at the moment and was attempting to write lots
> of
> > data to few files as opposed to writing lots of data to lots of little
> > files.  what are the thoughts on this?
> >
> > When I try and implement outputStream = hdfs.append(path); there doesn't
> > seem to be any locking mechanism in place ... or there is and it doesn't
> > work well enough for many writes per second?
> >
> > i have read and seen that the property "dfs.support.append" is not meant
> for
> > production use.  still, if millions of little files are as good or better
> > --- or no difference -- to a few massive files then i suppose append
> isn't
> > something i really need.
> >
> > I do see a lot of stack traces with messages like:
> >
> > org.apache.hadoop.ipc.RemoteException:
> > org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to
> > create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on
> client
> > 127.0.0.1 because current leaseholder is trying to recreate file.
> >
> > i hope this make sense.  still a little bit confused.
> >
> > thanks in advance
> > -sd
> >
> > --
> > Sasha Dolgy
> > sasha.dolgy@gmail.com
>

Re: large files vs many files

Posted by Tom White <to...@cloudera.com>.
Hi Sasha,

As you say, HDFS appends are not yet working reliably enough to be
suitable for production use. On the other hand, having lots of little
files is bad for the namenode, and inefficient for MapReduce (see
http://www.cloudera.com/blog/2009/02/02/the-small-files-problem/), so
it's best to avoid this too.

I would recommend using SequenceFile as a storage container for lots
of small pieces of data. Each key-value pair would represent one of
your little files (you can have a null key, if you only need to store
the contents of the file). You can also enable compression (use block
compression), and SequenceFiles are designed to work well with
MapReduce.

Cheers,

Tom

On Wed, May 6, 2009 at 12:34 AM, Sasha Dolgy <sa...@gmail.com> wrote:
> hi there,
> working through a concept at the moment and was attempting to write lots of
> data to few files as opposed to writing lots of data to lots of little
> files.  what are the thoughts on this?
>
> When I try and implement outputStream = hdfs.append(path); there doesn't
> seem to be any locking mechanism in place ... or there is and it doesn't
> work well enough for many writes per second?
>
> i have read and seen that the property "dfs.support.append" is not meant for
> production use.  still, if millions of little files are as good or better
> --- or no difference -- to a few massive files then i suppose append isn't
> something i really need.
>
> I do see a lot of stack traces with messages like:
>
> org.apache.hadoop.ipc.RemoteException:
> org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException: failed to
> create file /foo/bar/aaa.bbb.ccc.ddd.xxx for DFSClient_-1821265528 on client
> 127.0.0.1 because current leaseholder is trying to recreate file.
>
> i hope this make sense.  still a little bit confused.
>
> thanks in advance
> -sd
>
> --
> Sasha Dolgy
> sasha.dolgy@gmail.com
>