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

Benedict commented on CASSANDRA-7743:
-------------------------------------

It looks like the problem is caused by a number of changes in 2.1 composing to 
yield especially bad behaviour. We use pooled buffers in netty, but we also 
introduced an SEPWorker pool that has many threads (more than the number that 
actually service any single pool), and all threads may eventually service work 
on the netty executor side. This gives us ~130 threads periodically performing 
this work, and each of them apparently allocates a buffer at some point. These 
buffers are unfortunately allocated from a threadlocal pool, which starts at 
16Mb, so each thread retains at least 16Mb of largely useless memory.

The best fix will be to stop the SEPWorker tasks from allocating any buffers, 
but [~tjake] has pointed out we can also tweak some settings to mitigate the 
negative impact of this kind of problem as well.

I'll look into a patch tomorrow.

> Possible C* OOM issue during long running test
> ----------------------------------------------
>
>                 Key: CASSANDRA-7743
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7743
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: Google Compute Engine, n1-standard-1
>            Reporter: Pierre Laporte
>             Fix For: 2.1.0
>
>
> During a long running test, we ended up with a lot of 
> "java.lang.OutOfMemoryError: Direct buffer memory" errors on the Cassandra 
> instances.
> Here is an example of stacktrace from system.log :
> {code}
> ERROR [SharedPool-Worker-1] 2014-08-11 11:09:34,610 ErrorMessage.java:218 - 
> Unexpected exception during request
> java.lang.OutOfMemoryError: Direct buffer memory
>         at java.nio.Bits.reserveMemory(Bits.java:658) ~[na:1.7.0_25]
>         at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) 
> ~[na:1.7.0_25]
>         at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:306) 
> ~[na:1.7.0_25]
>         at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:434) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:179) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at io.netty.buffer.PoolArena.allocate(PoolArena.java:168) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at io.netty.buffer.PoolArena.allocate(PoolArena.java:98) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:251)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:155)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:146)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.buffer.AbstractByteBufAllocator.ioBuffer(AbstractByteBufAllocator.java:107)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.channel.AdaptiveRecvByteBufAllocator$HandleImpl.allocate(AdaptiveRecvByteBufAllocator.java:104)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:112)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:507) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:464)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:378) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:350) 
> ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at 
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116)
>  ~[netty-all-4.0.20.Final.jar:4.0.20.Final]
>         at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25]
> {code}
> The test consisted of a 3-nodes cluster of n1-standard-1 GCE instances (1 
> vCPU, 3.75 GB RAM) running cassandra-2.1.0-rc5, and a n1-standard-2 instance 
> running the test.
> After ~2.5 days, several requests start to fail and we see the previous 
> stacktraces in the system.log file.
> The output from linux ‘free’ and ‘meminfo’ suggest that there is still memory 
> available.
> {code}
> $ free -m
> total              used       free     shared    buffers     cached
> Mem:          3702       3532        169          0        161        854
> -/+ buffers/cache:       2516       1185
> Swap:            0          0          0
> $ head -n 4 /proc/meminfo
> MemTotal:        3791292 kB
> MemFree:          173568 kB
> Buffers:          165608 kB
> Cached:           874752 kB
> {code}
> These errors do not affect all the queries we run. The cluster is still 
> responsive but is unable to display tracing information using cqlsh :
> {code}
> $ ./bin/nodetool --host 10.240.137.253 status duration_test
> Datacenter: DC1
> ===============
> Status=Up/Down
> |/ State=Normal/Leaving/Joining/Moving
> --  Address         Load       Tokens  Owns (effective)  Host ID              
>                  Rack
> UN  10.240.98.27    925.17 KB  256     100.0%            
> 41314169-eff5-465f-85ea-d501fd8f9c5e  RAC1
> UN  10.240.137.253  1.1 MB     256     100.0%            
> c706f5f9-c5f3-4d5e-95e9-a8903823827e  RAC1
> UN  10.240.72.183   896.57 KB  256     100.0%            
> 15735c4d-98d4-4ea4-a305-7ab2d92f65fc  RAC1
> $ echo 'tracing on; select count(*) from duration_test.ints;' | ./bin/cqlsh 
> 10.240.137.253
> Now tracing requests.
>  count
> -------
>   9486
> (1 rows)
> Statement trace did not complete within 10 seconds
> {code}



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to