It's clear you are out of memory. How big is your data size? 
heap of 1867M is kind of small. According to the discussion on this list, it's 
advisable to have m1.xlarge.  

Attached please find the related thread.

-Wei

----- Original Message -----
From: "Dane Miller" <d...@optimalsocial.com>
To: user@cassandra.apache.org
Sent: Wednesday, March 20, 2013 7:13:44 PM
Subject: Stream fails during repair, two nodes out-of-memory

After having just solved one repair problem, I immediately hit
another.  Again, much appreciation for suggestions...

I'm having problems repairing a CF, and the failure consistenly brings
down 2 of the 6 nodes in the cluster.  I'm running "repair -pr" on a
single CF on node2, the repair starts streaming, and after about 60
seconds both node2 and node4 crash with java.lang.OutOfMemoryError.
The keyspace has rf=3 and is being actively written to by our
application.

The abbrieviated logs below show the pattern, after which I kill -9
and restart cassandra on the two nodes.  What extra info should I
include?  I'm kind of overwhelmed by the volume of logs being
generated and not sure what is signal vs noise.  I'm especially seeing
big repeating sections of StatusLogger and FlushWriter/Memtable.

Details:
6 node cluster
cassandra  1.2.2 - single token per node
RandomPartitioner, EC2Snitch
Replication: SimpleStrategy, rf=3
Ubuntu 10.10 x86_64
EC2 m1.large
Cassandra max heap: 1867M


node2 (abbrieviated logs)

ERROR 21:11:22 AbstractStreamSession.java Stream failed because [node4] died
GC for ConcurrentMarkSweep: 2365 ms for 2 collections, 1913603168
used; max is 1937768448
Pool Name                    Active   Pending   Blocked
ReadStage                         7         7         0
RequestResponseStage              0         0         0
ReadRepairStage                   0         0         0
MutationStage                    32      4707         0
ReplicateOnWriteStage             0         0         0
GossipStage                       0         0         0
AntiEntropyStage                  0         0         0
MigrationStage                    0         0         0
MemtablePostFlusher               1         1         0
FlushWriter                       1         1         0
MiscStage                         0         0         0
commitlog_archiver                0         0         0
InternalResponseStage             0         0         0
AntiEntropySessions               1         1         0
HintedHandoff                     0         0         0
CompactionManager                 1        21
MessagingService                n/a    291,35
WARN  21:12:52 GCInspector.java Heap is 0.9875293252788064 full
INFO  21:12:52 Gossiper.java InetAddress [node5] is now dead.
INFO  21:12:52 Gossiper.java InetAddress [node1] is now dead.
INFO  21:12:52 Gossiper.java InetAddress [node6] is now dead.
INFO  21:12:52 ColumnFamilyStore.java Enqueuing flush of Memtable-[MyCF]@...
INFO  21:12:52 MessagingService.java 4415 MUTATION messages dropped in
last 5000ms
INFO  21:12:52 Gossiper.java InetAddress [node5] is now UP
INFO  21:12:52 Gossiper.java InetAddress [node1] is now UP
INFO  21:12:52 Gossiper.java InetAddress [node6] is now UP
INFO  21:12:52 HintedHandOffManager.java Started hinted handoff for
host: [node5]
INFO  21:12:52 HintedHandOffManager.java Started hinted handoff for
host: [node1]
ERROR 21:12:56 CassandraDaemon.java java.lang.OutOfMemoryError: Java heap space
(full OutOfMemory stack trace is included at bottom)

node4 (abbrieviated logs)

