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

Jeff Griffith edited comment on CASSANDRA-10515 at 10/16/15 12:14 AM:
----------------------------------------------------------------------

[~tjake] I monitored live for a few hours to capture the behavior. See 
RUN3tpstats.jpg in the attachments:

Overview is:
Monitoring threads began to block before the memtable flushing did.
Memtable flushing seemed to be progressing slowly and then post flushing 
operations began to pile up. The primary things blocked were:
1. MemtableFlushWriter/handleNotif
2. CompactionExec/getNextBGTask
3. ServiceThread/getEstimatedRemTask

Those three blocked and never came unblocked so assume (?) the locker never 
completed or was very, very slow. Eventually a second MemtableFlushWriter 
thread blocks. I believe if I left it continue to run, all or many of them 
will. 

{code}
"CompactionExecutor:18" #1462 daemon prio=1 os_prio=4 tid=0x00007fd961410000 
nid=0x728b runnable [0x00007fda4ae0b000]
   java.lang.Thread.State: RUNNABLE
        at org.apache.cassandra.dht.Bounds.contains(Bounds.java:49)
        at org.apache.cassandra.dht.Bounds.intersects(Bounds.java:77)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:511)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:497)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:572)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:346)
        - locked <0x00000004a8bc5038> (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 <0x00000004a8af17d0> (a 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
        at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:84)
        - locked <0x00000004a894df10> (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)
{code}


I see one thread for MemtablePostFlush and this is it:

{code}
"MemtablePostFlush:8" #4866 daemon prio=5 os_prio=0 tid=0x00007fd91c0c5800 
nid=0x2d93 waiting on condition [0x00007fda4b46c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000005838ba468> (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 followed it for a while longer after this and it really looks like the post 
flush stays blocked on that latch forever:

{code}
00:01
MemtableFlushWriter               2         2           2024         0          
       0
MemtablePostFlush                 1     47159           4277         0          
       0
MemtableReclaimMemory             0         0           2024         0          
       0


00:03
MemtableFlushWriter               3         3           2075         0          
       0
MemtablePostFlush                 1     48853           4277         0          
       0
MemtableReclaimMemory             0         0           2075         0          
       0

00:11
MemtableFlushWriter               3         3           2078         0          
       0
MemtablePostFlush                 1     58634           4277         0          
       0
MemtableReclaimMemory             0         0           2078         0          
       0
{code}



was (Author: jeffery.griffith):
[~tjake] I monitored live for a few hours to capture the behavior. See 
RUN3tpstats.jpg in the attachments:

Overview is:
Monitoring threads began to block before the memtable flushing did.
Memtable flushing seemed to be progressing slowly and then post flushing 
operations began to pile up. The primary things blocked were:
1. MemtableFlushWriter/handleNotif
2. CompactionExec/getNextBGTask
3. ServiceThread/getEstimatedRemTask

Those three blocked and never came unblocked so assume (?) the locker never 
completed or was very, very slow. Eventually a second MemtableFlushWriter 
thread blocks. I believe if I left it continue to run, all or many of them 
will. 

{code}
"CompactionExecutor:18" #1462 daemon prio=1 os_prio=4 tid=0x00007fd961410000 
nid=0x728b runnable [0x00007fda4ae0b000]
   java.lang.Thread.State: RUNNABLE
        at org.apache.cassandra.dht.Bounds.contains(Bounds.java:49)
        at org.apache.cassandra.dht.Bounds.intersects(Bounds.java:77)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:511)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:497)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(LeveledManifest.java:572)
        at 
org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates(LeveledManifest.java:346)
        - locked <0x00000004a8bc5038> (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 <0x00000004a8af17d0> (a 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
        at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:84)
        - locked <0x00000004a894df10> (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)
{code}


I see one thread for MemtablePostFlush and this is it:

{code}
"MemtablePostFlush:8" #4866 daemon prio=5 os_prio=0 tid=0x00007fd91c0c5800 
nid=0x2d93 waiting on condition [0x00007fda4b46c000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00000005838ba468> (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 followed it for a while longer after this and it really looks like the post 
flush stacks blocked on that latch forever:

{code}
00:01
MemtableFlushWriter               2         2           2024         0          
       0
MemtablePostFlush                 1     47159           4277         0          
       0
MemtableReclaimMemory             0         0           2024         0          
       0


00:03
MemtableFlushWriter               3         3           2075         0          
       0
MemtablePostFlush                 1     48853           4277         0          
       0
MemtableReclaimMemory             0         0           2075         0          
       0

00:11
MemtableFlushWriter               3         3           2078         0          
       0
MemtablePostFlush                 1     58634           4277         0          
       0
MemtableReclaimMemory             0         0           2078         0          
       0
{code}


> 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: CommitLogProblem.jpg, CommitLogSize.jpg, 
> RUN3tpstats.jpg, 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)

Reply via email to