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 22:10:14 GMT

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

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

I understand it might be hard for you, Benedict, but just consider there could be a programming
error in the flush of the memtable or replacing flushed one, which is only triggered when
metadata about compaction is written back at the end of that compaction e.g. CompactionTask.runMayThrow()
L225, e.g. error mentioned in the description or "duplicate hard-link failure" or something
similar which has nothing to do with the underlaying (file-)system which means that #1 suggestion
is not going to help because compaction is blocked in SystemKeyspace.finishCompaction() and
flush retry is not going to help because it will just fail again and again trying to flush
the same data. As an end user I would prefer that nobody actually takes a decision to fail
on the floor for me except me because it means data loss even when problem is not affecting
actual write/read path, I would be fine though to fail on FS\{Read, Write\}Error if user explicitly
sets it to fail on I/O errors (e.g. "disk_failure_policy", it is like of your #2 but not exactly)
otherwise I would rather get notified in the log and carry on so I can take informed decision
on my next actions.

bq. Eventually, if it cannot recover safely, it should die though, as there will need to be
some operator involvement and the reality is not everybody monitors their log files.

I'm going to ignore this argument until you actually have experience of running Cassandra
in production, otherwise it's the same as talking to the wall.

bq. I'm much more comfortable with "things die if something goes catastrophically wrong" than
"things start returning nonsense on reads" which is what happens if we mark something flushed
that actually wasn't.

I remember it was already the same when the disk is full in the DSE, did people actually have
fun restoring cluster after it went completely dark? I'm also *not* saying that we shouldn't
fail on FS\{Read, Write\}Error if "disk_failure_policy" says otherwise.

> 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