cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung
Date Wed, 17 Dec 2014 11:17:14 GMT

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

Sylvain Lebresne commented on CASSANDRA-7275:
---------------------------------------------

The current behavior is that an unexpected flush error blocks any flush thereon. It does seems
to me that changing it so that it blocks only flushes for the column family on which there
was a problem (which is not exactly what the patch does, and I do agree with Benedict that
it does need to do that) is an improvement: if the problem happens for every CF then we're
no worst than currently, but if it's a one-time event it might leave time for operators to
take proper actions (of course, we should log a scary error, it's not something that should
be ignored). So maybe we can start there since we don't seem to agree on whether crashing
the node is an even better improvement?

As far as my own opinion goes, I do am not in favor of crashing in that case because again,
if you hold enough memtables in memory that your node become unresponsive, you're not really
worth off that if you had crashed it right away, but if the problem ends up impacting a low
traffic table (for instance a system table), you might be able to fix the problem in a way
that is less impactful for your cluster.

I'll note however that I would agree that if the error is a IO one, we should respect the
disk_failure_policy. And I don't know, maybe we need another failure policy (best_effort/crash)
for unexpected errors (aka bugs) that have the potential of destabilizing a node (I would
agree that adding this is pushing the problem to our users, but it appears not everyone has
the same idea on what is the best strategy, and there is maybe not a single good answer).


> 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)

Mime
View raw message