incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Wei Zhu <wz1...@yahoo.com>
Subject Re: Cassandra pending compaction tasks keeps increasing
Date Sat, 02 Feb 2013 01:33:35 GMT
I found it in the source code:

https://github.com/apache/cassandra/blob/cassandra-1.1.0/src/java/org/apache/cassandra/service/StorageService.java#L549


But only branch 1.1.0 has that message logged (See above link), 1.1 and 1.2 doesn't log that
message. We might consider adding that. That would have saved me...

Thanks.
-Wei


________________________________
 From: Wei Zhu <wz1975@yahoo.com>
To: Derek Williams <derek@fyrie.net>; "user@cassandra.apache.org" <user@cassandra.apache.org>

Sent: Friday, February 1, 2013 4:00 PM
Subject: Re: Cassandra pending compaction tasks keeps increasing
 

That is must be it.
Yes. it happens to be the seed. I should have tried "rebuild". Instead I did repair and now
I am sitting here waiting for the compaction to finish...

Thanks.
-Wei


________________________________
 From: Derek Williams <derek@fyrie.net>
To: user@cassandra.apache.org; Wei Zhu <wz1975@yahoo.com> 
Sent: Friday, February 1, 2013 1:56 PM
Subject: Re: Cassandra pending compaction tasks keeps increasing
 

Did the node list itself as a seed node in cassandra.yaml? Unless something has changed, a
node that considers itself a seed will not auto bootstrap. Although I haven't tried it, I
think running 'nodetool rebuild' will cause it to stream in the data it needs without doing
a repair.



On Wed, Jan 30, 2013 at 9:30 PM, Wei Zhu <wz1975@yahoo.com> wrote:

Some updates:
>Since we still have not fully turned on the system. We did something crazy today. We tried
to treat the node as dead one. (My boss wants us to practice replacing a dead node before
going to full production) and boot strap it. Here is what we did:
>
>
>	* drain the node
>	* check nodetool on other nodes, and this node is marked down (the token for this node
is 100)
>
>	* clear the data, commit log, saved cache
>	* change initial_token from 100 to 99 in the yaml file
>	* start the node
>	* check nodetool, the down node of 100 disappeared by itself (!!) and new node with token
99 showed up
>	* checked log, see the message saying bootstrap completed. But only a couple of MB streamed.

>
>	* nodetool movetoken 98
>	* nodetool, see the node with token 98 comes up. 
>
>	* check log, see the message saying bootstrap completed. But still only a couple of MB
streamed. The only reason I can think of is that the new node has the same IP as the "dead"
node we tried to replace? Will that cause  the symptom of no data streamed from other nodes?
Other nodes still think the node had all the data?
>
>We had to do nodetool repair -pr to bring in the data. After 3 hours, 150G  transferred.
And no surprise, pending compaction tasks are now at 30K. There are about 30K SStable transferred
and I guess all of them needs to be compacted since we use LCS.
>
>My concern is that if we did nothing wrong, replacing a dead node will cause such a hugh
back log of pending compaction. It might take a week to clear that off. And we have RF = 3,
we still need to bring in the data for the other two replicates since we use "pr" for nodetool
repair. It will take about 3 weeks to fully replace a
 200G node using LCS? We tried everything we can to speed up the compaction and no luck. The
