[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-12-02 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

That's correct. Two problems led independently to the build up:

Cause 1 fixed by Marcus: sstable leveling info was lost during sstable upgrade 
leading to thread contention due to large # of tables at L0.

Cause 2 fixed by Benedict: index out of bounds exception caused by integer 
overflow.


> 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: Streaming and Messaging
> Environment: redhat 6.5, cassandra 2.1.10
>Reporter: Jeff Griffith
>Assignee: Branimir Lambov
>  Labels: commitlog, triage
> Fix For: 3.0.1, 3.1, 2.1.x, 2.2.x
>
> Attachments: C5commitLogIncrease.jpg, CASSANDRA-19579.jpg, 
> CommitLogProblem.jpg, CommitLogSize.jpg, 
> MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-11-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

note that we do have a problem with upgradesstables losing level info: 
CASSANDRA-10692

> 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: Streaming and Messaging
> Environment: redhat 6.5, cassandra 2.1.10
>Reporter: Jeff Griffith
>Assignee: Branimir Lambov
>  Labels: commitlog, triage
> Fix For: 3.1, 2.1.x, 2.2.x
>
> Attachments: C5commitLogIncrease.jpg, CASSANDRA-19579.jpg, 
> CommitLogProblem.jpg, CommitLogSize.jpg, 
> MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-11-12 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Thanks [~krummas] i assume you mean this explains the large number of sstables 
(55k) we experienced? I see you've fixed it. I have moved to the latest 2.1 so 
this should help with our rollout.

> 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: Streaming and Messaging
> Environment: redhat 6.5, cassandra 2.1.10
>Reporter: Jeff Griffith
>Assignee: Branimir Lambov
>  Labels: commitlog, triage
> Fix For: 3.1, 2.1.x, 2.2.x
>
> Attachments: C5commitLogIncrease.jpg, CASSANDRA-19579.jpg, 
> CommitLogProblem.jpg, CommitLogSize.jpg, 
> MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-11-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

it is not committed or released yet

> 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: Streaming and Messaging
> Environment: redhat 6.5, cassandra 2.1.10
>Reporter: Jeff Griffith
>Assignee: Branimir Lambov
>  Labels: commitlog, triage
> Fix For: 3.1, 2.1.x, 2.2.x
>
> Attachments: C5commitLogIncrease.jpg, CASSANDRA-19579.jpg, 
> CommitLogProblem.jpg, CommitLogSize.jpg, 
> MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-11-12 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Good to know. We'll watch out for it and use the offline leveling trick you 
suggested.

> 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: Streaming and Messaging
> Environment: redhat 6.5, cassandra 2.1.10
>Reporter: Jeff Griffith
>Assignee: Branimir Lambov
>  Labels: commitlog, triage
> Fix For: 3.1, 2.1.x, 2.2.x
>
> Attachments: C5commitLogIncrease.jpg, CASSANDRA-19579.jpg, 
> CommitLogProblem.jpg, CommitLogSize.jpg, 
> MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-28 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Great info, thanks [~benedict] ! I guess my "all roads lead to Rome" analogy 
was a good one :-)

[~blambov] i have the rest running now. normally happens a couple of times a 
day so i should know by this evening.

On that first form of growth, I unfortunately did not get a chance to try it 
before the problem corrected itself. I believe that gradually our 3 remaining 
problematic nodes too longer to reduce the # of L0 files than did the rest of 
our clusters. It took weeks rather than days and coincidentally I got involved 
near the end. From what saw, though, the symptoms seemed to match exactly what 
[~krummas] described.

> 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, MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-28 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-10515:
-

Let us know if that fix does not solve the problem. On the first form, did 
re-levelling improve the situation?

> 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, MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-28 Thread Benedict (JIRA)

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

Benedict commented on CASSANDRA-10515:
--

Any failure during flush is known to cause commit log backup. See 
CASSANDRA-8496.

> 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, MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-27 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

[~krummas] [~tjake] something interesting on this second form of commit log 
growth where all nodes had uncontrolled commit log growth unless the first 
example (many files in L0) where it was isolated nodes. for this latter case, I 
think i'm able to relate this to a separate problem with an index out of bounds 
exception. working with [~benedict] it seems like we have that one solved. i'm 
hopeful that patch will solve this growing commit log problem as well. it seems 
like all roads lead to rome where rome is commit log growth :-)

here is the other JIRA identifying an integer overflow in 
AbstractNativeCell.java
https://issues.apache.org/jira/browse/CASSANDRA-10579

Still uncertain how to proceed with the first form that seems to be starvation 
as you have described.


> 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, MultinodeCommitLogGrowth-node1.tar.gz, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-20 Thread Jeff Griffith (JIRA)

