cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Stefania (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-9635) Silent startup failure with filesystem that does not support mmap
Date Mon, 29 Jun 2015 08:45:05 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-9635?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14605295#comment-14605295
] 

Stefania commented on CASSANDRA-9635:
-------------------------------------

Easiest way to reproduce the deadlock is to specify an insanely large {{commitlog_segment_size_in_mb}},
the exception is thrown 4 lines below and the behavior is the same, static deadlock. Problem
exists in 2.1 and 2.2 as well.

Proposed solution is a start-up check that ensures we will be able to create at least a segment
later on, if not fail fast and exit the process. A bonus would be to create a segment that
can be re-used later on. Problem is that the code is pretty different on all branches, especially
2.2 due to compression support, so this requires multiple patch versions. 

An alternative would be to give up after a timeout, if not segment can be retrieved from the
queue, but this is not fail fast and requires an additional config parameter to specify the
timeout.

An even simpler approach would be that we can mmap a file with the specified commit log size,
this is more generic and requires less work.

Before I start any work I would like to check with a commit log expert this the best approach?
cc [~blambov]

> Silent startup failure with filesystem that does not support mmap
> -----------------------------------------------------------------
>
>                 Key: CASSANDRA-9635
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-9635
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Kevin McLaughlin
>            Assignee: Stefania
>             Fix For: 2.0.x
>
>         Attachments: c_tdump.txt
>
>
> C* version 2.0.9.
> When running C* in virtualbox on OS X via boot2docker with the data directory on a shared
volume from the host system (vboxfs), C* fails to start without printing any errors.
> I do not know if C* is supposed to support filesystems that do not support mmap (does
not appear so), however, I think the failure exposes a static initialization deadlock (http://ternarysearch.blogspot.ru/2013/07/static-initialization-deadlock.html).
> I believe the virtualbox "bug" is https://www.virtualbox.org/ticket/819.
> Stacktrace of the deadlock is attached.  When placing a t.printStackTrace() between lines
115 and 116 in https://github.com/apache/cassandra/blob/cassandra-2.0.9/src/java/org/apache/cassandra/db/commitlog/CommitLogAllocator.java,
the stack trace at startup is:
> {quote}
> DEBUG 21:16:54,716 Creating new commit log segment /var/lib/cassandra/commitlog/CommitLog-3-1435007814714.log
> FSWriteError in /var/lib/cassandra/commitlog/CommitLog-3-1435007814714.log
>     at org.apache.cassandra.db.commitlog.CommitLogSegment.<init>(CommitLogSegment.java:143)
>     at org.apache.cassandra.db.commitlog.CommitLogSegment.freshSegment(CommitLogSegment.java:90)
>     at org.apache.cassandra.db.commitlog.CommitLogAllocator.createFreshSegment(CommitLogAllocator.java:263)
>     at org.apache.cassandra.db.commitlog.CommitLogAllocator.access$500(CommitLogAllocator.java:50)
>     at org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:109)
>     at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>     at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Invalid argument
>     at sun.nio.ch.FileChannelImpl.map0(Native Method)
>     at sun.nio.ch.FileChannelImpl.map(FileChannelImpl.java:893)
>     at org.apache.cassandra.db.commitlog.CommitLogSegment.<init>(CommitLogSegment.java:133)
>     ... 6 more
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message