only thing I can think of is to increase the default size of SSTable, so less number of compaction
will be needed. Can I just change it in yaml and restart C* and it will correct itself? Any
side effect? Since we are using SSD, a bit bigger SSD won't slow down the read too much, I
suppose that is the main concern for bigger size of SSTable?
> 
>I think 1.2 comes with parallel LC which should help the situation. But we are not going
to upgrade for a little while.
>
>Did I miss anything? It might not be practical to use LCS for 200G node? But if  we use
Sized compaction, we need to have at least 400G for the HD...Although SSD is cheap now, still
hard to convince the management. three replicates + double the Disk for compaction? that is
6 times of the real data size!
>
>Sorry for the long email. Any suggestion or advice?
>
>Thanks.
>-Wei 
>
>>________________________________
>
>From: "aaron morton" <aaron@thelastpickle.com>
>To: "Cassandra User" <user@cassandra.apache.org>
>Sent: Tuesday, January 29, 2013 12:59:42 PM
>
>Subject: Re: Cassandra pending compaction tasks keeps increasing
>
>
>* Will try it tomorrow. Do I need to restart server to change the log level?
>>You can set it via JMX, and supposedly log4j is configured to watch the config file. 
>
>
>Cheers
>
>
>-----------------
>Aaron Morton
>Freelance Cassandra Developer
>New Zealand
>
>
>@aaronmorton
>http://www.thelastpickle.com
>
>On 29/01/2013, at 9:36 PM, Wei Zhu <wz1975@yahoo.com> wrote:
>
>Thanks for the reply. Here is some information:
>>
>>Do you have wide rows ? Are you seeing logging about "Compacting wide rows" ? 
>>
>>* I don't see any log about "wide rows"
>>
>>Are you seeing GC activity logged or seeing CPU steal on a VM ? 
>>
>>* There is some GC, but CPU general is under 20%. We have heap size of 8G, RAM is
at 72G.
>>
>>Have you tried disabling multithreaded_compaction ? 
>>
>>* By default, it's disabled. We enabled it, but doesn't see much difference. Even
a little slower with it's enabled. Is it bad to enable it? We have SSD, according to comment
in yaml, it should help while using SSD.
>>
>>Are you using Key Caches ? Have you tried disabling compaction_preheat_key_cache?

>>
>>* We have fairly big Key caches, we set as 10%
 of Heap which is 800M. Yes, compaction_preheat_key_cache is disabled. 
>>
>>Can you enabled DEBUG level logging and make them available ? 
>>
>>* Will try it tomorrow. Do I need to restart server to change the log level?
>>
>>
>>-Wei
>>
>>
>>________________________________
>>
>>
>>From: "aaron morton" <aaron@thelastpickle.com>
>>To: user@cassandra.apache.org
>>Sent: Monday, January 28, 2013 11:31:42 PM
>>Subject: Re: Cassandra pending compaction tasks keeps increasing
>>
>>
>>
>>
>>
>>
>>
>>* Why nodetool repair increases the data size that much? It's not likely that much
data needs to be repaired. Will that happen for all the subsequent repair? 
>>Repair only detects differences in entire rows. If you have very wide rows then small
differences in rows can result in a large amount of streaming. 
>>Streaming creates new SSTables
 on the receiving side, which then need to be compacted. So repair often results in compaction
doing it's thing for a while. 
>>
>>
>>
>>
>>
>>
>>
>>
>>* How to make LCS run faster? After almost a day, the LCS tasks only dropped by 1000.
I am afraid it will never catch up. We set 
>>
>>
>>This is going to be tricky to diagnose, sorry for asking silly questions... 
>>
>>
>>Do you have wide rows ? Are you seeing logging about "Compacting wide rows" ? 
>>Are you seeing GC activity logged or seeing CPU steal on a VM ? 
>>Have you tried disabling multithreaded_compaction ? 
>>Are you using Key Caches ? Have you tried disabling compaction_preheat_key_cache?

>>Can you enabled DEBUG level logging and make them available ? 
>>
>>
>>Cheers 
>>
>>
>>
>>
>>
>>
>>
>>
>>----------------- 
>>Aaron Morton 
>>Freelance Cassandra Developer 
>>New Zealand 
>>
>>
>>@aaronmorton 
>>http://www.thelastpickle.com 
>>
>>
>>On 29/01/2013, at 8:59 AM, Derek Williams < derek@fyrie.net > wrote: 
>>
>>
>>
>>I could be wrong about this, but when repair is run, it isn't just values that are
streamed between nodes, it's entire sstables. This causes a lot of duplicate data to be written
which was already correct on the node, which needs to be compacted away. 
>>
>>
>>As for speeding it up, no idea. 
>>
>>
>>
>>On Mon, Jan 28, 2013 at 12:16 PM, Wei Zhu < wz1975@yahoo.com > wrote: 
>>
>>
>>Any thoughts? 
>>
>>
>>Thanks. 
>>-Wei 
>>
>>----- Original Message ----- 
>>
>>From: "Wei Zhu" < wz1975@yahoo.com > 
>>To: user@cassandra.apache.org 
>>
>>Sent: Friday,
 January 25, 2013 10:09:37 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>
>>
>>To recap the problem, 
>>1.1.6 on SSD, 5 nodes, RF = 3, one CF only. 
>>After data load, initially all 5 nodes have very even data size (135G, each). I ran
nodetool repair -pr on node 1 which have replicates on node 2, node 3 since we set RF = 3.

