[ 
https://issues.apache.org/jira/browse/HADOOP-8396?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13273883#comment-13273883
 ] 

Uma Maheswara Rao G commented on HADOOP-8396:
---------------------------------------------

{quote}
nd I've checked, after writing a few million records, executing a "reader" 
class on that data, returns the data, meaning Hadoop got to write these to the 
HDFS, but watching "htop" the number of threads and memory for this code kept 
increasing and only when writing started. We're writing from one single thread 
(main).
{quote}
You are not closing the file from application once your data write completed? 
When you open the stream to DFS, user only will start writing the data on that 
stream. Whatever data user writes, DataStreamer will writes to DNs. User only 
will will know, whether stream can be closed or not. Hadoop Client can not 
assume that user will not write any more data on that stream. Because, it is 
still possible that user can write some more data on that stream if stream 
still opens. Once that is closed automatically Streamer threads will exit. I am 
not sure you are about this lines or some other. Please correct if I understand 
ur point wrongly.
                
> DataStreamer, OutOfMemoryError, unable to create new native thread
> ------------------------------------------------------------------
>
>                 Key: HADOOP-8396
>                 URL: https://issues.apache.org/jira/browse/HADOOP-8396
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: io
>    Affects Versions: 1.0.2
>         Environment: Ubuntu 64bit, 4GB of RAM, Core Duo processors, commodity 
> hardware.
>            Reporter: Catalin Alexandru Zamfir
>            Priority: Blocker
>              Labels: DataStreamer, I/O, OutOfMemoryError, ResponseProcessor, 
> hadoop,, leak, memory, rpc,
>
> We're trying to write about 1 few billion records, via "Avro". When we got 
> this error, that's unrelated to our code:
> 10725984 [Main] INFO net.gameloft.RnD.Hadoop.App - ## At: 2:58:43.290 # 
> Written: 521000000 records
> Exception in thread "DataStreamer for file 
> /Streams/Cubed/Stuff/objGame/aRandomGame/objType/aRandomType/2012/05/11/20/29/Shard.avro
>  block blk_3254486396346586049_75838" java.lang.OutOfMemoryError: unable to 
> create new native thread
>         at java.lang.Thread.start0(Native Method)
>         at java.lang.Thread.start(Thread.java:657)
>         at 
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:612)
>         at 
> org.apache.hadoop.ipc.Client$Connection.access$2000(Client.java:184)
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1202)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1046)
>         at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:225)
>         at $Proxy8.getProtocolVersion(Unknown Source)
>         at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:396)
>         at 
> org.apache.hadoop.hdfs.DFSClient.createClientDatanodeProtocolProxy(DFSClient.java:160)
>         at 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.processDatanodeError(DFSClient.java:3117)
>         at 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.access$2200(DFSClient.java:2586)
>         at 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2790)
> 10746169 [Main] INFO net.gameloft.RnD.Hadoop.App - ## At: 2:59:03.474 # 
> Written: 522000000 records
> Exception in thread "ResponseProcessor for block 
> blk_4201760269657070412_73948" java.lang.OutOfMemoryError
>         at sun.misc.Unsafe.allocateMemory(Native Method)
>         at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:117)
>         at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:305)
>         at sun.nio.ch.Util.getTemporaryDirectBuffer(Util.java:75)
>         at sun.nio.ch.IOUtil.read(IOUtil.java:223)
>         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:254)
>         at 
> org.apache.hadoop.net.SocketInputStream$Reader.performIO(SocketInputStream.java:55)
>         at 
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142)
>         at 
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:155)
>         at 
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:128)
>         at java.io.DataInputStream.readFully(DataInputStream.java:195)
>         at java.io.DataInputStream.readLong(DataInputStream.java:416)
>         at 
> org.apache.hadoop.hdfs.protocol.DataTransferProtocol$PipelineAck.readFields(DataTransferProtocol.java:124)
>         at 
> org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$ResponseProcessor.run(DFSClient.java:2964)
> #
> # There is insufficient memory for the Java Runtime Environment to continue.
> # Native memory allocation (malloc) failed to allocate 32 bytes for intptr_t 
> in 
> /build/buildd/openjdk-6-6b23~pre11/build/openjdk/hotspot/src/share/vm/runtime/deoptimization.cpp
> [thread 1587264368 also had an error]
> [thread 1111309168 also had an error]
> [thread 1820371824 also had an error]
> [thread 1343454064 also had an error]
> [thread 1345444720 also had an error]
> # An error report file with more information is saved as:
> # [thread 1345444720 also had an error]
> [thread -1091290256 also had an error]
> [thread 678165360 also had an error]
> [thread 678497136 also had an error]
> [thread 675511152 also had an error]
> [thread 1385937776 also had an error]
> [thread 911969136 also had an error]
> [thread -1086207120 also had an error]
> [thread -1088251024 also had an error]
> [thread -1088914576 also had an error]
> [thread -1086870672 also had an error]
> [thread 441797488 also had an error][thread 445778800 also had an error]
> [thread 440400752 also had an error]
> [thread 444119920 also had an error][thread 1151298416 also had an error]
> [thread 443124592 also had an error]
> [thread 1152625520 also had an error]
> [thread 913628016 also had an error]
> [thread -1095345296 also had an error][thread 1390799728 also had an error]
> [thread 443788144 also had an error]
> [thread 676506480 also had an error]
> [thread 1630595952 also had an error]
> pure virtual method called
> terminate called without an active exception
> pure virtual method called
> Aborted
> It seems to be a memory leak. We were opening 5 - 10 buffers to different 
> paths when writing and closing them. We've tested that those buffers do not 
> overrun. And they don't. But watching the application continue writing, we 
> saw that over a period of 5 to 6 hours, it kept constantly increasing in 
> memory, not by the average of 8MB buffer that we've set, but my small values. 
> I'm reading the code and it seems there's a memory leak somewhere, in the way 
> Hadoop does buffer allocation. While we specifically close the buffers if the 
> count of open buffers is above 5 (meaning 5 * 8MB per buffer) this bug still 
> happens.
> Can it be fixed? As you can see from the strack trace, it writes a "fan-out" 
> path of the type you see in the strack trace. We've let it execute till about 
> 500M records, when this error blew. It's a blocker as these writers need to 
> be production-grade ready, while they're not due to this native buffer 
> allocation that when executing large amounts of writes, seems to generate a 
> memory leak.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to