You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jonathan Ellis (JIRA)" <ji...@apache.org> on 2014/12/16 23:42:15 UTC

[jira] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung

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

Jonathan Ellis edited comment on CASSANDRA-7275 at 12/16/14 10:41 PM:
----------------------------------------------------------------------

bq. consider there could be a programming error in the flush of the memtable or replacing flushed one

I don't know that hand waving about potential bugs gets us anywhere.  There could be programming errors anywhere, including in "mark the segment flushed when it wasn't" panic mode.  The right solution to bugs is QA, not hoping that you can guess where unexpected exceptions will happen and provide a safety net.

Edit: which is to say that I think the right scope here is, "what do we do if we can't flush because of an i/o error," which is an expected condition that C* should be able to cope with cleanly.


was (Author: jbellis):
bq. consider there could be a programming error in the flush of the memtable or replacing flushed one

I don't know that hand waving about potential bugs gets us anywhere.  There could be programming errors anywhere, including in "mark the segment flushed when it wasn't" panic mode.  The right solution to bugs is QA, not hoping that you can guess where unexpected exceptions will happen and provide a safety net.

> Errors in FlushRunnable may leave threads hung
> ----------------------------------------------
>
>                 Key: CASSANDRA-7275
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7275
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Tyler Hobbs
>            Assignee: Pavel Yaskevich
>            Priority: Minor
>             Fix For: 2.0.12
>
>         Attachments: 0001-Move-latch.countDown-into-finally-block.patch, 7252-2.0-v2.txt, CASSANDRA-7275-flush-info.patch
>
>
> In Memtable.FlushRunnable, the CountDownLatch will never be counted down if there are errors, which results in hanging any threads that are waiting for the flush to complete.  For example, an error like this causes the problem:
> {noformat}
> ERROR [FlushWriter:474] 2014-05-20 12:10:31,137 CassandraDaemon.java (line 198) Exception in thread Thread[FlushWriter:474,5,main]
> java.lang.IllegalArgumentException
>     at java.nio.Buffer.position(Unknown Source)
>     at org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:64)
>     at org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72)
>     at org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138)
>     at org.apache.cassandra.io.sstable.ColumnNameHelper.minComponents(ColumnNameHelper.java:103)
>     at org.apache.cassandra.db.ColumnFamily.getColumnStats(ColumnFamily.java:439)
>     at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:194)
>     at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:397)
>     at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
>     at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>     at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>     at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source)
>     at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>     at java.lang.Thread.run(Unknown Source)
> {noformat}



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