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] [Comment Edited] (CASSANDRA-7275) Errors in FlushRunnable may leave threads hung
Date Tue, 16 Dec 2014 23:51:14 GMT

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

Pavel Yaskevich edited comment on CASSANDRA-7275 at 12/16/14 11:50 PM:
-----------------------------------------------------------------------

The problem is that there is no way to tell if hard-link problem is a actual fs/disk problem
or programming error, right now it looks like a programming error because it snapshot tries
to create duplicate hard-link to the same file as I mentioned in the CASSANDRA-8476 so if
there is no way to tell how reasonable is it to enforce shutdown or any rule from "disk_failure_policy"?

bq. If it's our bug, then you may need a temporary patch while we figure out the cause, but
I still don't think that kind of // this shouldn't happen code should be shipped officially.

If it's your problem it's my problem as well, we have a work-around for now (as I guess most
of the people do) but my intention in this ticket to fix this problem for good instead of
just fixing the symptom of it (being aforementioned "duplicate hard-link" problem).


was (Author: xedin):
The problem is that there is no way to tell if hard-link problem is a actual fs/disk problem
or programming error, right now it looks like a programming error because it snapshot tries
to create duplicate hard-link to the same file as I mentioned in the CASSANDRA-8476 so if
there is no way to tell how reasonable is it to enforce shutdown or any rule from "disk_failure_policy"?

bq. If it's our bug, then you may need a temporary patch while we figure out the cause, but
I still don't think that kind of // this shouldn't happen code should be shipped officially.

If it's your problem it's my problem as well, we can work around for now (as I guess most
of the people do) but my intention in this ticket to fix this problem for good instead of
just fixing the symptom of it (being aforementioned "duplicate hard-link" problem).

> 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