incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aaron morton <aa...@thelastpickle.com>
Subject Re: Hanging after OutOfMemory exception
Date Wed, 20 Feb 2013 03:29:42 GMT
Nice. 
Can you please raise a ticket at https://issues.apache.org/jira/browse/CASSANDRA an include
the version you were using. 

Cheers

-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 19/02/2013, at 1:44 AM, "Desimpel, Ignace" <Ignace.Desimpel@nuance.com> wrote:

> Running Cassandra as an embedded service (not really related I think), and due to an
out-of-memory exception ( ‘wrong’ configuration from my part ), I could make the system
hang although I would have liked it to stop running.
>  
> Below I copied pieces of the stack trace that are related in my opinion. Attached is
a file with all parameters (java, machine, libs, …) and the stack trace.
> Hopefully this helps make the code better (or my code …)
>  
> Greets,
> Ignace
>  
> Here is my opinion :
>  
> Thread-174 is processing a System.exit() due to the uncaughtException handling in CassandraDaemon.
That System.exit() gets a lock and runs the ApplicationShutdownHooks.
>  
> The StorageService has a shutdownhook called StorageServiceShutdownHook. That hook is
calling CommitLog.instance.shutdownBlocking(), and that routine is (in my case) calling shutdown()
and awaitTermination() on a PeriodicCommitlogExecutorService. Therefore it is waiting on a
thread called COMMIT-LOG-WRITER to terminate.
> But that COMMIT-LOG-WRITER also has got an uncaughtException, and in response to that
it is calling System.exit(). But that call cannot return since “Thread-174” is already
holding a lock.
>  
> So the system is “hanging” on the System.exit due to a (multiple) OutOfMemory exception
>  
> "StorageServiceShutdownHook" Id=41 WAITING on java.lang.Thread@79dc2ba2
>                 at java.lang.Object.wait(Native Method)
>                 -  waiting on java.lang.Thread@79dc2ba2
>                 at java.lang.Thread.join(Thread.java:1186)
>                 at java.lang.Thread.join(Thread.java:1239)
>                 at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService.awaitTermination(PeriodicCommitLogExecutorService.java:131)
>                 at org.apache.cassandra.db.commitlog.CommitLog.shutdownBlocking(CommitLog.java:356)
>                 at org.apache.cassandra.service.StorageService$1.runMayThrow(StorageService.java:496)
>                 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>                 at java.lang.Thread.run(Thread.java:662)
>  
> "Commitlog Shutdown" Id=22868 TIMED_WAITING
>                 at java.lang.Thread.sleep(Native Method)
>                 at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$4.runMayThrow(PeriodicCommitLogExecutorService.java:122)
>                 at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>                 at java.lang.Thread.run(Thread.java:662)
>  
> "COMMIT-LOG-WRITER" Id=39 BLOCKED on java.lang.Class@1cc7b00c owned by "Thread-174" Id=242
>                 at java.lang.Shutdown.exit(Shutdown.java:168)
>                 -  blocked on java.lang.Class@1cc7b00c
>                 at java.lang.Runtime.exit(Runtime.java:90)
>                 at java.lang.System.exit(System.java:904)
>                 at org.apache.cassandra.service.CassandraDaemon$1.uncaughtException(CassandraDaemon.java:138)
>                 at java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:981)
>                 at java.lang.ThreadGroup.uncaughtException(ThreadGroup.java:976)
>                 at java.lang.Thread.dispatchUncaughtException(Thread.java:1874)
>  
> "Thread-174" Id=242 WAITING on java.lang.Thread@458e439a
>                 at java.lang.Object.wait(Native Method)
>                 -  waiting on java.lang.Thread@458e439a
>                 at java.lang.Thread.join(Thread.java:1186)
>                 at java.lang.Thread.join(Thread.java:1239)
>                 at java.lang.ApplicationShutdownHooks.runHooks(ApplicationShutdownHooks.java:79)
>                 at java.lang.ApplicationShutdownHooks$1.run(ApplicationShutdownHooks.java:24)
>                 at java.lang.Shutdown.runHooks(Shutdown.java:79)
>                 at java.lang.Shutdown.sequence(Shutdown.java:123)
>                 at java.lang.Shutdown.exit(Shutdown.java:168)
>                 -  locked java.lang.Class@1cc7b00c
>                 ...
> <CassHangs.txt>


Mime
View raw message