[
https://issues.apache.org/jira/browse/CASSANDRA-7275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14481349#comment-14481349
]
Jeremiah Jordan commented on CASSANDRA-7275:
--------------------------------------------
Just had a java.io.SyncFailedException cause this. After the exception
MemtablePostFlush was stuck.
{noformat}
ERROR [MemtableFlushWriter:6] 2015-04-03 01:57:06,973 CassandraDaemon.java:167
- Exception in thread Thread[MemtableFlushWriter:6,5,main]
org.apache.cassandra.io.FSWriteError: java.io.SyncFailedException: sync failed
at
org.apache.cassandra.io.util.SequentialWriter.syncDataOnlyInternal(SequentialWriter.java:254)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.util.SequentialWriter.syncInternal(SequentialWriter.java:263)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.util.SequentialWriter.close(SequentialWriter.java:451)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:664)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.java:495)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.sstable.SSTableWriter.finish(SSTableWriter.java:448)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:440)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:435)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:377)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:327)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
~[guava-16.0.1.jar:na]
at
org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1097)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
~[na:1.8.0_40]
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
~[na:1.8.0_40]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_40]
Caused by: java.io.SyncFailedException: sync failed
at java.io.FileDescriptor.sync(Native Method) ~[na:1.8.0_40]
at
org.apache.cassandra.io.util.SequentialWriter.syncDataOnlyInternal(SequentialWriter.java:250)
~[cassandra-all-2.1.3.329.jar:2.1.3.329]
... 15 common frames omitted
{noformat}
> 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.15
>
> 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)