[ https://issues.apache.org/jira/browse/CASSANDRA-9910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14644401#comment-14644401 ]
Mike Adamson commented on CASSANDRA-9910: ----------------------------------------- This also affects {{system_drop_keyspace}} reproducible with the following: {noformat} KsDef ksDef = new KsDef("new_ks_2", SimpleStrategy.class.getName(), Collections.<CfDef>emptyList()); ksDef.putToStrategy_options("replication_factor", "1"); client.system_add_keyspace(ksDef); client.system_drop_keyspace(ksDef.name); {noformat} > system_drop_column_family hangs on trunk > ---------------------------------------- > > Key: CASSANDRA-9910 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9910 > Project: Cassandra > Issue Type: Bug > Components: Core > Reporter: Mike Adamson > Fix For: 3.0.0 rc1 > > > {{system_drop_column_family}} is hanging on trunk. The code used to replicate > this is: > {noformat} > CfDef cfDef = new CfDef(ks, "new_cf_2"); > client.system_add_column_family(cfDef); > client.system_drop_column_family(cfDef.name); > {noformat} > The call is hanging in {{ColumnFamilyStore.switchMemtable}} because the > memtable flush is failing with > {noformat} > 1 ERROR [MemtableFlushWriter:2] 2015-07-28 11:59:26,688 CassandraDaemon.java > (line 189) Exception in thread > Thread[MemtableFlushWriter:2,5,FailOnTimeoutGroup] > java.lang.AssertionError: null > at > org.apache.cassandra.db.PartitionColumns$Builder.add(PartitionColumns.java:148) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > org.apache.cassandra.db.PartitionColumns$Builder.addAll(PartitionColumns.java:159) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > org.apache.cassandra.db.Memtable$ColumnsCollector.get(Memtable.java:507) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:373) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:359) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) > ~[guava-16.0.1.jar:na] > at > org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1034) > ~[cassandra-all-3.0.0-SNAPSHOT.jar:3.0.0-SNAPSHOT] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > ~[na:1.8.0_45] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > ~[na:1.8.0_45] > at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_45] > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)