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

Branimir Lambov commented on CASSANDRA-11710:
---------------------------------------------

Yes, it is. If we don't include it, nodes spawned with default ccm settings 
will oom in 3.6.

> Cassandra dies with OOM when running stress
> -------------------------------------------
>
>                 Key: CASSANDRA-11710
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11710
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Marcus Eriksson
>            Assignee: Branimir Lambov
>             Fix For: 3.6
>
>
> Running stress on trunk dies with OOM after about 3.5M ops:
> {code}
> ERROR [CompactionExecutor:1] 2016-05-04 15:01:31,231 
> JVMStabilityInspector.java:137 - JVM state determined to be unstable.  
> Exiting forcefully due to:
> java.lang.OutOfMemoryError: Direct buffer memory
>         at java.nio.Bits.reserveMemory(Bits.java:693) ~[na:1.8.0_91]
>         at java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:123) 
> ~[na:1.8.0_91]
>         at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311) 
> ~[na:1.8.0_91]
>         at 
> org.apache.cassandra.utils.memory.BufferPool.allocateDirectAligned(BufferPool.java:519)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool.access$600(BufferPool.java:46) 
> ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool$GlobalPool.allocateMoreChunks(BufferPool.java:276)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool$GlobalPool.get(BufferPool.java:249)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool$LocalPool.addChunkFromGlobalPool(BufferPool.java:338)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool$LocalPool.get(BufferPool.java:381)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool.maybeTakeFromPool(BufferPool.java:142)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool.takeFromPool(BufferPool.java:114)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.memory.BufferPool.get(BufferPool.java:84) 
> ~[main/:na]
>         at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:135) 
> ~[main/:na]
>         at org.apache.cassandra.cache.ChunkCache.load(ChunkCache.java:19) 
> ~[main/:na]
>         at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache$BoundedLocalLoadingCache.lambda$new$0(BoundedLocalCache.java:2949)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.lambda$doComputeIfAbsent$15(BoundedLocalCache.java:1807)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> java.util.concurrent.ConcurrentHashMap.compute(ConcurrentHashMap.java:1853) 
> ~[na:1.8.0_91]
>         at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.doComputeIfAbsent(BoundedLocalCache.java:1805)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> com.github.benmanes.caffeine.cache.BoundedLocalCache.computeIfAbsent(BoundedLocalCache.java:1788)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> com.github.benmanes.caffeine.cache.LocalCache.computeIfAbsent(LocalCache.java:97)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> com.github.benmanes.caffeine.cache.LocalLoadingCache.get(LocalLoadingCache.java:66)
>  ~[caffeine-2.2.6.jar:na]
>         at 
> org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:215)
>  ~[main/:na]
>         at 
> org.apache.cassandra.cache.ChunkCache$CachingRebufferer.rebuffer(ChunkCache.java:193)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.LimitingRebufferer.rebuffer(LimitingRebufferer.java:34)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:78)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.RandomAccessReader.reBuffer(RandomAccessReader.java:72)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.RebufferingInputStream.read(RebufferingInputStream.java:88)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:66)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:400) 
> ~[main/:na]
>         at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithVIntLength(ByteBufferUtil.java:338)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.marshal.AbstractType.readValue(AbstractType.java:414) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.Cell$Serializer.deserialize(Cell.java:243) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.readSimpleColumn(UnfilteredSerializer.java:473)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserializeRowBody(UnfilteredSerializer.java:451)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.UnfilteredSerializer.deserialize(UnfilteredSerializer.java:380)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:87)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.SSTableSimpleIterator$CurrentFormatIterator.computeNext(SSTableSimpleIterator.java:65)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.computeNext(SSTableIdentityIterator.java:100)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.SSTableIdentityIterator.computeNext(SSTableIdentityIterator.java:30)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:100)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.LazilyInitializedUnfilteredRowIterator.computeNext(LazilyInitializedUnfilteredRowIterator.java:32)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[main/:na]
>         at 
> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:374)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:186)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:155)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:419)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.rows.UnfilteredRowIterators$UnfilteredRowMergeIterator.computeNext(UnfilteredRowIterators.java:279)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.transform.BaseRows.hasNext(BaseRows.java:112) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.ColumnIndex.buildRowIndex(ColumnIndex.java:93) 
> ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.format.big.BigTableWriter.append(BigTableWriter.java:161)
>  ~[main/:na]
>         at 
> org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:129)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.writers.MaxSSTableSizeWriter.realAppend(MaxSSTableSizeWriter.java:98)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.writers.CompactionAwareWriter.append(CompactionAwareWriter.java:141)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:187)
>  ~[main/:na]
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 
> ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:82)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60)
>  ~[main/:na]
>         at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:264)
>  ~[main/:na]
>         at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> ~[na:1.8.0_91]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
> ~[na:1.8.0_91]
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  ~[na:1.8.0_91]
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  [na:1.8.0_91]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_91]
> {code}
> with this stress profile:
> {code}
> keyspace: stresscql
> table: datatest
> keyspace_definition: |
>   CREATE KEYSPACE stresscql WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 3};
> table_definition: |
>   CREATE TABLE datatest (
>         id int,
>         id2 int,
>         xyz text,
>         PRIMARY KEY(id,id2)
>   )
>   WITH compaction = { 'class':'LeveledCompactionStrategy'}
> columnspec:
>   - name: id
>     population: GAUSSIAN(1..5000000)     # the range of unique values to 
> select for the field (default is 100Billion)
>   - name: id2
>     population: GAUSSIAN(1..50000)
>     cluster: uniform(1..10)
>   - name: xyz
>     size: uniform(1..100)
> insert:
>   partitions: fixed(1)       # number of unique partitions to update in a 
> single operation
>                                   # if batchcount > 1, multiple batches will 
> be used but all partitions will
>                                   # occur in all batches (unless they finish 
> early); only the row counts will vary
>   batchtype: UNLOGGED               # type of batch to use
>   select: uniform(1..10)/10       # uniform chance any single generated CQL 
> row will be visited in a partition;
>                                   # generated for each partition 
> independently, each time we visit it
> queries:
>    simple1:
>       cql: select * from datatest where id = ? and id2 = ? LIMIT 100
>       fields: samerow             # samerow or multirow (select arguments 
> from the same row, or randomly from all rows in the partition)
> {code}
> run like this:
> {code}
> $ ccm create -n2 xyz
> $ ccm start
> $ tools/bin/cassandra-stress user n=10M profile=tools/cqlstress-data.yaml 
> ops\(insert=1\) -rate threads=100
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to