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

Piotr Kołaczkowski updated CASSANDRA-8884:
------------------------------------------
    Description: 
I created a writer like this:
{code}
val writer = CQLSSTableWriter.builder()
      .forTable(tableDef.cql)
      .using(insertStatement)
      .withPartitioner(partitioner)
      .inDirectory(outputDirectory)    
      .withBufferSizeInMB(bufferSizeInMB)
      .build()
{code}

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.

{noformat}
"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)
{noformat}



  was:
I created a writer like this:
{code}
val writer = CQLSSTableWriter.builder()
      .forTable(tableDef.cql)
      .using(insertStatement)
      .withPartitioner(partitioner)
      .inDirectory(outputDirectory)    
      .withBufferSizeInMB(bufferSizeInMB)
      .build()
{code}

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.

{noformat}
"SSTableBatchOpen:3" daemon prio=10 tid=0x00007f0051b86800 nid=0x100d waiting 
for monitor entry [0x00007f003aad1000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
        - waiting to lock <0x00000000e10dfaf0> (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:1439)
        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:520)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:393)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:303)
        at 
org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:439)
        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=0x00007f0051b85000 nid=0x100c waiting 
for monitor entry [0x00007f00398c0000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
        - waiting to lock <0x00000000e10dfaf0> (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:1439)
        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:520)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:393)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:303)
        at 
org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:439)
        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=0x00007f0051b0a000 nid=0x100b waiting 
for monitor entry [0x00007f003a6ce000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.cassandra.db.Keyspace.open(Keyspace.java:118)
        - waiting to lock <0x00000000e10dfaf0> (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:1439)
        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:520)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:393)
        at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:303)
        at 
org.apache.cassandra.io.sstable.SSTableReader$4.run(SSTableReader.java:439)
        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=0x00007f00500d4000 nid=0xfa3 
waiting on condition [0x00007f003d130000]
   java.lang.Thread.State: TIMED_WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000000f9ac66a0> (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:455)
        at 
org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:286)
        at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:443)
        - locked <0x00000000e10a0808> (a java.lang.Class for 
org.apache.cassandra.db.ColumnFamilyStore)
        at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:414)
        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 <0x00000000e10dfaf0> (a java.lang.Class for 
org.apache.cassandra.db.Keyspace)          <----------------------------------- 
// this
        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 <0x00000000e26f26a0> (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)
{noformat}




> CQLSSTableWriter freezes on addRow
> ----------------------------------
>
>                 Key: CASSANDRA-8884
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-8884
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Piotr Kołaczkowski
>         Attachments: bulk.jstack
>
>
> I created a writer like this:
> {code}
> val writer = CQLSSTableWriter.builder()
>       .forTable(tableDef.cql)
>       .using(insertStatement)
>       .withPartitioner(partitioner)
>       .inDirectory(outputDirectory)    
>       .withBufferSizeInMB(bufferSizeInMB)
>       .build()
> {code}
> 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.
> {noformat}
> "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)
> {noformat}



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

Reply via email to