INFO 21:10:05 StreamOutSession.java Streaming to [node2]
INFO 21:10:14 CompactionTask.java Compacted 4 sstables to [MyCF-ib-17665]
INFO 21:10:24 StreamReplyVerbHandler.java Successfully sent
[MyCF]-ib-17647-Data.db to [node2]
INFO 21:10:24 GCInspector.java GC for ConcurrentMarkSweep
GC for ConcurrentMarkSweep: 764 ms for 3 collections, 1408393640 used;
max is 1937768448
GC for ConcurrentMarkSweep: 2198 ms for 2 collections, 1882942392
used; max is 1937768448
Pool Name                    Active   Pending   Blocked
ReadStage                         5         5         0
RequestResponseStage              0        20         0
ReadRepairStage                   0         0         0
MutationStage                     0         0         0
ReplicateOnWriteStage             0         0         0
GossipStage                       0         8         0
AntiEntropyStage                  0         0         0
MigrationStage                    0         0         0
MemtablePostFlusher               0         0         0
FlushWriter                       0         0         0
MiscStage                         0         0         0
commitlog_archiver                0         0         0
InternalResponseStage             0         0         0
AntiEntropySessions               0         0         0
HintedHandoff                     1         1         0
CompactionManager                 0         6
MessagingService                n/a     10,15
INFO 21:11:35 Gossiper.java InetAddress [node5] is now dead.
INFO 21:11:35 Gossiper.java InetAddress [node2] is now dead.
ERROR 21:13:17 CassandraDaemon.java java.lang.OutOfMemoryError: Java heap space
(full OutOfMemory stack trace is included at bottom)




node2 full OOM stack trace:

ERROR [Thread-417] 2013-03-20 21:12:56,114 CassandraDaemon.java (line
133) Exception in thread Thread[Thread-417,5,main]
java.lang.OutOfMemoryError: Java heap space
        at org.apache.cassandra.utils.obs.OpenBitSet.<init>(OpenBitSet.java:76)
        at 
org.apache.cassandra.utils.FilterFactory.createFilter(FilterFactory.java:143)
        at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:114)
        at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:101)
        at org.apache.cassandra.db.ColumnIndex.<init>(ColumnIndex.java:40)
        at org.apache.cassandra.db.ColumnIndex.<init>(ColumnIndex.java:31)
        at 
org.apache.cassandra.db.ColumnIndex$Builder.<init>(ColumnIndex.java:74)
        at 
org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:243)
        at 
org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
        at 
org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
        at 
org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:226)
        at 
org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:166)
        at 
org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:66)

node4 full OOM stack trace:

ERROR [Thread-326] 2013-03-20 21:13:22,829 CassandraDaemon.java (line
133) Exception in thread Thread[Thread-326,5,main]
java.lang.OutOfMemoryError: Java heap space
        at org.apache.cassandra.utils.obs.OpenBitSet.<init>(OpenBitSet.java:76)
        at 
org.apache.cassandra.utils.FilterFactory.createFilter(FilterFactory.java:143)
        at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:114)
        at 
org.apache.cassandra.utils.FilterFactory.getFilter(FilterFactory.java:101)
        at org.apache.cassandra.db.ColumnIndex.<init>(ColumnIndex.java:40)
        at org.apache.cassandra.db.ColumnIndex.<init>(ColumnIndex.java:31)
        at 
org.apache.cassandra.db.ColumnIndex$Builder.<init>(ColumnIndex.java:74)
        at 
org.apache.cassandra.io.sstable.SSTableWriter.appendFromStream(SSTableWriter.java:243)
        at 
org.apache.cassandra.streaming.IncomingStreamReader.streamIn(IncomingStreamReader.java:179)
        at 
org.apache.cassandra.streaming.IncomingStreamReader.read(IncomingStreamReader.java:122)
        at 
org.apache.cassandra.net.IncomingTcpConnection.stream(IncomingTcpConnection.java:226)
        at 
org.apache.cassandra.net.IncomingTcpConnection.handleStream(IncomingTcpConnection.java:166)
        at 
org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:66)


Dane
--- Begin Message ---
> If you have the chance, could you expand on m1.xlarge being the much better 
> choice? 
For background 
http://perfcap.blogspot.co.nz/2011/03/understanding-and-using-amazon-ebs.html

