cassandra-commits mailing list archives

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

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

Pavel Yaskevich commented on CASSANDRA-7275:
--------------------------------------------

There is an option to die on the I/O error and I'm happy to make it so we die if we got FSWriteError
or similar if requested by config.

bq. Generally we take the approach of dying if a non-recoverable error occurs, and while I
agree the risk of killing a whole cluster through a bug is suboptimal, we already run that
risk in a number of places in the codebase (current behaviour here included, just with less
alacrity). In my opinion this is preferable to potentially re-introducing dead data, or having
the complexity of safely keeping the process alive as a zombie, and ensuring that zombie doesn't
degrade cluster performance by hobbling instead of dying.

Here is your real world scenario, which we are hitting from time to time, right now if I/O
error occurs in the replaceFlushed (e.g. trying to create hard-link for system.compactions_in_progress)
all of the compaction threads are going to get blocked and performance is going to gradually
degrade until it gets to the point when alerts from compaction pending trigger, at that time
somebody has to (most luckily wake up) figure out what is going on and restart the node, once
it starts back up the amount of catching up it has to do in terms of the compaction is substantial.
This problem happens on the number of machines at the same time so if we were to kill the
nodes right when aforementioned error occurs (although it's not affecting actual flush or
compaction) that would mean that part of the ring just went dark and one just has to pray
that those nodes weren't neighbors, so in this case serve some stale reads (which is not even
the case if failure in in bookeeping CF) with error in the log is much better than loose portion
of the cluster for (possibly tens) minutes without any idea of what is going on.

In this situation I would rather ignore problems with book-keeping CFs or save CL segments
forget about it and/or bumping up read-repair chance at the same time.
 
Everybody who is running Cassandra or any other database/system wants a peace of mind that's
why regular repairs and all sorts of the alerting/monitoring systems are in-place, if there
is something in the log which indicates a problem it gives people time to think about their
next steps instead of chaotically trying to fix what ever mess we left on failure.

bq. Other than dying, periodically trying to re-flush and only keeling over when we run out
of room or have failed for a long period (possibly random? to avoid the tiny risk of bunching)
seems like a good idea.

This is not going to help if the problem data driven or external, you just going to trash
flusher threads without doing any useful work.

> 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