You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by "Desimpel, Ignace" <Ig...@nuance.com> on 2014/01/08 14:15:06 UTC

OOM after some days related to RunnableScheduledFuture and meter persistance

Hi,

On linux and cassandra version 2.0.2 I had an OOM after a heavy load and then some (15 ) days of idle running (not exactly idle but very very low activity).
Two out of a 4 machine cluster had this OOM.

I checked the heap dump (9GB) and that tells me :

One instance of "java.util.concurrent.ScheduledThreadPoolExecutor" loaded by "<system class loader>" occupies 8.927.175.368 (94,53%) bytes. The instance is referenced by org.apache.cassandra.io.sstable.SSTableReader @ 0x7fadf89e0 , loaded by "sun.misc.Launcher$AppClassLoader @ 0x683e6ad30". The memory is accumulated in one instance of "java.util.concurrent.RunnableScheduledFuture[]" loaded by "<system class loader>".

So I checked the SSTableReader instance and found out the 'ScheduledThreadPoolExecutor syncExecutor ' object is holding about 600k of ScheduledFutureTasks.
According to the code on SSTableReader these tasks must have been created by the code line syncExecutor.scheduleAtFixedRate. That means that none of these tasks ever get scheduled because some (and only one) initial task is probably blocking.
But then again, the one thread to execute these tasks, seems to be in a 'normal' state (at time of OOM) and is executing with a stack trace pasted below :