>>It appears that huge amount of data got transferred. Node 1 has 220G, node 2, 3 have
around 170G. Pending LCS task on node 1 is 15K and node 2, 3 have around 7K each. 
>>Questions: 
>>
>>* Why nodetool repair increases the data size that much? It's not likely that much
data needs to be repaired. Will that happen for all the subsequent repair? 
>>* How to make LCS run faster? After almost a day, the LCS tasks only dropped by 1000.
I am afraid it will never catch up. We set 
>>
>>
>>* compaction_throughput_mb_per_sec = 500 
>>* multithreaded_compaction: true 
>>
>>
>>
>>Both Disk and
 CPU util are less than 10%. I understand LCS is single threaded, any chance to speed it up?

>>
>>
>>* We use default SSTable size as 5M, Will increase the size of SSTable help? What
will happen if I change the setting after the data is loaded. 
>>
>>
>>Any suggestion is very much appreciated. 
>>
>>-Wei 
>>
>>
>>----- Original Message ----- 
>>
>>From: "Wei Zhu" < wz1975@yahoo.com > 
>>To: user@cassandra.apache.org 
>>
>>Sent: Thursday, January 24, 2013 11:46:04 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>I believe I am running into this one: 
>>
>>https://issues.apache.org/jira/browse/CASSANDRA-4765 
>>
>>By the way, I am using 1.1.6 (I though I was using 1.1.7) and this one is fixed in
1.1.7. 
>>
>>
>>
>>----- Original Message ----- 
>>
>>From: "Wei Zhu" < wz1975@yahoo.com > 
>>To: user@cassandra.apache.org 
>>Sent: Thursday, January 24, 2013 11:18:59 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>Thanks Derek, 
>>in the cassandra-env.sh, it says 
>>
>># reduce the per-thread stack size to minimize the impact of Thrift 
>># thread-per-client. (Best practice is for client connections to 
>># be pooled anyway.) Only do so on Linux where it is known to be 
>># supported. 
>># u34 and greater need 180k 
>>JVM_OPTS="$JVM_OPTS -Xss180k" 
>>
>>What value should I use? Java defaults at 400K? Maybe try that first. 
>>
>>Thanks. 
>>-Wei 
>>
>>----- Original Message ----- 
>>From: "Derek Williams" < derek@fyrie.net > 
>>To: user@cassandra.apache.org , "Wei Zhu" < wz1975@yahoo.com > 
>>Sent: Thursday, January 24, 2013 11:06:00 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>
>>Increasing the stack size in cassandra-env.sh should help you get past the stack overflow.
Doesn't help with your original problem though. 
>>
>>
>>
>>On Fri, Jan 25, 2013 at 12:00 AM, Wei Zhu < wz1975@yahoo.com > wrote: 
>>
>>
>>Well, even after restart, it throws the the same exception. I am basically stuck.
Any suggestion to clear the pending compaction tasks? Below is the end of stack trace: 
>>
>>at com.google.common.collect.Sets$1.iterator(Sets.java:578) 
>>at com.google.common.collect.Sets$1.iterator(Sets.java:578) 
>>at
 com.google.common.collect.Sets$1.iterator(Sets.java:578) 
>>at com.google.common.collect.Sets$1.iterator(Sets.java:578) 
>>at com.google.common.collect.Sets$3.iterator(Sets.java:667) 
>>at com.google.common.collect.Sets$3.size(Sets.java:670) 
>>at com.google.common.collect.Iterables.size(Iterables.java:80) 
>>at org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:557) 
>>at org.apache.cassandra.db.compaction.CompactionController.<init>(CompactionController.java:69)

>>at org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:105)

>>at org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)

>>at org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:154)

>>at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) 
>>at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source) 
>>at
 java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source) 
>>at java.util.concurrent.FutureTask.run(Unknown Source) 
>>at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source) 
>>at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) 
>>at java.lang.Thread.run(Unknown Source) 
>>
>>Any suggestion is very much appreciated 
>>
>>-Wei 
>>
>>
>>
>>----- Original Message ----- 
>>From: "Wei Zhu" < wz1975@yahoo.com > 
>>To: user@cassandra.apache.org 
>>Sent: Thursday, January 24, 2013 10:55:07 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>Do you mean 90% of the reads should come from 1 SSTable? 
>>
>>By the way, after I finished the data migrating, I ran nodetool repair -pr on one
of the nodes. Before nodetool repair, all the nodes have the same disk
 space usage. After I ran the nodetool repair, the disk space for that node jumped from 135G
