[ https://issues.apache.org/jira/browse/CASSANDRA-10515?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14965636#comment-14965636 ]
Jeff Griffith commented on CASSANDRA-10515: ------------------------------------------- Hi again [~krummas] Before trying the leveling, the remaining problematic clusters seemed to work out the # of files in L0 problem. They were all trending downward but there were several days where it was very frequent. Alas, the isolated node with large SSTable counts does not seem to be the only issue where commit logs break the limit. I'm tempted to open this as a separate issue, but let's see what you think first. In some cases, we see all 3 nodes in those small clusters break the limit at the same time. I will do better monitoring but I did manage to catch one in progress and here i observed. There was not a lot of blocked threads like before but it did have the MemtablePostFlusher blocked on the countdown latch. So here are the tpstats for that: {code} MemtableFlushWriter 8 30 7200 0 0 MemtablePostFlush 1 45879 16841 0 0 MemtableReclaimMemory 0 0 7199 0 0 {code} With 46K pending. The only thread I see for that is here: {code} "MemtablePostFlush:3" #3054 daemon prio=5 os_prio=0 tid=0x00007f806fb71000 nid=0x2e5c waiting on condition [0x00007f804366c000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000005de8976f8> (a java.util.concurrent.CountDownLatch$Sync) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231) at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.run(ColumnFamilyStore.java:998) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.run(FutureTask.java:266) 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) {code} I don't know who counts that latch down, but there were a couple of blocked threads, here: {code} "HintedHandoff:2" #1429 daemon prio=1 os_prio=4 tid=0x00007f80895c4800 nid=0x1242 waiting for monitor entry [0x00007f804321b000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.cassandra.db.HintedHandOffManager.compact(HintedHandOffManager.java:267) - waiting to lock <0x00000004e2e689a8> (a org.apache.cassandra.db.HintedHandOffManager) at org.apache.cassandra.db.HintedHandOffManager$5.run(HintedHandOffManager.java:561) 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) "HintedHandoff:1" #1428 daemon prio=1 os_prio=4 tid=0x00007f80895c3800 nid=0x1241 waiting for monitor entry [0x00007f7838855000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.cassandra.db.HintedHandOffManager.compact(HintedHandOffManager.java:267) - waiting to lock <0x00000004e2e689a8> (a org.apache.cassandra.db.HintedHandOffManager) at org.apache.cassandra.db.HintedHandOffManager$5.run(HintedHandOffManager.java:561) 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) {code} and the lock was held here: {code} "HintedHandoffManager:1" #1430 daemon prio=1 os_prio=4 tid=0x00007f808aaf1800 nid=0x1243 waiting on condition [0x00007f8043423000] java.lang.Thread.State: WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x000000060bdc0b98> (a java.util.concurrent.FutureTask) at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429) at java.util.concurrent.FutureTask.get(FutureTask.java:191) at org.apache.cassandra.db.HintedHandOffManager.compact(HintedHandOffManager.java:276) - locked <0x00000004e2e689a8> (a org.apache.cassandra.db.HintedHandOffManager) at org.apache.cassandra.db.HintedHandOffManager.scheduleAllDeliveries(HintedHandOffManager.java:520) at org.apache.cassandra.db.HintedHandOffManager.access$000(HintedHandOffManager.java:93) at org.apache.cassandra.db.HintedHandOffManager$1.run(HintedHandOffManager.java:184) at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) 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) {code} I only got one snapshot of that before it restarted but I'll set up some monitoring to check it over time as I did with the SSTable count problem. Had the chance progress on that previous one? > Commit logs back up with move to 2.1.10 > --------------------------------------- > > Key: CASSANDRA-10515 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10515 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: redhat 6.5, cassandra 2.1.10 > Reporter: Jeff Griffith > Assignee: Branimir Lambov > Priority: Critical > Labels: commitlog, triage > Attachments: C5commitLogIncrease.jpg, CommitLogProblem.jpg, > CommitLogSize.jpg, RUN3tpstats.jpg, cassandra.yaml, cfstats-clean.txt, > stacktrace.txt, system.log.clean > > > After upgrading from cassandra 2.0.x to 2.1.10, we began seeing problems > where some nodes break the 12G commit log max we configured and go as high as > 65G or more before it restarts. Once it reaches the state of more than 12G > commit log files, "nodetool compactionstats" hangs. Eventually C* restarts > without errors (not sure yet whether it is crashing but I'm checking into it) > and the cleanup occurs and the commit logs shrink back down again. Here is > the nodetool compactionstats immediately after restart. > {code} > jgriffith@prod1xc1.c2.bf1:~$ ndc > pending tasks: 2185 > compaction type keyspace table completed > total unit progress > Compaction SyncCore *cf1* 61251208033 > 170643574558 bytes 35.89% > Compaction SyncCore *cf2* 19262483904 > 19266079916 bytes 99.98% > Compaction SyncCore *cf3* 6592197093 > 6592316682 bytes 100.00% > Compaction SyncCore *cf4* 3411039555 > 3411039557 bytes 100.00% > Compaction SyncCore *cf5* 2879241009 > 2879487621 bytes 99.99% > Compaction SyncCore *cf6* 21252493623 > 21252635196 bytes 100.00% > Compaction SyncCore *cf7* 81009853587 > 81009854438 bytes 100.00% > Compaction SyncCore *cf8* 3005734580 > 3005768582 bytes 100.00% > Active compaction remaining time : n/a > {code} > I was also doing periodic "nodetool tpstats" which were working but not being > logged in system.log on the StatusLogger thread until after the compaction > started working again. -- This message was sent by Atlassian JIRA (v6.3.4#6332)