[ 
http://issues.apache.org/jira/browse/HADOOP-210?page=comments#action_12379098 ] 

Doug Cutting commented on HADOOP-210:
-------------------------------------

I'd guess you're out of file handles or threads (both which can appear as 
OutOfMemoryError).  Each DFS client JVM and each datanode keeps a connection 
open to the namenode with a corresponding thread.  The number of threads per 
process in some older kernels was limited, but more recent kernels have mostly 
removed that limit, and the scheduler now also supports large numbers of 
threads effectively.  But you may need to change some limits.  Use 'ulimit -n' 
to see how many file handles you are permitted, and increase that to at least 
4x the number of nodes in your cluster.  You may need to change some kernel 
options to increase the number of threads:

http://www.kegel.com/c10k.html#limits.threads

You can monitor the number of open file handles with 'lsof', and the number of 
threads with 'ps'.

I spent some time trying to get Hadoop's IPC to use non-blocking IO a while 
back (and hence far fewer threads).  The problem is that, since IPC requests 
include objects, we cannot start processing a request until we've recieved the 
complete request, and requests can be bigger than a single packet.  Moreover, 
the end of one request and the beginning of the next can be combined in a 
packet.  So it's easy to accumulate buffers for many connections using just a 
single thread, the problem is knowing when a buffer has a complete request that 
should be dispatched to a worker thread.  So we'd need to length-prefix 
requests, or break them into length-prefixed chunks.  This may be required for 
effective operation of very large clusters, or perhaps Linux kernel threads are 
now up to the task.  We'll soon see.


> Namenode not able to accept connections
> ---------------------------------------
>
>          Key: HADOOP-210
>          URL: http://issues.apache.org/jira/browse/HADOOP-210
>      Project: Hadoop
>         Type: Bug

>   Components: dfs
>  Environment: linux
>     Reporter: Mahadev konar
>     Assignee: Mahadev konar

>
> I am running owen's random writer on a 627 node cluster (writing 10GB/node).  
> After running for a while (map 12% reduce 1%) I get the following error on 
> the Namenode:
> Exception in thread "Server listener on port 60000" 
> java.lang.OutOfMemoryError: unable to create new native thread
>         at java.lang.Thread.start0(Native Method)
>         at java.lang.Thread.start(Thread.java:574)
>         at org.apache.hadoop.ipc.Server$Listener.run(Server.java:105)
> After this, the namenode does not seem to be accepting connections from any 
> of the clients. All the DFSClient calls get timeout. Here is a trace for one 
> of them:
> java.net.SocketTimeoutException: timed out waiting for rpc response
>       at org.apache.hadoop.ipc.Client.call(Client.java:305)
>       at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:149)
>       at org.apache.hadoop.dfs.$Proxy1.open(Unknown Source)
>       at 
> org.apache.hadoop.dfs.DFSClient$DFSInputStream.openInfo(DFSClient.java:419)
>       at org.apache.hadoop.dfs.DFSClient$DFSInputStream.(DFSClient.java:406)
>       at org.apache.hadoop.dfs.DFSClient.open(DFSClient.java:171)
>       at 
> org.apache.hadoop.dfs.DistributedFileSystem.openRaw(DistributedFileSystem.java:78)
>       at 
> org.apache.hadoop.fs.FSDataInputStream$Checker.(FSDataInputStream.java:46)
>       at org.apache.hadoop.fs.FSDataInputStream.(FSDataInputStream.java:228)
>       at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:157)
>       at 
> org.apache.hadoop.mapred.TextInputFormat.getRecordReader(TextInputFormat.java:43)
>       at org.apache.hadoop.mapred.MapTask.run(MapTask.java:105)
>       at 
> org.apache.hadoop.mapred.TaskTracker$Child.main(TaskTracker.java:785).
> The namenode then has around 1% CPU utilization at this time (after the 
> outofmemory exception has been thrown). I have profiled the NameNode and it 
> seems to be using around a maixmum heap size of 57MB (which is not much). So, 
> heap size does not seem to be a problem. It might be happening due to lack of 
> Stack space? Any pointers?

-- 
This message is automatically generated by JIRA.
-
If you think it was sent incorrectly contact one of the administrators:
   http://issues.apache.org/jira/secure/Administrators.jspa
-
For more information on JIRA, see:
   http://www.atlassian.com/software/jira

Reply via email to