Uploaded image for project: 'Apache Cassandra'
  1. Apache Cassandra
  2. CASSANDRA-8884

Opening a non-system keyspace before first accessing the system keyspace results in deadlock

Details

    • Bug
    • Status: Resolved
    • Low
    • Resolution: Cannot Reproduce
    • None
    • None
    • None
    • Low

    Description

      I created a writer like this:

      val writer = CQLSSTableWriter.builder()
            .forTable(tableDef.cql)
            .using(insertStatement)
            .withPartitioner(partitioner)
            .inDirectory(outputDirectory)    
            .withBufferSizeInMB(bufferSizeInMB)
            .build()
      

      Then I'm trying to write a row with addRow and it blocks forever.
      Everything related to CQLSSTableWriter, including its creation, is happening in only one thread.

      "SSTableBatchOpen:3" daemon prio=10 tid=0x00007f4b399d7000 nid=0x4778 waiting for monitor entry [0x00007f4b240a7000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
      	- waiting to lock <0x00000000e35fd6d0> (a java.lang.Class for org.apache.cassandra.db.Keyspace)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:99)
      	at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:1464)
      	at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:517)
      	at org.apache.cassandra.cql3.QueryProcessor.parseStatement(QueryProcessor.java:265)
      	at org.apache.cassandra.cql3.QueryProcessor.prepareInternal(QueryProcessor.java:306)
      	at org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:316)
      	at org.apache.cassandra.db.SystemKeyspace.getSSTableReadMeter(SystemKeyspace.java:910)
      	at org.apache.cassandra.io.sstable.SSTableReader.<init>(SSTableReader.java:561)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:433)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:343)
      	at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:480)
      	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:745)
      
      "SSTableBatchOpen:2" daemon prio=10 tid=0x00007f4b399e7800 nid=0x4777 waiting for monitor entry [0x00007f4b23ca3000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
      	- waiting to lock <0x00000000e35fd6d0> (a java.lang.Class for org.apache.cassandra.db.Keyspace)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:99)
      	at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:1464)
      	at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:517)
      	at org.apache.cassandra.cql3.QueryProcessor.parseStatement(QueryProcessor.java:265)
      	at org.apache.cassandra.cql3.QueryProcessor.prepareInternal(QueryProcessor.java:306)
      	at org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:316)
      	at org.apache.cassandra.db.SystemKeyspace.getSSTableReadMeter(SystemKeyspace.java:910)
      	at org.apache.cassandra.io.sstable.SSTableReader.<init>(SSTableReader.java:561)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:433)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:343)
      	at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:480)
      	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:745)
      
      "SSTableBatchOpen:1" daemon prio=10 tid=0x00007f4b399e7000 nid=0x4776 waiting for monitor entry [0x00007f4b2359d000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
      	- waiting to lock <0x00000000e35fd6d0> (a java.lang.Class for org.apache.cassandra.db.Keyspace)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:99)
      	at org.apache.cassandra.cql3.statements.SelectStatement$RawStatement.prepare(SelectStatement.java:1464)
      	at org.apache.cassandra.cql3.QueryProcessor.getStatement(QueryProcessor.java:517)
      	at org.apache.cassandra.cql3.QueryProcessor.parseStatement(QueryProcessor.java:265)
      	at org.apache.cassandra.cql3.QueryProcessor.prepareInternal(QueryProcessor.java:306)
      	at org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:316)
      	at org.apache.cassandra.db.SystemKeyspace.getSSTableReadMeter(SystemKeyspace.java:910)
      	at org.apache.cassandra.io.sstable.SSTableReader.<init>(SSTableReader.java:561)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:433)
      	at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:343)
      	at org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:480)
      	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:745)
      
      "Executor task launch worker-3" daemon prio=10 tid=0x00007f4b38ce6000 nid=0x472c waiting on condition [0x00007f4b26deb000]
         java.lang.Thread.State: TIMED_WAITING (parking)
      	at sun.misc.Unsafe.park(Native Method)
      	- parking to wait for  <0x00000000fac03b40> (a java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
      	at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226)
      	at java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2082)
      	at java.util.concurrent.ThreadPoolExecutor.awaitTermination(ThreadPoolExecutor.java:1468)
      	at org.apache.cassandra.io.sstable.SSTableReader.openAll(SSTableReader.java:496)
      	at org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:321)
      	at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:478)
      	- locked <0x00000000e374a2b0> (a java.lang.Class for org.apache.cassandra.db.ColumnFamilyStore)
      	at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:449)
      	at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:327)
      	at org.apache.cassandra.db.Keyspace.<init>(Keyspace.java:280)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:122)
      	- locked <0x00000000e35fd6d0> (a java.lang.Class for org.apache.cassandra.db.Keyspace)
      	at org.apache.cassandra.db.Keyspace.open(Keyspace.java:99)
      	at org.apache.cassandra.cql3.statements.UpdateStatement.addUpdateForKey(UpdateStatement.java:101)
      	at org.apache.cassandra.io.sstable.CQLSSTableWriter.rawAddRow(CQLSSTableWriter.java:225)
      	at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:144)
      	at org.apache.cassandra.io.sstable.CQLSSTableWriter.addRow(CQLSSTableWriter.java:119)
      	at com.datastax.bdp.spark.writer.BulkTableWriter$$anonfun$write$1.apply(BulkTableWriter.scala:101)
      	at com.datastax.bdp.spark.writer.BulkTableWriter$$anonfun$write$1.apply(BulkTableWriter.scala:97)
      	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
      	at org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28)
      	at com.datastax.bdp.spark.writer.BulkTableWriter.write(BulkTableWriter.scala:97)
      	- locked <0x00000000e0f00b78> (a com.datastax.bdp.spark.writer.BulkTableWriter$)
      	at com.datastax.bdp.spark.writer.BulkTableWriter$$anonfun$saveRddToCassandra$1.apply(BulkTableWriter.scala:144)
      	at com.datastax.bdp.spark.writer.BulkTableWriter$$anonfun$saveRddToCassandra$1.apply(BulkTableWriter.scala:144)
      	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
      	at org.apache.spark.scheduler.Task.run(Task.scala:56)
      	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:200)
      	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:745)
      

      Attachments

        1. bulk.jstack
          47 kB
          Piotr Kolaczkowski

        Issue Links

          Activity

            BTW: I'm not sure which C* version exactly it is, because C* reports it in logs as 2.1.3-SNAPSHOT and the library version is 2.1.2.157. I'll ask DSE core team where they got it from (I hope it is not a fork).

            pkolaczk Piotr Kolaczkowski added a comment - BTW: I'm not sure which C* version exactly it is, because C* reports it in logs as 2.1.3-SNAPSHOT and the library version is 2.1.2.157. I'll ask DSE core team where they got it from (I hope it is not a fork).
            pkolaczk Piotr Kolaczkowski added a comment - - edited

            Got the same with a newer version of DSE which includes 2.1.2.174 build. This is not a C* fork, but a development version cut off 2.1 branch.

            pkolaczk Piotr Kolaczkowski added a comment - - edited Got the same with a newer version of DSE which includes 2.1.2.174 build. This is not a C* fork, but a development version cut off 2.1 branch.

            Found a git commit for the build I'm using: cdcf6461a309a4d8b6e4be1dc56cba453eef0d45

            pkolaczk Piotr Kolaczkowski added a comment - Found a git commit for the build I'm using: cdcf6461a309a4d8b6e4be1dc56cba453eef0d45
            pkolaczk Piotr Kolaczkowski added a comment - - edited

            Found a workaround:

            Adding

                Keyspace.open("system")
            

            before the first use of CQLSStableWriter fixes it.

            pkolaczk Piotr Kolaczkowski added a comment - - edited Found a workaround: Adding Keyspace.open( "system" ) before the first use of CQLSStableWriter fixes it.
            blerer Benjamin Lerer added a comment -

            @pkolaczk Could you provides a more detailed example? It would help to know for which table you are trying to build an SSTable and if you are calling CQLSSTableWriter inside a running Cassandra instance or outside (in client mode).

            blerer Benjamin Lerer added a comment - @pkolaczk Could you provides a more detailed example? It would help to know for which table you are trying to build an SSTable and if you are calling CQLSSTableWriter inside a running Cassandra instance or outside (in client mode).
            blerer Benjamin Lerer added a comment -

            I am not able to reproduce that problem. My code use CQLSSTableBuilder without calling first Keyspace.open("system") and do not trigger any deadlock.
            It would help to have a simple program that I can use to reproduce the problem.

            blerer Benjamin Lerer added a comment - I am not able to reproduce that problem. My code use CQLSSTableBuilder without calling first Keyspace.open("system") and do not trigger any deadlock. It would help to have a simple program that I can use to reproduce the problem.

            I'm running my code outside of Cassandra instance.
            I'll create a simple program to demonstrate the issue.

            pkolaczk Piotr Kolaczkowski added a comment - I'm running my code outside of Cassandra instance. I'll create a simple program to demonstrate the issue.
            blerer Benjamin Lerer added a comment - - edited

            pkolaczk any update ?

            blerer Benjamin Lerer added a comment - - edited pkolaczk any update ?
            xuzhongxing Xu Zhongxing added a comment -

            Hi Piotr,

            I had the same problem. After adding Keyspace.open("system"), the program does not exit.
            What do I need to do to "close" the Keyspace?

            xuzhongxing Xu Zhongxing added a comment - Hi Piotr, I had the same problem. After adding Keyspace.open("system"), the program does not exit. What do I need to do to "close" the Keyspace?

            There hasn't been any update since March and we don't know how to reproduce so closing as such. Please feel free to reopen with reproduction steps if that's still an issue.

            slebresne Sylvain Lebresne added a comment - There hasn't been any update since March and we don't know how to reproduce so closing as such. Please feel free to reopen with reproduction steps if that's still an issue.

            People

              blerer Benjamin Lerer
              pkolaczk Piotr Kolaczkowski
              Benjamin Lerer
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: