Details
-
Bug
-
Status: Resolved
-
Normal
-
Resolution: Duplicate
-
None
-
None
-
Normal
Description
We observe nodetool compactionstats hanging when there are a lot of SSTables in one table. We have about 30.000 SSTables most likely created by an incremental repair (why it's that many is still a mystery to us).
Looking at the stacktraces of some selected threads it becomes apparent that a single CompactionExecutor blocks several other threads:
"CompactionExecutor:4065" - Thread t@282454 java.lang.Thread.State: RUNNABLE at java.util.TimSort.binarySort(TimSort.java:292) at java.util.TimSort.sort(TimSort.java:235) at java.util.Arrays.sort(Arrays.java:1512) at java.util.ArrayList.sort(ArrayList.java:1454) at java.util.Collections.sort(Collections.java:175) at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.trimToThresholdWithHotness(SizeTieredCompactionStrategy.java:139) at org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.mostInterestingBucket(SizeTieredCompactionStrategy.java:119) at org.apache.cassandra.db.compaction.LeveledManifest.getSSTablesForSTCS(LeveledManifest.java:360) at org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:318) - locked <25446e7b> (a org.apache.cassandra.db.compaction.LeveledManifest) at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getMaximalTask(LeveledCompactionStrategy.java:101) at org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:90) - locked <33506169> (a org.apache.cassandra.db.compaction.LeveledCompactionStrategy) at org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:77) - locked <31924612> (a org.apache.cassandra.db.compaction.WrappingCompactionStrategy) at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:230) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)
Because this thread has locked <31924612> (a org.apache.cassandra.db.compaction.WrappingCompactionStrategy) the following threads are blocked:
"CompactionExecutor:4064" - Thread t@282337 java.lang.Thread.State: BLOCKED at org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:72) - waiting to lock <31924612> (a org.apache.cassandra.db.compaction.WrappingCompactionStrategy) owned by "CompactionExecutor:4065" t@282454 at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:230) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)
"MemtableFlushWriter:2431" - Thread t@283411 java.lang.Thread.State: BLOCKED at org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:250) - waiting to lock <31924612> (a org.apache.cassandra.db.compaction.WrappingCompactionStrategy) owned by "CompactionExecutor:4065" t@282454 at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:518) at org.apache.cassandra.db.DataTracker.replaceFlushed(DataTracker.java:178) at org.apache.cassandra.db.compaction.AbstractCompactionStrategy.replaceFlushed(AbstractCompactionStrategy.java:234) at org.apache.cassandra.db.ColumnFamilyStore.replaceFlushed(ColumnFamilyStore.java:1541) at org.apache.cassandra.db.Memtable$FlushRunnable.runMayThrow(Memtable.java:336) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1154) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)
"RMI TCP Connection(128577)-192.168.0.1" - Thread t@282977 java.lang.Thread.State: BLOCKED at org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getEstimatedRemainingTasks(WrappingCompactionStrategy.java:147) - waiting to lock <31924612> (a org.apache.cassandra.db.compaction.WrappingCompactionStrategy) owned by "CompactionExecutor:4065" t@282454 at org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:65) at org.apache.cassandra.metrics.CompactionMetrics$1.value(CompactionMetrics.java:57) at com.yammer.metrics.reporting.JmxReporter$Gauge.getValue(JmxReporter.java:63) at sun.reflect.GeneratedMethodAccessor31.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:71) at sun.reflect.GeneratedMethodAccessor19.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:275) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) at com.sun.jmx.mbeanserver.PerInterface.getAttribute(PerInterface.java:83) at com.sun.jmx.mbeanserver.MBeanSupport.getAttribute(MBeanSupport.java:206) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttribute(DefaultMBeanServerInterceptor.java:647) at com.sun.jmx.mbeanserver.JmxMBeanServer.getAttribute(JmxMBeanServer.java:678) at com.sun.jmx.remote.security.MBeanServerAccessController.getAttribute(MBeanServerAccessController.java:320) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1443) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:76) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1307) at java.security.AccessController.doPrivileged(Native Method) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1406) at javax.management.remote.rmi.RMIConnectionImpl.getAttribute(RMIConnectionImpl.java:637) at sun.reflect.GeneratedMethodAccessor21.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:497) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:323) at sun.rmi.transport.Transport$1.run(Transport.java:200) at sun.rmi.transport.Transport$1.run(Transport.java:197) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:196) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:568) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:826) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.lambda$run$241(TCPTransport.java:683) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler$$Lambda$1/183345253.run(Unknown Source) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:682) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745)
The lock protects from concurrent access through the two synchronized methods WrappingCompactionStrategy.getEstimatedRemainingTasks and WrappingCompactionStrategy.getNextBackgroundTask.
The lock is not just hold for a short time but for several minutes. We think that this combined with suboptimal values for memtable_flush_writers and concurrent_compactors caused our nodes to exhaust their heap and die after running incremental repair.
There also seems to be no other way to reduce the number of SSTables as a user except than waiting for the compactions to eventually finish which looks like it can take days.
Attachments
Issue Links
- duplicates
-
CASSANDRA-10099 Improve concurrency in CompactionStrategyManager
- Resolved