it's an older post, but the idea is the same. On smaller machines you share 
more, and sharing is bad. Many times I've seen clusters moved from m1.xlarge 
have better performance and less issues, such as nodes flapping. 

C* runs a lot of threads, running these on a system with 2 cores is less 
efficient that 4 and 4 is less efficient than 8. Memory wise a 4Gb to 8Gb JVM 
heap is desirable, as is as much memory as possible for the OS page cache or C* 
row cache. So a machine with 16GB and 4 to 8 cores is a good starting point. 

> of expanding from a 12 node -> 24 node cluster using .large instances, vs. 
> upgrading all instances to m1.xlarge, soon and the justifications would be 
> helpful (although Aaron says so does help ;) ).  
Aside from the CPU / Memory / IO issues, you will also need to consider the 
data load per node. In EC2 try to keep between 300GB and 500GB per node. Once 
you get above 500GB maintenance tasks like repair and replacing nodes take a 
long time. 

> Doubling the cluster size would, in theory, halve the time for this overhead, 
> but would still impact performance during that time.  Going to xlarge would 
> lessen the impact of these activities on operations.
My experience is that an m1.xlarge is more than twice the performance of an 
m1.large. 

Hope that helps. 

-----------------
Aaron Morton
Freelance Cassandra Consultant
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 15/03/2013, at 4:44 AM, Michael Theroux <mthero...@yahoo.com> wrote:

