Paolo,

try a few things in cassandra-env.sh
1. HEAP_NEWSIZE="2G". "The 100mb/core commentary in cassandra-env.sh for
setting HEAP_NEWSIZE is *wrong*" (
https://tobert.github.io/pages/als-cassandra-21-tuning-guide.html)
2. MaxTenuringThreshold=8
3. enable GC logging (under "# GC logging options -- uncomment to enable"
section) to compare GC behaviors on good and bad nodes.


On Fri, May 27, 2016 at 5:36 AM, Paolo Crosato <
paolo.cros...@targaubiest.com> wrote:

> Hi,
>
> thanks for the answer. There were no large insertions and the saved_caches
> dir had a resonable size. I tried to delete the cashes and set
> key_cache_size_in_mb to zero, but it didn't help.
> Today our virtual hardware provided raised cpus to 4, memory to 32GB and
> doubled the disk size, and the nodes are stable again. So it was probably
> an issue of severe lack of resources.
> About HEAP_NEWSIZE, your suggestion is quite intriguing. I thought it was
> better to set it 100mb*#cores, so in my case I set it to 200 and now I
> should set it to 400. Do larger values help without being harmful?
>
> Regards,
>
> Paolo
>
>
> Il 27/05/2016 03:05, Mike Yeap ha scritto:
>
> Hi Paolo,
>
> a) was there any large insertion done?
> b) are the a lot of files in the saved_caches directory?
> c) would you consider to increase the HEAP_NEWSIZE to, say, 1200M?
>
>
> Regards,
> Mike Yeap
>
> On Fri, May 27, 2016 at 12:39 AM, Paolo Crosato <
> paolo.cros...@targaubiest.com> wrote:
>
>> Hi,
>>
>> we are running a cluster of 4 nodes, each one has the same sizing: 2
>> cores, 16G ram and 1TB of disk space.
>>
>> On every node we are running cassandra 2.0.17, oracle java version
>> "1.7.0_45", centos 6 with this kernel version 2.6.32-431.17.1.el6.x86_64
>>
>> Two nodes are running just fine, the other two have started to go OOM at
>> every start.
>>
>> This is the error we get:
>>
>> INFO [ScheduledTasks:1] 2016-05-26 18:15:58,460 StatusLogger.java (line
>> 70) ReadRepairStage                   0         0            116
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:15:58,462 StatusLogger.java (line
>> 70) MutationStage                    31      1369          20526
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:15:58,590 StatusLogger.java (line
>> 70) ReplicateOnWriteStage             0         0              0
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:15:58,591 StatusLogger.java (line
>> 70) GossipStage                       0         0            335
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:04,195 StatusLogger.java (line
>> 70) CacheCleanupExecutor              0         0              0
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:06,526 StatusLogger.java (line
>> 70) MigrationStage                    0         0              0
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:06,527 StatusLogger.java (line
>> 70) MemoryMeter                       1         4             26
>> 0                 0
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:06,527 StatusLogger.java (line
>> 70) ValidationExecutor                0         0              0
>> 0                 0
>> DEBUG [MessagingService-Outgoing-/10.255.235.19] 2016-05-26 18:16:06,518
>> OutboundTcpConnection.java (line 290) attempting to connect to /
>> 10.255.235.19
>>  INFO [GossipTasks:1] 2016-05-26 18:16:22,912 Gossiper.java (line 992)
>> InetAddress /10.255.235.28 is now DOWN
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:22,952 StatusLogger.java (line
>> 70) FlushWriter                       1         5             47
>> 0                25
>>  INFO [ScheduledTasks:1] 2016-05-26 18:16:22,953 StatusLogger.java (line
>> 70) InternalResponseStage             0         0              0
>> 0                 0
>> ERROR [ReadStage:27] 2016-05-26 18:16:29,250 CassandraDaemon.java (line
>> 258) Exception in thread Thread[ReadStage:27,5,main]
>> java.lang.OutOfMemoryError: Java heap space
>>     at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:347)
>>     at
>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:392)
>>     at
>> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:355)
>>     at
>> org.apache.cassandra.db.ColumnSerializer.deserializeColumnBody(ColumnSerializer.java:124)
>>     at
>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:85)
>>     at org.apache.cassandra.db.Column$1.computeNext(Column.java:75)
>>     at org.apache.cassandra.db.Column$1.computeNext(Column.java:64)
>>     at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>     at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>     at
>> com.google.common.collect.AbstractIterator.next(AbstractIterator.java:153)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:434)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.fetchMoreData(IndexedSliceReader.java:387)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:145)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:45)
>>     at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>     at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>     at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:82)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:157)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:140)
>>     at
>> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:144)
>>     at
>> org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:87)
>>     at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:46)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:120)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:80)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:72)
>>     at
>> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:297)
>>     at
>> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
>>     at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1619)
>>     at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1438)
>>     at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:340)
>>     at
>> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:89)
>>     at
>> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47)
>> ERROR [ReadStage:32] 2016-05-26 18:16:29,357 CassandraDaemon.java (line
>> 258) Exception in thread Thread[ReadStage:32,5,main]
>> java.lang.OutOfMemoryError: Java heap space
>>     at
>> org.apache.cassandra.io.util.RandomAccessReader.readBytes(RandomAccessReader.java:347)
>>     at
>> org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:392)
>>     at
>> org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:355)
>>     at
>> org.apache.cassandra.db.ColumnSerializer.deserializeColumnBody(ColumnSerializer.java:124)
>>     at
>> org.apache.cassandra.db.OnDiskAtom$Serializer.deserializeFromSSTable(OnDiskAtom.java:85)
>>     at org.apache.cassandra.db.Column$1.computeNext(Column.java:75)
>>     at org.apache.cassandra.db.Column$1.computeNext(Column.java:64)
>>     at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>     at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>     at
>> com.google.common.collect.AbstractIterator.next(AbstractIterator.java:153)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.getNextBlock(IndexedSliceReader.java:434)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetcher.fetchMoreData(IndexedSliceReader.java:387)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:145)
>>     at
>> org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(IndexedSliceReader.java:45)
>>     at
>> com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143)
>>     at
>> com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138)
>>     at
>> org.apache.cassandra.db.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:82)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter$2.getNext(QueryFilter.java:157)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter$2.hasNext(QueryFilter.java:140)
>>     at
>> org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:144)
>>     at
>> org.apache.cassandra.utils.MergeIterator$ManyToOne.<init>(MergeIterator.java:87)
>>     at org.apache.cassandra.utils.MergeIterator.get(MergeIterator.java:46)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:120)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:80)
>>     at
>> org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:72)
>>     at
>> org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:297)
>>     at
>> org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53)
>>     at
>> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1619)
>>     at
>> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1438)
>>     at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:340)
>>     at
>> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:89)
>>     at
>> org.apache.cassandra.db.ReadVerbHandler.doVerb(ReadVerbHandler.java:47)
>>
>> We are observing that the heap is never flushed, it keeps increasing
>> until reaching the limit, then the OOM errors appear and after a short
>> while the node crashes.
>>
>> These are the relevant settings in cassandra_env for one of the crashing
>> nodes:
>>
>> MAX_HEAP_SIZE="6G"
>> HEAP_NEWSIZE="200M"
>>
>> This is the complete error log http://pastebin.com/QGaACyhR
>>
>> This is cassandra_env http://pastebin.com/6SLeVmtv
>>
>> This is cassandra.yaml http://pastebin.com/wb1axHtV
>>
>> Can anyone help?
>>
>> Regards,
>>
>> Paolo Crosato
>>
>> --
>> Paolo Crosato
>> Software engineer/Custom Solutions
>> e-mail: paolo.cros...@targaubiest.com
>>
>>
>
>
> --
> Paolo Crosato
> Software engineer/Custom Solutions
> e-mail: paolo.cros...@targaubiest.com
>
>

Reply via email to