[ 
https://issues.apache.org/jira/browse/CASSANDRA-10515?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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   830   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=0x7f806fb71000 
nid=0x2e5c waiting on condition [0x7f804366c000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x0005de8976f8> (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=0x7f80895c4800 
nid=0x1242 waiting for monitor entry [0x7f804321b000]
   java.lang.Thread.State: BLOCKED (on object monitor)
at 
org.apache.cassandra.db.HintedHandOffManager.compact(HintedHandOffManager.java:267)
- waiting to lock <0x0004e2e689a8> (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=0x7f80895c3800 
nid=0x1241 waiting for monitor entry [0x7f7838855000]
   java.lang.Thread.State: BLOCKED (on object monitor)
at 
org.apache.cassandra.db.HintedHandOffManager.compact(HintedHandOffManager.java:267)
- waiting to lock <0x0004e2e689a8> (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=0x7f808aaf1800 
nid=0x1243 waiting on condition [0x7f8043423000]
   java.lang.Thread.State: WAITING (parking)
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <0x00060bdc0b98> (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 

[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

thanks [~krummas] see cfstats-clean.txt which i obfuscated and uploaded. we 
didn't actually name them CF001 ;-)

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

is the cfstats from a single node or did you combine?

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread T Jake Luciani (JIRA)

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

T Jake Luciani commented on CASSANDRA-10515:


[~krummas] is there a fix we can provide as well? Running repair for example 
can put a lot of tiny sstables into L0

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

[~tjake] yes, we should probably the {{synchronized}} on every method in the 
compaction strategies and optimize the LCS overlap check like we did with the 
global check in CASSANDRA-9882 (if possible, have not looked at that code in a 
while)

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

No, that is one node.

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Indeed the symptoms here look like the other jira you mentioned. I have 
followed the thread dumps over time and it looks very much like it's spending a 
lot of time in the "overlapping" calculation as you see above.

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

k, so you have 1.5T on that node, do you have as much data on the other nodes?

how did you end up in this situation (with 50k+ sstables in L0 in one table), 
just upgrade?

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

yes, we just upgraded from 2.0. would that explain the 50k+? i have checked and 
that is NOT the case on the other nodes. yes, they are balanced in terms of 
data (40 core machines with lots of memory). in this stage of our rollout to 
2.1, we have gone to 10 small clusters of 3 nodes each (30 nodes total). only 
THREE nodes of the thirty are now exhibiting this behavior. for the first few 
days, several others did however they seem to have self corrected. I will go 
back and check for large sstable counts to see if that explains all of them. 
after this first stage, we'll be rolling out to the larger 24-node customers 
but we are pausing here on the small clusters until we figure this out.


> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

bq. yes, we just upgraded from 2.0. would that explain the 50k+?
nope, unless we have a bug in upgradesstables that loses the level info, but 
since it is not the case everywhere i doubt it

to get out of your current situation you could stop the node and try to run 
'tools/bin/sstableofflinerelevel' which tries to give the table a good leveling

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

great, thanks [~krummas]. please let me know if there is any more information i 
can provide to help resolve it.  i'll get more info across the 30 nodes on 
large sstables and make sure this correlates to the problem.

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

[~jeffery.griffith] have you upgraded all your nodes yet? If not, could you 
take a cfstats before and after upgrade?

> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

No, this is only the beginning (300M users and a petabyte to go :) ) . We were 
kind of pausing here since the bigger clusters carry so many more users but we 
can definitely do that if we move forward with the rollout. We'll apply the 
releveling where can and see how it behaviors. is the 5K sstable count enough 
to be concerned about? i'll do some more analysis on these and compare to 
clusters that have not yet been upgraded.


> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

if the 5k sstables are all in L0, then yes, that can be painful

> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

ok, that's good news then. we'll apply tools/bin/sstableofflinerelevel to 
everything above 1K or so?

> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-16 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

try it with --dry-run to see if it helps first :)

> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Mikhail Stepura (JIRA)

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

Mikhail Stepura commented on CASSANDRA-10515:
-

[~jeffery.griffith] could you please attach a thread dump as well?

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread T Jake Luciani (JIRA)

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

T Jake Luciani commented on CASSANDRA-10515:


That's RUNNABLE though not BLOCKED.  Are you actually deadlocking or only 
seeing slow flushes?

[~krummas] any ideas?


> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread T Jake Luciani (JIRA)

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

T Jake Luciani commented on CASSANDRA-10515:


Yeah if the COMPLETED column for flushing is incrementing

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

I had restarted but I'll watch live the next iteration. As you see upwards in 
the comments though, they do start piling up:
MemtableFlushWriter   1 1   1574 0  
   0
MemtablePostFlush 1 13755 134889 0  
   0
MemtableReclaimMemory 0 0   1574 0  
   0

In the previous iteration, there were four threads for MemtableFlushWriter all 
blocked behind the runnable 
LeveledManifest.getCandidatesFor(LeveledManifest.java:572)


> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Yeah doesn't look blocked. How can i check for the slow flushes?

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-10515:
-

could you post nodetool cfstats and your node config ? This looks like 
CASSANDRA-9882 but that problem was with DTCS and very many sstables.

> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

btw, we tried commitlog_segment_recycling: false but we realized after this 
should already be the default. we briefly thought it made a difference after 
restart that node but the problem did return after several hours. there is some 
mention in another jira about tuning the number of memtable flush writers. 
could this be an issue? it's still difficult to explain though why we only see 
this in a few nodes in the ten clusters all with the same config.

will try to get the thread dump asap.


> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

working on it [~mishail]

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

A second iteration. Ran into a second instance of metrics via RMI but caught it 
very early when only a few were blocked behind the compaction. Still looks like 
the same general place:

{code}
"CompactionExecutor:16" #1502 daemon prio=1 os_prio=4 tid=0x7fb78c4f2000 
nid=0xf7ff runnable [0x7fb751941000]
   java.lang.Thread.State: RUNNABLE
at java.util.HashMap.putVal(HashMap.java:641)
at java.util.HashMap.put(HashMap.java:611)
at java.util.HashSet.add(HashSet.java:219)
at 
org.apache.cassandra.db.compaction.LeveledManifest.overlapping(LeveledManifest.java:512)
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 <0x0004bcf24298> (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 <0x0004bcbec488> (a 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:84)
- locked <0x0004b98f1b00> (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}

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

[~mishail] [~blambov] thread dump attached.

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

I followed the thead dumps over time. a lot of metrics (getting estimated 
pending tasks) blocked behind this thread:

{code}
"CompactionExecutor:11" #1591 daemon prio=1 os_prio=4 tid=0x7f30f1338800 
nid=0xba6b runnable [0x7f2e75bfd000]
   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 <0x000498f172b0> (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 <0x0004989bb5c0> (a 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy)
at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.getNextBackgroundTask(WrappingCompactionStrategy.java:84)
- locked <0x000498151af8> (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}

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Oh look, the memtable flusher is blocked on the same lock:

{code}
"MemtableFlushWriter:1166" #18316 daemon prio=5 os_prio=0 
tid=0x7f33ac5f8800 nid=0xb649 waiting for monitor entry [0x7f31c5acc000]
   java.lang.Thread.State: BLOCKED (on object monitor)
at 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy.handleNotification(WrappingCompactionStrategy.java:250)
- waiting to lock <0x000498151af8> (a 
org.apache.cassandra.db.compaction.WrappingCompactionStrategy)
at org.apache.cassandra.db.DataTracker.notifyAdded(DataTracker.java:518)

{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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

Yeah, i saw all the blocked threads behind it. checking to see what monitoring 
tools are not checking for the previous instance to finish. but this is just an 
ugly side effect, isn't it? (a side effect of lock?) 

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-15 Thread T Jake Luciani (JIRA)

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

T Jake Luciani commented on CASSANDRA-10515:


You seem to have 64 JMX connections open all trying to get compaction stats.

grep "org.apache.cassandra.db.compaction.WrappingCompactionStrategy." 
stacktrace.log

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-14 Thread Jeff Griffith (JIRA)

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

Jeff Griffith commented on CASSANDRA-10515:
---

A test tried: lowered max commit log size to 6g from 12g. It respected the 
limit through 8 iterations then began to grow again (same behavior).

> 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, 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)


[jira] [Commented] (CASSANDRA-10515) Commit logs back up with move to 2.1.10

2015-10-14 Thread Branimir Lambov (JIRA)

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

Branimir Lambov commented on CASSANDRA-10515:
-

Is it possible to share the log of one the affected nodes? As far as I can see 
flushes or post-flush actions have stopped progressing but I do not have enough 
context to understand why.

> 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
>
>
> 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
>   totalunit   progress
> Compaction   SyncCore  *cf1*   61251208033   
> 170643574558   bytes 35.89%
> Compaction   SyncCore  *cf2*   19262483904
> 19266079916   bytes 99.98%
> Compaction   SyncCore  *cf3*6592197093
>  6592316682   bytes100.00%
> Compaction   SyncCore  *cf4*3411039555
>  3411039557   bytes100.00%
> Compaction   SyncCore  *cf5*2879241009
>  2879487621   bytes 99.99%
> Compaction   SyncCore  *cf6*   21252493623
> 21252635196   bytes100.00%
> Compaction   SyncCore  *cf7*   81009853587
> 81009854438   bytes100.00%
> Compaction   SyncCore  *cf8*3005734580
>  3005768582   bytes100.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)