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

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

bq. I'm not sure this really improves the current state of affairs very much, 
and introduces a bug as well.

It improve the current state of affairs in the way that failures in the flush 
are not going to incur compaction freeze anymore.

bq. If we fail to flush and simply carry on without clearing the CL, then the 
host will retain the memtable it wanted to flush forever, leaving it in a 
potentially severely degraded state (increasing risk of exceeding heap limit, 
or possible failing to ever accept writes in 2.1 due to insufficient memory). 
If the same table has another flush backed up (or another is later scheduled) 
then we will also end up expiring the commit log records anyway, despite not 
having flushed successfully.

I think what we can do to prevent CL expiry is to mark it as discarded but 
without deleting actual file on disk, this way it can be replayed on start up 
and memtable flushes that follow are not going to delete any potentially 
unflushed data. 

bq. Either we need to reattempt the flush, prevent any further flushes on that 
column family from ever succeeding, or - more simply - kill the C* process.

Let's start with reattempting flush - we don't really have enough information 
to make a decision to re-attempt flushing as it can fail for the number of 
reasons, I/O error is just being one of them.
Killing C* process is harmful as if we have code problem in writeSortedContents 
or replaceFlushed code it would potentially result in shutdown of the whole 
cluster or at least of all of the neighbors sharing replica range.


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

Reply via email to