cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ernst D Schoen-René <er...@peoplebrowsr.com>
Subject Re: cassandra unexpected shutdown
Date Thu, 25 Aug 2011 23:11:58 GMT
Thanks.  The only logs I have are system and cassandra.  I've included 
those.  I don't have gcinspector logs.  I log gc via munin on other 
machines, but I need to install it on these.

On 8/25/11 2:22 PM, Adi wrote:
> Ernst,
>           Can you share the logs just before the crash. Specially the
> GCInspector logs.Check the last reported used heap space and whether
> it was close to the threshold for full GC.
> Also how frequent are your OOM crashes?
>
>   The cassandra default for kicking in full GC is 75% (
> -XX:CMSInitiatingOccupancyFraction=75). You can try lowering that to
> maybe 65 or some lower number and see if that helps. Basically you
> will have a little more frequent GCs which will take a little less
> time than before. And possibly that unused heap buffer will provide
> for the operation which is causing OOM.
>
> I saw an OOM on one node after 2 weeks. The heap used was close to the
> GC threshold and full GC takes around 80 seconds.
>
> -Adi
>
>
> 2011/8/24 Ernst D Schoen-René<ernst@peoplebrowsr.com>:
>> So, we're on 8, so I don't think there's a key cache setting.  Am I wrong?
>>
>> here's my newest crash log:
>>
>> ERROR [Thread-210] 2011-08-24 06:29:53,247 AbstractCassandraDaemon.java
>> (line 113) Fatal exception in thread Thread[Thread-210,5,main]
>> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
>> down
>>      at
>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73)
>>      at
>> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>>      at
>> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>>      at
>> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:444)
>>      at
>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:117)
>>   INFO [ScheduledTasks:1] 2011-08-24 06:29:53,249 StatusLogger.java (line 65)
>> InternalResponseStage             0         0
>>   INFO [ScheduledTasks:1] 2011-08-24 06:29:53,250 StatusLogger.java (line 65)
>> HintedHandoff                     1        16
>>   INFO [ScheduledTasks:1] 2011-08-24 06:29:53,250 StatusLogger.java (line 69)
>> CompactionManager               n/a      1298
>>   INFO [ScheduledTasks:1] 2011-08-24 06:29:53,250 StatusLogger.java (line 81)
>> MessagingService                n/a       1,0
>>   INFO [ScheduledTasks:1] 2011-08-24 06:29:53,250 StatusLogger.java (line 85)
>> ColumnFamily                Memtable ops,data  Row cache size/cap  Key cache
>> size/cap
>>
>>
>>
>> On 8/24/11 2:38 AM, aaron morton wrote:
>>
>> First thing is are you on 0.8 ? It has some automagical memory management
>> that is both automatic and
>> magical http://thelastpickle.com/2011/05/04/How-are-Memtables-measured/
>> Secondly if you are OOM'ing you need to look at how much memory your schema
>> is taking. See the link above, or just use 0.8
>> Thirdly, try to reduce the cache as Adi noted.
>> Finally try to work out when it's OOM'ing. Is it during a compaction, a
>> repair, during peak load or just regular operations. With 1TB of data i'd be
>> taking a look at the compaction or repair processes to see whats happening
>> GC wise. Also check the GC settings as described in cassandra-env.sh
>> Hope that helps.
>>
>> -----------------
>> Aaron Morton
>> Freelance Cassandra Developer
>> @aaronmorton
>> http://www.thelastpickle.com
>> On 24/08/2011, at 12:38 PM, Ernst D Schoen-René wrote:
>>
>> Thanks,
>>     We had already been running cassandra with a larger heap size, but it
>> meant that java took way too long between garbage collections.  The advice
>> I'd found was to set the heap size at the 8 we're running at.  It was ok for
>> a while, but now some nodes crash.  It's definitely our experience that
>> adding more memory per node actually makes things worse eventually, as java
>> starts eating up too many resources for it to handle.
>>
>>
>> On 8/23/11 5:28 PM, Adi wrote:
>>
>> 2011/8/23 Ernst D Schoen-René<ernst@peoplebrowsr.com>:
>>
>> Hi,
>>
>>   I'm running a 16-node cassandra cluster, with a reasonably large amount of
>>
>> data per node (~1TB).  Nodes have 16G ram, but heap is set to 8G.
>>
>> The nodes keep stopping with this output in the log.  Any ideas?
>>
>> ERROR [Thread-85] 2011-08-23 21:00:38,723 AbstractCassandraDaemon.java (line
>>
>> 113) Fatal exception in thread Thread[Thread-85,5,main]
>>
>> java.lang.OutOfMemoryError: Java heap space
>>
>> ERROR [ReadStage:568] 2011-08-23 21:00:38,723 AbstractCassandraDaemon.java
>>
>> (line 113) Fatal exception in thread Thread[ReadStage:568,5,main]
>>
>> java.lang.OutOfMemoryError: Java heap space
>>
>>   INFO [HintedHandoff:1] 2011-08-23 21:00:38,720 HintedHandOffManager.java
>>
>> (line 320) Started hinted handoff for endpoint /10.28.0.184
>>
>>   INFO [GossipStage:2] 2011-08-23 21:00:50,751 Gossiper.java (line 606)
>>
>> InetAddress /10.29.20.67 is now UP
>>
>> ERROR [Thread-34] 2011-08-23 21:00:50,525 AbstractCassandraDaemon.java (line
>>
>> 113) Fatal exception in thread Thread[Thread-34,5,main]
>>
>> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
>>
>> down
>>
>>     at
>>
>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>>
>>     at
>>
>> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:444)
>>
>>     at
>>
>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:117)
>>
>> ERROR [Thread-36] 2011-08-23 21:00:50,518 AbstractCassandraDaemon.java (line
>>
>> 113) Fatal exception in thread Thread[Thread-36,5,main]
>>
>> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
>>
>> down
>>
>>     at
>>
>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>>
>>     at
>>
>> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:444)
>>
>>     at
>>
>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:117)
>>
>>   INFO [GossipTasks:1] 2011-08-23 21:00:50,466 Gossiper.java (line 620)
>>
>> InetAddress /10.29.20.67 is now dead.
>>
>>   INFO [HintedHandoff:1] 2011-08-23 21:00:50,751 HintedHandOffManager.java
>>
>> (line 376) Finished hinted handoff of 0 rows to endpoint /10.28.0.184
>>
>> ERROR [Thread-33] 2011-08-23 21:01:05,048 AbstractCassandraDaemon.java (line
>>
>> 113) Fatal exception in thread Thread[Thread-33,5,main]
>>
>> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
>>
>> down
>>
>>     at
>>
>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>>
>>     at
>>
>> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:444)
>>
>>     at
>>
>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:117)
>>
>> ERROR [Thread-128] 2011-08-23 21:01:05,048 AbstractCassandraDaemon.java
>>
>> (line 113) Fatal exception in thread Thread[Thread-128,5,main]
>>
>> java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut
>>
>> down
>>
>>     at
>>
>> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:73)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:767)
>>
>>     at
>>
>> java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:658)
>>
>>     at
>>
>> org.apache.cassandra.net.MessagingService.receive(MessagingService.java:444)
>>
>>     at
>>
>> org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:117)
>>
>> root@cass1:~#
>>
>>
>>
>> You can try the  cargo cult solution of upping the heap to 12GB and
>>
>> see if the nodes stabilize. We have a 4-node cluster with 2-3 TB data
>>
>> per node and that was the heap at which it the nodes were managing to
>>
>> serve requests without running out of memory. Ultimately we ordered
>>
>> more memory and are running it with 24 GB heap and the cluster has
>>
>> been stable without complains.
>>
>> Other things you can do for reducing memory usage if they are
>>
>> appropriate for your read/write profile:
>>
>> a) reduce memtable throughput(most reduction in mem footprint)
>>
>> b) disable row caching
>>
>> c) reduce/disable key caching(least reduction)
>>
>> Ultimately you will have to tune based on your
>>
>> 1)  row sizes
>>
>> 2) read/write load
>>
>> -Adi
>>
>>
>>
>>


Mime
View raw message