Pavel Yaskevich created CASSANDRA-8476:
------------------------------------------

             Summary: RE in writeSortedContents or replaceFlushed blocks 
compaction threads indefinitely.
                 Key: CASSANDRA-8476
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8476
             Project: Cassandra
          Issue Type: Bug
          Components: Core
            Reporter: Pavel Yaskevich
            Assignee: Pavel Yaskevich
             Fix For: 2.0.12


Encountered this problem while generating some test data, incremental backup 
failed to create hard-links to some of the of the system files (which is done 
at the end of each compaction):

Example of the RE stacktrace:
{noformat}
14/12/12 15:47:47 ERROR cassandra.SchemaLoader: Fatal exception in thread 
Thread[FlushWriter:5,5,main]
java.lang.RuntimeException: Tried to create duplicate hard link to 
<path>/cassandra/data/system/IndexInfo/backups/system-IndexInfo-jb-1-Index.db
        at 
org.apache.cassandra.io.util.FileUtils.createHardLink(FileUtils.java:75)
        at 
org.apache.cassandra.io.sstable.SSTableReader.createLinks(SSTableReader.java:1222)
        at 
org.apache.cassandra.db.DataTracker.maybeIncrementallyBackup(DataTracker.java:189)
        at 
org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:166)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:231)
        at 
org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1141)
        at 
org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:343)
        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(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)
14
{noformat}

jstack shows that CompactionExecutor threads are now blocked waiting on the 
flush future which will actually never decrement a latch.

{noformat}
"CompactionExecutor:125" daemon prio=5 tid=0x00007fb3a10da800 nid=0x13c43 
waiting on condition [0x000000012a900000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x000000071b669088> (a 
java.util.concurrent.FutureTask)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:425)
        at java.util.concurrent.FutureTask.get(FutureTask.java:187)
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:409)
        at 
org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:457)
        at 
org.apache.cassandra.db.SystemKeyspace.finishCompaction(SystemKeyspace.java:203)
        at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:225)
        at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
        at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)

"CompactionExecutor:124" daemon prio=5 tid=0x00007fb35cc09800 nid=0x13a2b 
waiting on condition [0x000000012934f000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000007ce4bf918> (a 
java.util.concurrent.FutureTask)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:425)
        at java.util.concurrent.FutureTask.get(FutureTask.java:187)
        at 
org.apache.cassandra.utils.FBUtilities.waitOnFuture(FBUtilities.java:409)
        at 
org.apache.cassandra.db.SystemKeyspace.forceBlockingFlush(SystemKeyspace.java:457)
        at 
org.apache.cassandra.db.SystemKeyspace.finishCompaction(SystemKeyspace.java:203)
        at 
org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:225)
        at 
org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at 
org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60)
        at 
org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
        at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
        at java.util.concurrent.FutureTask.run(FutureTask.java:262)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)
{noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to