[ 
https://issues.apache.org/jira/browse/CASSANDRA-14908?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

C. Scott Andreas updated CASSANDRA-14908:
-----------------------------------------
    Component/s: Compaction

> Deadlock occurs when executing a file selection in levelcompact
> ---------------------------------------------------------------
>
>                 Key: CASSANDRA-14908
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-14908
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Compaction
>         Environment: version : cassandra 2.1.15
> jdk: 8
> os:suse
>            Reporter: wu taiyin
>            Priority: Major
>         Attachments: deadlock stack.txt
>
>
>  detailed exception stack as follows: 
> "CompactionExecutor:33616" #142049 daemon prio=1 os_prio=4 
> tid=0x00007f73244cc000 nid=0x1919a waiting for monitor entry 
> [0x00007fa94e13b000]
>  java.lang.Thread.State: {color:#FF0000}BLOCKED{color} (on object monitor)
>  at 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:265)
>  - waiting to lock <0x00007faa776209b0> (a 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
>  at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:517)
>  at org.apache.cassandra.db.DataTracker.replaceReaders(DataTracker.java:408)
>  at 
> org.apache.cassandra.db.DataTracker.replaceWithNewInstances(DataTracker.java:305)
>  at 
> org.apache.cassandra.io.sstable.SSTableRewriter.moveStarts(SSTableRewriter.java:337)
>  at 
> org.apache.cassandra.io.sstable.SSTableRewriter.maybeReopenEarly(SSTableRewriter.java:187)
>  at 
> org.apache.cassandra.io.sstable.SSTableRewriter.append(SSTableRewriter.java:126)
>  at 
> org.apache.cassandra.db.compaction.CompactionTask.runMayThrow(CompactionTask.java:197)
>  at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>  at 
> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:73)
>  at 
> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59)
>  at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:264)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  at java.lang.Thread.run(Thread.java:745)
> "CompactionExecutor:33615" #142048 daemon prio=1 os_prio=4 
> tid=0x00007f7324451800 nid=0x19199 runnable [0x00007fa7a096f000]
>  java.lang.Thread.State: RUNNABLE
>  at java.util.HashMap.hash(HashMap.java:338)
>  at java.util.HashMap.put(HashMap.java:611)
>  at java.util.HashSet.add(HashSet.java:219)
>  at java.util.AbstractCollection.addAll(AbstractCollection.java:344)
>  at java.util.HashSet.<init>(HashSet.java:119)
>  at com.google.common.collect.Sets.newHashSet(Sets.java:218)
>  at 
> {color:#FF0000}org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:307){color}
> {color:#FF0000} - locked <0x00007faa77620e70> (a 
> org.apache.cassandra.db.compaction.LeveledManifest){color}
> {color:#FF0000} at{color} 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getMaximalTask(LeveledCompactionStrategy.java:101)
>  at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:90)
>  - locked <0x00007faa77620e30> (a 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
>  at 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:84)
>  - locked <0x00007faa776209b0> (a 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
>  at 
> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:258)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  at java.lang.Thread.run(Thread.java:745)
> "MemtableFlushWriter:22715" #142123 daemon prio=5 os_prio=0 
> tid=0x00007f5a3d4ba000 nid=0x24063 waiting for monitor entry 
> [0x00007fa8337bd000]
>  java.lang.Thread.State: BLOCKED (on object monitor)
>  at 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:265)
>  - waiting to lock <0x00007faa776209b0> (a 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
>  at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:531)
>  at org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:179)
>  at 
> org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:234)
>  at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1521)
>  at 
> org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:336)
>  at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>  at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1127)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  at java.lang.Thread.run(Thread.java:745)
> "MemtableFlushWriter:22711" #142116 daemon prio=5 os_prio=0 
> tid=0x00007f5a3d338000 nid=0x21ccf waiting for monitor entry 
> [0x00007fa94e084000]
>  java.lang.Thread.State: BLOCKED (on object monitor)
>  at 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:265)
>  - waiting to lock <0x00007faa776209b0> (a 
> org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
>  at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:531)
>  at org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:179)
>  at 
> org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:234)
>  at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1521)
>  at 
> org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:336)
>  at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>  at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
>  at 
> org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1127)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  at java.lang.Thread.run(Thread.java:745)
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to