to 220G, also there are more than 15000 pending compaction tasks. After a while , Cassandra
started to throw the exception like below and stop compacting. I had to restart the node.
By the way, we are using 1.1.7. Something doesn't seem right. 
>>
>>
>>INFO [CompactionExecutor:108804] 2013-01-24 22:23:10,427 CompactionTask.java (line
109) Compacting [SSTableReader(path='/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-753782-Data.db')]

>>INFO [CompactionExecutor:108804] 2013-01-24 22:23:11,610 CompactionTask.java (line
221) Compacted to [/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-754996-Data.db,]. 5,259,403
to 5,259,403 (~100% of original) bytes for 1,983 keys at 4.268730MB/s. Time: 1,175ms. 
>>INFO [CompactionExecutor:108805] 2013-01-24 22:23:11,617 CompactionTask.java (line
109) Compacting
 [SSTableReader(path='/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-754880-Data.db')]

>>INFO [CompactionExecutor:108805] 2013-01-24 22:23:12,828 CompactionTask.java (line
221) Compacted to [/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-754997-Data.db,]. 5,272,746
to 5,272,746 (~100% of original) bytes for 1,941 keys at 4.152339MB/s. Time: 1,211ms. 
>>ERROR [CompactionExecutor:108806] 2013-01-24 22:23:13,048 AbstractCassandraDaemon.java
(line 135) Exception in thread Thread[CompactionExecutor:108806,1,main] 
>>java.lang.StackOverflowError 
>>at java.util.AbstractList$Itr.hasNext(Unknown Source) 
>>at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517) 
>>at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114) 
>>at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517) 
>>at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114) 
>>at
 com.google.common.collect.Iterators$5.hasNext(Iterators.java:517) 
>>at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114) 
>>at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517) 
>>at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114) 
>>
>>
>>----- Original Message ----- 
>>From: "aaron morton" < aaron@thelastpickle.com > 
>>To: user@cassandra.apache.org 
>>Sent: Wednesday, January 23, 2013 2:40:45 PM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>The histogram does not look right to me, too many SSTables for an LCS CF. 
>>
>>
>>It's a symptom no a cause. If LCS is catching up though it should be more like the
distribution in the linked article. 
>>
>>
>>Cheers 
>>
>>
>>
>>
>>
>>
>>
>>
>>----------------- 
>>Aaron Morton 
>>Freelance Cassandra Developer 
>>New Zealand 
>>
>>
>>@aaronmorton 
>>http://www.thelastpickle.com 
>>
>>
>>On 23/01/2013, at 10:57 AM, Jim Cistaro < jcistaro@netflix.com > wrote: 
>>
>>
>>
>>
>>What version are you using? Are you seeing any compaction related assertions in the
logs? 
>>
>>
>>Might be https://issues.apache.org/jira/browse/CASSANDRA-4411 
>>
>>
>>We experienced this problem of the count only decreasing to a certain number and then
stopping. If you are idle, it should go to 0. I have not seen it overestimate for zero, only
for non-zero amounts. 
>>
>>
>>As for timeouts etc, you will need to look at things like nodetool tpstats to see
if you have pending transactions queueing up. 
>>
>>
>>Jc 
>>
>>
>>From:
 Wei Zhu < wz1975@yahoo.com > 
>>Reply-To: " user@cassandra.apache.org " < user@cassandra.apache.org >, Wei Zhu
< wz1975@yahoo.com > 
>>Date: Tuesday, January 22, 2013 12:56 PM 
>>To: " user@cassandra.apache.org " < user@cassandra.apache.org > 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>
>>
>>
>>
>>
>>Thanks Aaron and Jim for your reply. The data import is done. We have about 135G on
each node and it's about 28K SStables. For normal operation, we only have about 90 writes
per seconds, but when I ran nodetool compationstats, it remains at
 9 and hardly changes. I guess it's just an estimated number. 
