Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 15F7196F3 for ; Sat, 13 Dec 2014 04:14:14 +0000 (UTC) Received: (qmail 23481 invoked by uid 500); 13 Dec 2014 04:14:14 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 23464 invoked by uid 500); 13 Dec 2014 04:14:13 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 23448 invoked by uid 99); 13 Dec 2014 04:14:13 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 13 Dec 2014 04:14:13 +0000 Date: Sat, 13 Dec 2014 04:14:13 +0000 (UTC) From: "Pavel Yaskevich (JIRA)" To: commits@cassandra.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (CASSANDRA-8476) RE in writeSortedContents or replaceFlushed blocks compaction threads indefinitely. MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/CASSANDRA-8476?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14245174#comment-14245174 ] Pavel Yaskevich commented on CASSANDRA-8476: -------------------------------------------- If we don't and there is a problem in replaceFlushed it results in at least whole compaction freeze so the only way out is to restart process, we have to figure out a way to avoid truncating commitlog and still countdown that latch on problem, e.g. propagate compilation status or similar, we can't just ignore this. > 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 > > Attachments: CASSANDRA-8476.patch > > > 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 /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)