Thread 0x696777eb8
  at org.apache.cassandra.db.AtomicSortedColumns$1.create(Lorg/apache/cassandra/config/CFMetaData;Z)Lorg/apache/cassandra/db/AtomicSortedColumns; (AtomicSortedColumns.java:58)
  at org.apache.cassandra.db.AtomicSortedColumns$1.create(Lorg/apache/cassandra/config/CFMetaData;Z)Lorg/apache/cassandra/db/ColumnFamily; (AtomicSortedColumns.java:55)
  at org.apache.cassandra.db.ColumnFamily.cloneMeShallow(Lorg/apache/cassandra/db/ColumnFamily$Factory;Z)Lorg/apache/cassandra/db/ColumnFamily; (ColumnFamily.java:70)
  at org.apache.cassandra.db.Memtable.resolve(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V (Memtable.java:187)
  at org.apache.cassandra.db.Memtable.put(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V (Memtable.java:158)
  at org.apache.cassandra.db.ColumnFamilyStore.apply(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V (ColumnFamilyStore.java:840)
  at org.apache.cassandra.db.Keyspace.apply(Lorg/apache/cassandra/db/RowMutation;ZZ)V (Keyspace.java:373)
  at org.apache.cassandra.db.Keyspace.apply(Lorg/apache/cassandra/db/RowMutation;Z)V (Keyspace.java:338)
  at org.apache.cassandra.db.RowMutation.apply()V (RowMutation.java:201)
  at org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(Lorg/apache/cassandra/service/QueryState;)Lorg/apache/cassandra/transport/messages/ResultMessage; (ModificationStatement.java:477)
  at org.apache.cassandra.cql3.QueryProcessor.processInternal(Ljava/lang/String;)Lorg/apache/cassandra/cql3/UntypedResultSet; (QueryProcessor.java:178)
  at org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(Ljava/lang/String;Ljava/lang/String;ILorg/apache/cassandra/metrics/RestorableMeter;)V (SystemKeyspace.java:938)
  at org.apache.cassandra.io.sstable.SSTableReader$2.run()V (SSTableReader.java:342)
  at java.util.concurrent.Executors$RunnableAdapter.call()Ljava/lang/Object; (Executors.java:471)
  at java.util.concurrent.FutureTask.runAndReset()Z (FutureTask.java:304)
  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(Ljava/util/concurrent/ScheduledThreadPoolExecutor$ScheduledFutureTask;)Z (ScheduledThreadPoolExecutor.java:178)
  at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run()V (ScheduledThreadPoolExecutor.java:293)
  at java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V (ThreadPoolExecutor.java:1145)
  at java.util.concurrent.ThreadPoolExecutor$Worker.run()V (ThreadPoolExecutor.java:615)
  at java.lang.Thread.run()V (Thread.java:724)


Since each of these tasks are throttled by meterSyncThrottle.acquire() I suspect that the RateLimiter is causing a delay. The RateLimiter instance attributes are :
Type|Name                |Value
long|nextFreeTicketMicros|3016022567383
double|maxPermits|100.0
double|storedPermits|99.0
long|offsetNanos|334676357831746

I guess that these attributes will practically result in a blocking behavior, resulting in the OOM ...

Is there someone that can make sense out of it?
I hope this helps in finding out what the reason is for this and maybe could be avoided in the future. I still have the heap dump, so I can always pass more information if needed.

Regards,

Ignace Desimpel

Re: OOM after some days related to RunnableScheduledFuture and meter persistance

Posted by Tyler Hobbs <ty...@datastax.com>.
I believe this is https://issues.apache.org/jira/browse/CASSANDRA-6358,
which was fixed in 2.0.3.


On Wed, Jan 8, 2014 at 7:15 AM, Desimpel, Ignace <Ignace.Desimpel@nuance.com
> wrote:

>  Hi,
>
>
>
> On linux and cassandra version 2.0.2 I had an OOM after a heavy load and
> then some (15 ) days of idle running (not exactly idle but very very low
> activity).
>
> Two out of a 4 machine cluster had this OOM.
>
>
>
> I checked the heap dump (9GB) and that tells me :
>
>
>
> One instance of *"java.util.concurrent.ScheduledThreadPoolExecutor"*loaded by *"<system
> class loader>"* occupies *8.927.175.368 (94,53%)* bytes. The instance is
> referenced by *org.apache.cassandra.io.sstable.SSTableReader @
> 0x7fadf89e0* , loaded by *"sun.misc.Launcher$AppClassLoader @
> 0x683e6ad30"*. The memory is accumulated in one instance of
> *"java.util.concurrent.RunnableScheduledFuture[]"* loaded by *"<system
> class loader>"*.
>
>
>
> So I checked the SSTableReader instance and found out the
> ‘ScheduledThreadPoolExecutor syncExecutor ‘ object is holding about 600k of
> ScheduledFutureTasks.
>
> According to the code on SSTableReader these tasks must have been created
> by the code line syncExecutor.scheduleAtFixedRate. That means that none of
> these tasks ever get scheduled because some (and only one) initial task is
> probably blocking.
>
> But then again, the one thread to execute these tasks, seems to be in a
> ‘normal’ state (at time of OOM) and is executing with a stack trace pasted
> below :
>
>
>
> Thread 0x696777eb8
>
>   at
> org.apache.cassandra.db.AtomicSortedColumns$1.create(Lorg/apache/cassandra/config/CFMetaData;Z)Lorg/apache/cassandra/db/AtomicSortedColumns;
> (AtomicSortedColumns.java:58)
>
>   at
> org.apache.cassandra.db.AtomicSortedColumns$1.create(Lorg/apache/cassandra/config/CFMetaData;Z)Lorg/apache/cassandra/db/ColumnFamily;
> (AtomicSortedColumns.java:55)
>
>   at
> org.apache.cassandra.db.ColumnFamily.cloneMeShallow(Lorg/apache/cassandra/db/ColumnFamily$Factory;Z)Lorg/apache/cassandra/db/ColumnFamily;
> (ColumnFamily.java:70)
>
>   at
> org.apache.cassandra.db.Memtable.resolve(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V
> (Memtable.java:187)
>
>   at
> org.apache.cassandra.db.Memtable.put(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V
> (Memtable.java:158)
>
>   at
> org.apache.cassandra.db.ColumnFamilyStore.apply(Lorg/apache/cassandra/db/DecoratedKey;Lorg/apache/cassandra/db/ColumnFamily;Lorg/apache/cassandra/db/index/SecondaryIndexManager$Updater;)V
> (ColumnFamilyStore.java:840)
>
>   at
> org.apache.cassandra.db.Keyspace.apply(Lorg/apache/cassandra/db/RowMutation;ZZ)V
> (Keyspace.java:373)
>
>   at
> org.apache.cassandra.db.Keyspace.apply(Lorg/apache/cassandra/db/RowMutation;Z)V
> (Keyspace.java:338)
>
>   at org.apache.cassandra.db.RowMutation.apply()V (RowMutation.java:201)
>
>   at
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(Lorg/apache/cassandra/service/QueryState;)Lorg/apache/cassandra/transport/messages/ResultMessage;
> (ModificationStatement.java:477)
>
>   at
> org.apache.cassandra.cql3.QueryProcessor.processInternal(Ljava/lang/String;)Lorg/apache/cassandra/cql3/UntypedResultSet;
> (QueryProcessor.java:178)
>
>   at
> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(Ljava/lang/String;Ljava/lang/String;ILorg/apache/cassandra/metrics/RestorableMeter;)V
> (SystemKeyspace.java:938)
>
>   at org.apache.cassandra.io.sstable.SSTableReader$2.run()V
> (SSTableReader.java:342)
>
>   at
> java.util.concurrent.Executors$RunnableAdapter.call()Ljava/lang/Object;
> (Executors.java:471)
>
>   at java.util.concurrent.FutureTask.runAndReset()Z (FutureTask.java:304)
>
>   at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(Ljava/util/concurrent/ScheduledThreadPoolExecutor$ScheduledFutureTask;)Z
> (ScheduledThreadPoolExecutor.java:178)
>
>   at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run()V
> (ScheduledThreadPoolExecutor.java:293)
>
>   at
> java.util.concurrent.ThreadPoolExecutor.runWorker(Ljava/util/concurrent/ThreadPoolExecutor$Worker;)V
> (ThreadPoolExecutor.java:1145)
>
>   at java.util.concurrent.ThreadPoolExecutor$Worker.run()V
> (ThreadPoolExecutor.java:615)
>
>   at java.lang.Thread.run()V (Thread.java:724)
>
>
>
>
>
> Since each of these tasks are throttled by meterSyncThrottle.acquire() I
> suspect that the RateLimiter is causing a delay. The RateLimiter instance
> attributes are :
>
> Type|Name                |Value
>
> long|nextFreeTicketMicros|3016022567383
>
> double|maxPermits|100.0
>
> double|storedPermits|99.0
>
> long|offsetNanos|334676357831746
>
>
>
> I guess that these attributes will practically result in a blocking
> behavior, resulting in the OOM …
>
>
>
> Is there someone that can make sense out of it?
>
> I hope this helps in finding out what the reason is for this and maybe
> could be avoided in the future. I still have the heap dump, so I can always
> pass more information if needed.
>
>
>
> Regards,
>
>
>
> Ignace Desimpel
>



-- 
Tyler Hobbs
DataStax <http://datastax.com/>