>>
>>
>>When I ran histogram, 
>>
>>
>>
>>Offset SSTables Write Latency Read Latency Row Size Column Count 
>>1 2644 0 0 0 18660057 
>>2 8204 0 0 0 9824270 
>>3 11198 0 0 0 6968475 
>>4 4269 6 0 0 5510745 
>>5 517 29 0 0 4595205 
>>
>>
>>
>>
>>You can see about half of the reads result in 3 SSTables. Majority of read latency
are under 5ms, only a dozen are over 10ms. We haven't fully turn on reads yet, only 60 reads
per second. We see about 20 read timeout during the past 12 hours. Not a single warning from
Cassandra Log. 
>>
>>
>>Is it normal for Cassandra to timeout some requests? We set rpc timeout to be 1s,
it shouldn't time out any of them? 
>>
>>
>>Thanks. 
>>-Wei 
>>
>>
>>
>>
>>
>>From: aaron morton < aaron@thelastpickle.com > 
>>To: user@cassandra.apache.org 
>>Sent: Monday, January 21, 2013 12:21 AM 
>>Subject: Re: Cassandra pending compaction tasks keeps increasing 
>>
>>
>>
>>The main guarantee LCS gives you is that most reads will only touch 1 row http://www.datastax.com/dev/blog/when-to-use-leveled-compaction

>>
>>
>>If compaction is falling behind this may not hold. 
>>
>>
>>nodetool cfhistograms tells you how many SSTables were read from for reads. It's a
recent histogram that resets each time you read from it. 
>>
>>
>>Also, parallel levelled compaction in 1.2 http://www.datastax.com/dev/blog/performance-improvements-in-cassandra-1-2

>>
>>
>>Cheers 
>>
>>
>>
>>
>>
>>
>>
>>
>>----------------- 
>>Aaron Morton 
>>Freelance
 Cassandra Developer 
>>New Zealand 
>>
>>
>>@aaronmorton 
>>http://www.thelastpickle.com 
>>
>>
>>On 20/01/2013, at 7:49 AM, Jim Cistaro < jcistaro@netflix.com > wrote: 
>>
>>
>>
>>
>>
>>1) In addition to iostat, dstat is a good tool to see wht kind of disck throuput your
are getting. That would be one thing to monitor. 
>>2) For LCS, we also see pending compactions skyrocket. During load, LCS will create
a lot of small sstables which will queue up for compaction. 
>>3) For us the biggest concern is not how high the pending count gets, but how often
it gets back down near zero. If your load is something you can do in segments or pause, then
you can see how fast the cluster recovers on the compactions. 
>>4) One thing which we tune per cluster is the size of the files. Increasing this from
5MB can sometimes improve
 things. But I forget if we have ever changed this after starting data load. 
>>
>>
>>Is your cluster receiving read traffic during this data migration? If so, I would
say that read latency is your best measure. If the high number of SSTables waiting to compact
is not hurting your reads, then you are probably ok. Since you are on SSD, there is a good
chance the compactions are not hurting you. As for compactionthroughput, we set ours high
for SSD. You usually wont use it all because the compactions are usually single threaded.
Dstat will help you measure this. 
>>
>>
>>I hope this helps, 
>>jc 
>>
>>
>>From: Wei Zhu < wz1975@yahoo.com > 
>>Reply-To: " user@cassandra.apache.org " < user@cassandra.apache.org >, Wei Zhu
< wz1975@yahoo.com > 
>>Date: Friday, January 18, 2013 12:10 PM 
>>To: Cassandr usergroup < user@cassandra.apache.org > 
>>Subject: Cassandra pending compaction tasks keeps increasing 
>>
>>
>>
>>
>>
>>
>>Hi, 
>>When I run nodetool compactionstats 
>>
>>
>>I see the number of pending tasks keep going up steadily. 
>>
>>
>>I tried to increase the compactionthroughput, by using 
>>
>>
>>nodetool setcompactionthroughput 
>>
>>
>>I even tried the extreme to set it to 0 to disable the throttling. 
>>
>>
>>I checked iostats and we have SSD for data, the disk util is less than 5% which means
it's not I/O bound, CPU is also less than 10% 
>>
>>
>>We are using levelcompaction and in the process of migrating data. We have 4500 writes
per second and very few reads. We have about 70G data now and will grow to 150G when the migration
finishes. We
 only have one CF and right now the number of SSTable is around 15000, write latency is still
under 0.1ms. 
>>
>>
>>Anything needs to be concerned? Or anything I can do to reduce the number of pending
compaction? 
>>
>>
>>Thanks. 
>>-Wei 
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>
>>-- 
>>
>>Derek Williams 
>>
>>
>>
>>
>>
>>
>>
>>-- 
>>
>>Derek Williams 
>>
>>
>
>


-- 

Derek Williams
Mime
View raw message