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

Sam Tunnicliffe updated CASSANDRA-10181:
----------------------------------------
    Component/s: Local Write-Read Paths
                 CQL

> Deadlock flushing tables with CUSTOM indexes
> --------------------------------------------
>
>                 Key: CASSANDRA-10181
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-10181
>             Project: Cassandra
>          Issue Type: Bug
>          Components: CQL, Local Write-Read Paths
>            Reporter: Tyler Hobbs
>            Assignee: Sam Tunnicliffe
>             Fix For: 3.0 beta 2
>
>         Attachments: flush-deadlock-repro.txt
>
>
> In 3.0, if a table with a CUSTOM secondary index is force flushed, Cassandra 
> will deadlock while attempting to perform a blocking flush on the tables 
> backing the secondary indexes.
> The basic problem is that the base table's post-flush task ends up waiting on 
> the post-flush task for the secondary index to complete.  However, since the 
> post-flush executor is single-threaded, this results in a deadlock.
> Here's the partial stacktrace for the base table part of this (line numbers 
> may not be 100% accurate):
> {noformat}
> org.apache.cassandra.db.ColumnFamilyStore.forceBlockingFlush(ColumnFamilyStore.java:927)
>       at 
> org.apache.cassandra.index.internal.CustomIndex.lambda$getBlockingFlushTask$0(VertexCentricIndex.java:114)
>       at 
> org.apache.cassandra.index.internal.CustomIndex$$Lambda$95/057902870.call(Unknown
>  Source)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> com.google.common.util.concurrent.MoreExecutors$DirectExecutorService.execute(MoreExecutors.java:299)
>       at 
> java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:134)
>       at 
> com.google.common.util.concurrent.AbstractListeningExecutorService.submit(AbstractListeningExecutorService.java:58)
>       at 
> com.google.common.util.concurrent.AbstractListeningExecutorService.submit(AbstractListeningExecutorService.java:37)
>       at 
> org.apache.cassandra.index.SecondaryIndexManager.lambda$executeAllBlocking$39(SecondaryIndexManager.java:896)
>       at 
> org.apache.cassandra.index.SecondaryIndexManager$$Lambda$94/25774682.accept(Unknown
>  Source)
>       at 
> java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1374)
>       at 
> java.util.stream.ReferencePipeline$Head.forEach(ReferencePipeline.java:580)
>       at 
> org.apache.cassandra.index.SecondaryIndexManager.executeAllBlocking(SecondaryIndexManager.java:893)
>       at 
> org.apache.cassandra.index.SecondaryIndexManager.flushIndexesBlocking(SecondaryIndexManager.java:346)
>       at 
> org.apache.cassandra.index.SecondaryIndexManager.flushAllCustomIndexesBlocking(SecondaryIndexManager.java:358)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore$PostFlush.run(ColumnFamilyStore.java:960)
> {noformat}
> First, note that the base of this stacktrace is in CFS$PostFlush.run(), which 
> means it's running on the post-flush executor.  When 
> {{CFS.forceBlockingFlush()}} is called on the secondary index table, we end 
> up blocking on another task that's submitted to the post-flush executor.  
> Since that executor is single-threaded and is already running the base table 
> task, this results in deadlock.
> The attached patch includes a unit test and custom secondary index class 
> (basically just KeysIndex) to reproduce the issue.



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

Reply via email to