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

Jon Haddad commented on CASSANDRA-21065:
----------------------------------------

I took a brief look at the source to see if I could make sense of this.  My 
hunch is that it has to do with the way UCS does partitioning to allow for 
parallel compactions.  I wonder if synchronizing the code around the 
LifecycleTransaction would be enough to avoid it.

> ConcurrentModificationException during nodetool garbagecollect with 
> UnifiedCompactionStrategy and only_purge_repaired_tombstones enabled
> ----------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-21065
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-21065
>             Project: Apache Cassandra
>          Issue Type: Bug
>          Components: Local/Compaction/UCS
>            Reporter: Jon Haddad
>            Priority: Normal
>
> When running nodetool garbagecollect on a table configured with 
> UnifiedCompactionStrategy and only_purge_repaired_tombstones: true, a 
> ConcurrentModificationException is thrown. This occurs regardless of whether 
> any other repairs or compactions are running in parallel.
> Failing configuration:
> {noformat}
> Compaction = {
>     'class': 'org.apache.cassandra.db.compaction.UnifiedCompactionStrategy',
>     'only_purge_repaired_tombstones': 'true',
>     'scaling_parameters': 'L10'
> }
> {noformat}
> The issue has been reproduced across 3 separate clusters.
> This stack trace is thrown when attempting to run nodetool garbagecollect:
> {noformat}
>  error: null
> -- StackTrace --
> java.util.ConcurrentModificationException
> at java.base/java.util.HashMap$HashIterator.nextNode(HashMap.java:1597)
> at java.base/java.util.HashMap$KeyIterator.next(HashMap.java:1620)
> at 
> java.base/java.util.Collections$UnmodifiableCollection$1.next(Collections.java:1055)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$6.filterSSTables(CompactionManager.java:691)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.lambda$parallelAllSSTableOperation$0(CompactionManager.java:441)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.withAllSSTables(ColumnFamilyStore.java:2940)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.parallelAllSSTableOperation(CompactionManager.java:432)
> at 
> org.apache.cassandra.db.compaction.CompactionManager.performGarbageCollection(CompactionManager.java:683)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.garbageCollect(ColumnFamilyStore.java:1823)
> at 
> org.apache.cassandra.service.StorageService.garbageCollect(StorageService.java:4285)
> at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native 
> Method)
> at 
> java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77)
> at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.base/java.lang.reflect.Method.invoke(Method.java:569)
> at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:72)
> at jdk.internal.reflect.GeneratedMethodAccessor25.invoke(Unknown Source)
> at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.base/java.lang.reflect.Method.invoke(Method.java:569)
> at java.base/sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:262)
> at 
> java.management/com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112)
> at 
> java.management/com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46)
> at 
> java.management/com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237)
> at 
> java.management/com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138)
> at 
> java.management/com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252)
> at 
> java.management/com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:814)
> at 
> java.management/com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:802)
> at 
> java.management/com.sun.jmx.remote.security.MBeanServerAccessController.invoke(MBeanServerAccessController.java:472)
> at 
> java.management.rmi/javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1472)
> at 
> java.management.rmi/javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1310)
> at 
> java.base/java.security.AccessController.doPrivileged(AccessController.java:712)
> at 
> java.management.rmi/javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1412)
> at 
> java.management.rmi/javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:829)
> at java.base/jdk.internal.reflect.GeneratedMethodAccessor33.invoke(Unknown 
> Source)
> at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.base/java.lang.reflect.Method.invoke(Method.java:569)
> at 
> java.rmi/sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:360)
> at java.rmi/sun.rmi.transport.Transport$1.run(Transport.java:200)
> at java.rmi/sun.rmi.transport.Transport$1.run(Transport.java:197)
> at 
> java.base/java.security.AccessController.doPrivileged(AccessController.java:712)
> at java.rmi/sun.rmi.transport.Transport.serviceCall(Transport.java:196)
> at 
> java.rmi/sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:587)
> at 
> java.rmi/sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:828)
> at 
> java.rmi/sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$0(TCPTransport.java:705)
> at 
> java.base/java.security.AccessController.doPrivileged(AccessController.java:399)
> at 
> java.rmi/sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:704)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
> at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
> at java.base/java.lang.Thread.run(Thread.java:840)
> {noformat}



--
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