[ 
https://issues.apache.org/jira/browse/CASSANDRA-20950?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18024406#comment-18024406
 ] 

Sam Tunnicliffe commented on CASSANDRA-20950:
---------------------------------------------

Exactly which version are you seeing this with? I'm completely unable to 
reproduce it with trunk at {{61959e2}}, either in a single or multi-node 
cluster. 
>From your stacktrace, the issue looks to be the CFS mbean not getting 
>unregistered when the keyspace is dropped. Checking with jconsole locally and 
>I can't find any evidence of that either. 



> TCM - can't drop a keyspace with a table and recreate same structure
> --------------------------------------------------------------------
>
>                 Key: CASSANDRA-20950
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-20950
>             Project: Apache Cassandra
>          Issue Type: Bug
>          Components: Transactional Cluster Metadata
>            Reporter: Jon Haddad
>            Priority: Urgent
>
> Looks like if you create a keyspace and a table, then drop the keyspace, 
> trying to re-create that same keyspace and table will completely lock up TCM.
>  
> Steps to repro:
>  
> {noformat}
> cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> create table test.t1 (id int primary key);
> cqlsh> drop KEYSPACE test;
> cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> create table test.t1 (id int primary key);{noformat}
> Error:
> {noformat}
> INFO  [GlobalLogFollower] 2025-10-03T00:09:56,074 ColumnFamilyStore.java:513 
> - Initializing test.t1
> ERROR [GlobalLogFollower] 2025-10-03T00:09:56,076 LocalLog.java:546 - Could 
> not process the entry
> java.lang.RuntimeException: javax.management.InstanceAlreadyExistsException: 
> org.apache.cassandra.db.compression:type=CompressionDictionaryManager,keyspace=test,table=t1
>     at 
> org.apache.cassandra.utils.MBeanWrapper$OnException.lambda$static$0(MBeanWrapper.java:365)
>     at 
> org.apache.cassandra.utils.MBeanWrapper$PlatformMBeanWrapper.registerMBean(MBeanWrapper.java:184)
>     at 
> org.apache.cassandra.utils.MBeanWrapper.registerMBean(MBeanWrapper.java:97)
>     at 
> org.apache.cassandra.utils.MBeanWrapper.registerMBean(MBeanWrapper.java:101)
>     at 
> org.apache.cassandra.db.compression.CompressionDictionaryManager.registerMBean(CompressionDictionaryManager.java:82)
>     at 
> org.apache.cassandra.db.compression.CompressionDictionaryManager.<init>(CompressionDictionaryManager.java:74)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:582)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:801)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:778)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:768)
>     at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:380)
>     at 
> org.apache.cassandra.schema.DistributedSchema.createTable(DistributedSchema.java:396)
>     at 
> org.apache.cassandra.schema.DistributedSchema.lambda$initializeKeyspaceInstances$9(DistributedSchema.java:245)
>     at java.base/java.lang.Iterable.forEach(Iterable.java:75)
>     at 
> org.apache.cassandra.schema.DistributedSchema.lambda$initializeKeyspaceInstances$13(DistributedSchema.java:245)
>     at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:422)
>     at 
> org.apache.cassandra.schema.DistributedSchema.initializeKeyspaceInstances(DistributedSchema.java:231)
>     at 
> org.apache.cassandra.tcm.listeners.SchemaListener.notifyInternal(SchemaListener.java:50)
>     at 
> org.apache.cassandra.tcm.listeners.SchemaListener.notifyPreCommit(SchemaListener.java:43)
>     at 
> org.apache.cassandra.tcm.log.LocalLog.notifyPreCommit(LocalLog.java:620)
>     at 
> org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:521)
>     at 
> org.apache.cassandra.tcm.log.LocalLog$Async$AsyncRunnable.run(LocalLog.java:813)
>     at 
> org.apache.cassandra.concurrent.InfiniteLoopExecutor.loop(InfiniteLoopExecutor.java:119)
>     at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>     at java.base/java.lang.Thread.run(Thread.java:840)
> Caused by: javax.management.InstanceAlreadyExistsException: 
> org.apache.cassandra.db.compression:type=CompressionDictionaryManager,keyspace=test,table=t1
>     at 
> java.management/com.sun.jmx.mbeanserver.Repository.addMBean(Repository.java:436)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerWithRepository(DefaultMBeanServerInterceptor.java:1865)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(DefaultMBeanServerInterceptor.java:960)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(DefaultMBeanServerInterceptor.java:895)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(DefaultMBeanServerInterceptor.java:320)
>     at 
> java.management/com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(JmxMBeanServer.java:523)
>     at 
> org.apache.cassandra.utils.MBeanWrapper$PlatformMBeanWrapper.registerMBean(MBeanWrapper.java:180)
>     ... 23 common frames omitted{noformat}
>  
> This is fun too:
>  
>  
>  
> {noformat}
>  
> cqlsh> drop KEYSPACE test;
> cqlsh> create KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> cqlsh> create table test.t2 (id int primary key);
> cqlsh> select * from test.t2;
> InvalidRequest: Error from server: code=2200 [Invalid query] message="table 
> t2 does not exist"
> {noformat}
>  
> Exception thrown in C*:
> {noformat}
> INFO  [GlobalLogFollower] 2025-10-03T00:14:24,536 ColumnFamilyStore.java:513 
> - Initializing test.t2
> ERROR [GlobalLogFollower] 2025-10-03T00:14:24,538 LocalLog.java:546 - Could 
> not process the entry
> java.lang.RuntimeException: javax.management.InstanceAlreadyExistsException: 
> org.apache.cassandra.db.compression:type=CompressionDictionaryManager,keyspace=test,table=t2
>     at 
> org.apache.cassandra.utils.MBeanWrapper$OnException.lambda$static$0(MBeanWrapper.java:365)
>     at 
> org.apache.cassandra.utils.MBeanWrapper$PlatformMBeanWrapper.registerMBean(MBeanWrapper.java:184)
>     at 
> org.apache.cassandra.utils.MBeanWrapper.registerMBean(MBeanWrapper.java:97)
>     at 
> org.apache.cassandra.utils.MBeanWrapper.registerMBean(MBeanWrapper.java:101)
>     at 
> org.apache.cassandra.db.compression.CompressionDictionaryManager.registerMBean(CompressionDictionaryManager.java:82)
>     at 
> org.apache.cassandra.db.compression.CompressionDictionaryManager.<init>(CompressionDictionaryManager.java:74)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.<init>(ColumnFamilyStore.java:582)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:801)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:778)
>     at 
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:768)
>     at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:380)
>     at 
> org.apache.cassandra.schema.DistributedSchema.createTable(DistributedSchema.java:396)
>     at 
> org.apache.cassandra.schema.DistributedSchema.lambda$initializeKeyspaceInstances$9(DistributedSchema.java:245)
>     at java.base/java.lang.Iterable.forEach(Iterable.java:75)
>     at 
> org.apache.cassandra.schema.DistributedSchema.lambda$initializeKeyspaceInstances$13(DistributedSchema.java:245)
>     at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:422)
>     at 
> org.apache.cassandra.schema.DistributedSchema.initializeKeyspaceInstances(DistributedSchema.java:231)
>     at 
> org.apache.cassandra.tcm.listeners.SchemaListener.notifyInternal(SchemaListener.java:50)
>     at 
> org.apache.cassandra.tcm.listeners.SchemaListener.notifyPreCommit(SchemaListener.java:43)
>     at 
> org.apache.cassandra.tcm.log.LocalLog.notifyPreCommit(LocalLog.java:620)
>     at 
> org.apache.cassandra.tcm.log.LocalLog.processPendingInternal(LocalLog.java:521)
>     at 
> org.apache.cassandra.tcm.log.LocalLog$Async$AsyncRunnable.run(LocalLog.java:813)
>     at 
> org.apache.cassandra.concurrent.InfiniteLoopExecutor.loop(InfiniteLoopExecutor.java:119)
>     at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>     at java.base/java.lang.Thread.run(Thread.java:840)
> Caused by: javax.management.InstanceAlreadyExistsException: 
> org.apache.cassandra.db.compression:type=CompressionDictionaryManager,keyspace=test,table=t2
>     at 
> java.management/com.sun.jmx.mbeanserver.Repository.addMBean(Repository.java:436)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerWithRepository(DefaultMBeanServerInterceptor.java:1865)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerDynamicMBean(DefaultMBeanServerInterceptor.java:960)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerObject(DefaultMBeanServerInterceptor.java:895)
>     at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.registerMBean(DefaultMBeanServerInterceptor.java:320)
>     at 
> java.management/com.sun.jmx.mbeanserver.JmxMBeanServer.registerMBean(JmxMBeanServer.java:523)
>     at 
> org.apache.cassandra.utils.MBeanWrapper$PlatformMBeanWrapper.registerMBean(MBeanWrapper.java:180){noformat}
>  
> Trying to create any other tables ends up rethrowing the above exception for 
> the table name that originally triggered the problem.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to