[ https://issues.apache.org/jira/browse/CASSANDRA-7275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14248554#comment-14248554 ]
Benedict commented on CASSANDRA-7275: ------------------------------------- bq. 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. Well, let's try and address each problem independently. A data induced bug that can occur across many nodes simultaneously is likely to occur repeatedly and cause the cluster to degrade probably quite rapidly, and will likely occur on all owners of a given token at once. Coupled with the stop-gap measures we're discussing might well run the risk of actual data loss or data corruption cross-cluster. Read repair would _not_ help for such a data bug, since none of the nodes would be in a safe state. However the transient file system problems you're encountering would be helped by reattempting the flush. So, an initial and completely safe approach would be to retry a few times and _then_ crash the server (possibly with some random waiting involved to avoid a disastrous cascade of cluster-wide death). Wasting work isn't really a big problem if the system cannot make progress without this success, so I don't see a downside on that front. It's possible if, once this fails, we could negotiate a safe crash with our peers, so that if there is a data bug at most one replica dies, the operator is well aware of the problem, but the cluster continues to operate. Although this is difficult with vnodes, and perhaps a little contrived for the current state of c*. Separately, we can look into perhaps weakening our constraints in various ways. The big issue you raise is that compaction is specifically held up. There seem to be two things we can do to help this: 1) We can make the dependency queue for marking commit log records unused table-specific, so that compactions only get held up if there has been an error on the compaction queue; 2) We can report these exceptions back to the waiter on the Future result, and this waiter can choose how to behave. If, say, the memtable of a system column family that can be worked-around fails to flush (for instance, compactions_in_progress) then instead of retrying, it can simply take some other action to ensure the system continues to make safe progress. If a data table fails to flush it can attempt to retry. 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 am very -1 on introducing a change that knowingly produces a complex failure condition that will not be widely known or understood, but I may be alone on that. > 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)