Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Stas Oskin
Hi. Thank for the advice, just to clarify: The upgrade of you speak of of cleaning the pipes/epolls more often, is regarding the issue discussed (HADOOP-4346, fixed in my distribution), or it's some other issue? If yes, does it has a ticket I can see, or it should be filled to Jira? Thanks!

Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Raghu Angadi
Stas Oskin wrote: Hi. Any idea if calling System.gc() periodically will help reducing the amount of pipes / epolls? since you have HADOOP-4346, you should not have excessive epoll/pipe fds open. First of all do you still have the problem? If yes, how many hadoop streams do you have at a

Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Raghu Angadi
To be more accurate, once you have HADOOP-4346, fds for epoll and pipes = 3 * threads blocked on Hadoop I/O Unless you have hundreds of threads at a time, you should not see hundreds of these. These fds stay up to 10sec even after the threads exit. I am a bit confused about your exact

Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Stas Oskin
Hi. In my testings, I typically opened between 20 and 40 concurrent streams. Regards. 2009/6/23 Raghu Angadi rang...@yahoo-inc.com Stas Oskin wrote: Hi. Any idea if calling System.gc() periodically will help reducing the amount of pipes / epolls? since you have HADOOP-4346, you should

Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Raghu Angadi
how many threads do you have? Number of active threads is very important. Normally, #fds = (3 * #threads_blocked_on_io) + #streams 12 per stream is certainly way off. Raghu. Stas Oskin wrote: Hi. In my case it was actually ~ 12 fd's per stream, which included pipes and epolls. Could it

Re: Too many open files error, which gets resolved after some time

2009-06-23 Thread Stas Oskin
Hi. So if I open one stream, it should be 4? 2009/6/23 Raghu Angadi rang...@yahoo-inc.com how many threads do you have? Number of active threads is very important. Normally, #fds = (3 * #threads_blocked_on_io) + #streams 12 per stream is certainly way off. Raghu. Stas Oskin wrote:

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Raghu Angadi
Is this before 0.20.0? Assuming you have closed these streams, it is mostly https://issues.apache.org/jira/browse/HADOOP-4346 It is the JDK internal implementation that depends on GC to free up its cache of selectors. HADOOP-4346 avoids this by using hadoop's own cache. Raghu. Stas Oskin

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Stas Oskin
know when the finalizer will get called, if ever. -brian -Original Message- From: ext jason hadoop [mailto:jason.had...@gmail.com] Sent: Sunday, June 21, 2009 11:19 AM To: core-user@hadoop.apache.org Subject: Re: Too many open files error, which gets

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Steve Loughran
jason hadoop wrote: Yes. Otherwise the file descriptors will flow away like water. I also strongly suggest having at least 64k file descriptors as the open file limit. On Sun, Jun 21, 2009 at 12:43 PM, Stas Oskin stas.os...@gmail.com wrote: Hi. Thanks for the advice. So you advice explicitly

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Steve Loughran
Scott Carey wrote: Furthermore, if for some reason it is required to dispose of any objects after others are GC'd, weak references and a weak reference queue will perform significantly better in throughput and latency - orders of magnitude better - than finalizers. Good point. I would

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Steve Loughran
Raghu Angadi wrote: Is this before 0.20.0? Assuming you have closed these streams, it is mostly https://issues.apache.org/jira/browse/HADOOP-4346 It is the JDK internal implementation that depends on GC to free up its cache of selectors. HADOOP-4346 avoids this by using hadoop's own cache.

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Stas Oskin
Hi. So what would be the recommended approach to pre-0.20.x series? To insure each file is used only by one thread, and then it safe to close the handle in that thread? Regards. 2009/6/22 Steve Loughran ste...@apache.org Raghu Angadi wrote: Is this before 0.20.0? Assuming you have closed

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Raghu Angadi
64k might help in the sense, you might hit GC before you hit the limit. Otherwise, your only options are to use the patch attached to HADOOP-4346 or run System.gc() occasionally. I think it should be committed to 0.18.4 Raghu. Stas Oskin wrote: Hi. Yes, it happens with 0.18.3. I'm

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Stas Oskin
Hi Rahid. A question - this issue does not influence Hadoop itself (DataNodes, etc...), but rather influence any application using DFS, correct? If so, without patching iI should either to increase fd limit (which might fill-up as well???), or periodically launch the GC? Regards. 2009/6/22

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Steve Loughran
Stas Oskin wrote: Hi. So what would be the recommended approach to pre-0.20.x series? To insure each file is used only by one thread, and then it safe to close the handle in that thread? Regards. good question -I'm not sure. For anythiong you get with FileSystem.get(), its now dangerous to

Re: Too many open files error, which gets resolved after some time

2009-06-22 Thread Stas Oskin
Ok, seems this issue is already patched in the Hadoop distro I'm using (Cloudera). Any idea if I still should call GC manually/periodically to clean out all the stale pipes / epolls? 2009/6/22 Steve Loughran ste...@apache.org Stas Oskin wrote: Hi. So what would be the recommended approach

Re: Too many open files error, which gets resolved after some time

2009-06-21 Thread Stas Oskin
Hi. After tracing some more with the lsof utility, and I managed to stop the growth on the DataNode process, but still have issues with my DFS client. It seems that my DFS client opens hundreds of pipes and eventpolls. Here is a small part of the lsof output: java10508 root 387w FIFO

Re: Too many open files error, which gets resolved after some time

2009-06-21 Thread jason hadoop
HDFS/DFS client uses quite a few file descriptors for each open file. Many application developers (but not the hadoop core) rely on the JVM finalizer methods to close open files. This combination, expecially when many HDFS files are open can result in very large demands for file descriptors for

RE: Too many open files error, which gets resolved after some time

2009-06-21 Thread Brian.Levine
@hadoop.apache.org Subject: Re: Too many open files error, which gets resolved after some time HDFS/DFS client uses quite a few file descriptors for each open file. Many application developers (but not the hadoop core) rely on the JVM finalizer methods to close open files. This combination, expecially when many

Re: Too many open files error, which gets resolved after some time

2009-06-21 Thread jason hadoop
will get called, if ever. -brian -Original Message- From: ext jason hadoop [mailto:jason.had...@gmail.com] Sent: Sunday, June 21, 2009 11:19 AM To: core-user@hadoop.apache.org Subject: Re: Too many open files error, which gets resolved after some time HDFS/DFS client uses quite a few

Re: Too many open files error, which gets resolved after some time

2009-06-21 Thread jason hadoop
-user@hadoop.apache.org Subject: Re: Too many open files error, which gets resolved after some time HDFS/DFS client uses quite a few file descriptors for each open file. Many application developers (but not the hadoop core) rely on the JVM finalizer methods to close open files

Re: Too many open files error, which gets resolved after some time

2009-06-21 Thread Scott Carey
To: core-user@hadoop.apache.org Subject: Re: Too many open files error, which gets resolved after some time HDFS/DFS client uses quite a few file descriptors for each open file. Many application developers (but not the hadoop core) rely on the JVM finalizer methods to close open files. This combination