> Hi Aaron,
> 
> If you have the chance, could you expand on m1.xlarge being the much better 
> choice?  We are going to need to make a choice of expanding from a 12 node -> 
> 24 node cluster using .large instances, vs. upgrading all instances to 
> m1.xlarge, soon and the justifications would be helpful (although Aaron says 
> so does help ;) ).  
> 
> One obvious reason is administrating a 24 node cluster does add person-time 
> overhead.  
> 
> Another reason includes less impact of maintenance activities such as repair, 
> as these activites have significant CPU overhead.  Doubling the cluster size 
> would, in theory, halve the time for this overhead, but would still impact 
> performance during that time.  Going to xlarge would lessen the impact of 
> these activities on operations.
> 
> Anything else?
> 
> Thanks,
> 
> -Mike
> 
> On Mar 14, 2013, at 9:27 AM, aaron morton wrote:
> 
>>> Because of this I have an unstable cluster and have no other choice than 
>>> use Amazon EC2 xLarge instances when we would rather use twice more EC2 
>>> Large nodes.
>> m1.xlarge is a MUCH better choice than m1.large.
>> You get more ram and better IO and less steal. Using half as many m1.xlarge 
>> is the way to go. 
>> 
>>> My heap is actually changing from 3-4 GB to 6 GB and sometimes growing to 
>>> the max 8 GB (crashing the node).
>> How is it crashing ?
>> Are you getting too much GC or running OOM ? 
>> Are you using the default GC configuration ?
>> Is cassandra logging a lot of GC warnings ?
>> 
>> If you are running OOM then something has to change. Maybe bloom filters, 
>> maybe caches.
>> 
>> Enable the GC logging in cassandra-env.sh to check how low a CMS compaction 
>> get's the heap, or use some other tool. That will give an idea of how much 
>> memory you are using. 
>> 
>> Here is some background on what is kept on heap in pre 1.2
>> http://www.mail-archive.com/user@cassandra.apache.org/msg25762.html
>> 
>> Cheers
>> 
>> -----------------
>> Aaron Morton
>> Freelance Cassandra Consultant
>> New Zealand
>> 
>> @aaronmorton
>> http://www.thelastpickle.com
>> 
>> On 13/03/2013, at 12:19 PM, Wei Zhu <wz1...@yahoo.com> wrote:
>> 
>>> Here is the JIRA I submitted regarding the ancestor.
>>> 
>>> https://issues.apache.org/jira/browse/CASSANDRA-5342
>>> 
>>> -Wei
>>> 
>>> 
>>> ----- Original Message -----
>>> From: "Wei Zhu" <wz1...@yahoo.com>
>>> To: user@cassandra.apache.org
>>> Sent: Wednesday, March 13, 2013 11:35:29 AM
>>> Subject: Re: About the heap
>>> 
>>> Hi Dean,
>>> The index_interval is controlling the sampling of the SSTable to speed up 
>>> the lookup of the keys in the SSTable. Here is the code:
>>> 
>>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/DataTracker.java#L478
>>> 
>>> To increase the interval meaning, taking less samples, less memory, slower 
>>> lookup for read.
>>> 
>>> I did do a heap dump on my production system which caused about 10 seconds 
>>> pause of the node. I found something interesting, for LCS, it could involve 
>>> thousands of SSTables for one compaction, the ancestors are recorded in 
>>> case something goes wrong during the compaction. But those are never 
>>> removed after the compaction is done. In our case, it takes about 1G of 
>>> heap memory to store that. I am going to submit a JIRA for that. 
>>> 
>>> Here is the culprit:
>>> 
>>> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/SSTableMetadata.java#L58
>>> 
>>> Enjoy looking at Cassandra code:)
>>> 
>>> -Wei
>>> 
>>> 
>>> ----- Original Message -----
>>> From: "Dean Hiller" <dean.hil...@nrel.gov>
>>> To: user@cassandra.apache.org
>>> Sent: Wednesday, March 13, 2013 11:11:14 AM
>>> Subject: Re: About the heap
>>> 
>>> Going to 1.2.2 helped us quite a bit as well as turning on LCS from STCS 
>>> which gave us smaller bloomfilters.
>>> 
>>> As far as key cache.  There is an entry in cassandra.yaml called 
>>> index_interval set to 128.  I am not sure if that is related to key_cache.  
>>> I think it is.  By turning that to 512 or maybe even 1024, you will consume 
>>> less ram there as well though I ran this test in QA and my key cache size 
>>> stayed the same so I am really not sure(I am actually checking out 
>>> cassandra code now to dig a little deeper into this property.
>>> 
>>> Dean
>>> 
>>> From: Alain RODRIGUEZ <arodr...@gmail.com<mailto:arodr...@gmail.com>>
>>> Reply-To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" 
>>> <user@cassandra.apache.org<mailto:user@cassandra.apache.org>>
>>> Date: Wednesday, March 13, 2013 10:11 AM
>>> To: "user@cassandra.apache.org<mailto:user@cassandra.apache.org>" 
>>> <user@cassandra.apache.org<mailto:user@cassandra.apache.org>>
>>> Subject: About the heap
>>> 
>>> Hi,
>>> 
>>> I would like to know everything that is in the heap.
>>> 
>>> We are here speaking of C*1.1.6
>>> 
>>> Theory :
>>> 
>>> - Memtable (1024 MB)
>>> - Key Cache (100 MB)
>>> - Row Cache (disabled, and serialized with JNA activated anyway, so should 
>>> be off-heap)
>>> - BloomFilters (about 1,03 GB - from cfstats, adding all the "Bloom Filter 
>>> Space Used" and considering they are showed in Bytes - 1103765112)
>>> - Anything else ?
>>> 
>>> So my heap should be fluctuating between 1,15 GB and 2.15 GB and growing 
>>> slowly (from the new BF of my new data).
>>> 
>>> My heap is actually changing from 3-4 GB to 6 GB and sometimes growing to 
>>> the max 8 GB (crashing the node).
>>> 
>>> Because of this I have an unstable cluster and have no other choice than 
>>> use Amazon EC2 xLarge instances when we would rather use twice more EC2 
>>> Large nodes.
>>> 
>>> What am I missing ?
>>> 
>>> Practice :
>>> 
>>> Is there a way not inducing any load and easy to do to dump the heap to 
>>> analyse it with MAT (or anything else that you could advice) ?
>>> 
>>> Alain
>>> 
>>> 
>> 
> 


--- End Message ---

Reply via email to