Return-Path: X-Original-To: apmail-cassandra-user-archive@www.apache.org Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id AD49CE355 for ; Fri, 1 Feb 2013 17:49:47 +0000 (UTC) Received: (qmail 32737 invoked by uid 500); 1 Feb 2013 17:49:45 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 32685 invoked by uid 500); 1 Feb 2013 17:49:45 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 32607 invoked by uid 99); 1 Feb 2013 17:49:45 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 01 Feb 2013 17:49:45 +0000 X-ASF-Spam-Status: No, hits=-0.0 required=5.0 tests=RCVD_IN_DNSWL_NONE,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (athena.apache.org: local policy) Received: from [208.113.200.5] (HELO homiemail-a51.g.dreamhost.com) (208.113.200.5) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 01 Feb 2013 17:49:40 +0000 Received: from homiemail-a51.g.dreamhost.com (localhost [127.0.0.1]) by homiemail-a51.g.dreamhost.com (Postfix) with ESMTP id C5A7A2E807F; Fri, 1 Feb 2013 09:49:19 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha1; c=relaxed; d=thelastpickle.com; h= content-type:mime-version:subject:from:in-reply-to:date :content-transfer-encoding:message-id:references:to; s= thelastpickle.com; bh=HXmFNupzFnAqOS0dz4lvEp0NC/4=; b=x29Li7rd6A hDi1lnMAM+uqftuvvoJZu9QaX4U4FifrFJ2ljjILadQ0N3lRrWSOtPjwdkzaO5/3 s+Akncz+JLJ/19s1mjtI+c8Wpsc+e0Y5BBw/is5nP048/RWSG5yY0fNHA4/YEyKQ CxQzIlh28566apO0BFZ6DPb+EZ4o71h/U= Received: from [172.16.1.8] (unknown [203.86.207.101]) (using TLSv1 with cipher AES128-SHA (128/128 bits)) (No client certificate requested) (Authenticated sender: aaron@thelastpickle.com) by homiemail-a51.g.dreamhost.com (Postfix) with ESMTPSA id D71802E807D; Fri, 1 Feb 2013 09:49:18 -0800 (PST) Content-Type: text/plain; charset=windows-1252 Mime-Version: 1.0 (Mac OS X Mail 6.2 \(1499\)) Subject: Re: Cassandra pending compaction tasks keeps increasing From: aaron morton In-Reply-To: <1359606618.81211.GenericBBA@web160904.mail.bf1.yahoo.com> Date: Sat, 2 Feb 2013 06:49:18 +1300 Content-Transfer-Encoding: quoted-printable Message-Id: <6A7B6693-D4DE-4A98-ACB7-404FE050F28F@thelastpickle.com> References: <1359606618.81211.GenericBBA@web160904.mail.bf1.yahoo.com> To: user@cassandra.apache.org, Wei Zhu X-Mailer: Apple Mail (2.1499) X-Virus-Checked: Checked by ClamAV on apache.org > Will that cause the symptom of no data streamed from other nodes? = Other nodes still think the node had all the data? AFAIk they will not make assumptions like that.=20 > Can I just change it in yaml and restart C* and it will correct = itself? It's a schema config change, check the help for the CLI or the CQL docs.=20= > 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? Do some experiments to see how it works, and let others know :)=20 Cheers ----------------- Aaron Morton Freelance Cassandra Developer New Zealand @aaronmorton http://www.thelastpickle.com On 31/01/2013, at 5:30 PM, Wei Zhu 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: >=20 > =95 drain the node > =95 check nodetool on other nodes, and this node is marked down = (the token for this node is 100) > =95 clear the data, commit log, saved cache > =95 change initial_token from 100 to 99 in the yaml file > =95 start the node > =95 check nodetool, the down node of 100 disappeared by itself = (!!) and new node with token 99 showed up > =95 checked log, see the message saying bootstrap completed. But = only a couple of MB streamed.=20 > =95 nodetool movetoken 98 > =95 nodetool, see the node with token 98 comes up.=20 > =95 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? >=20 > 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. >=20 > 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 =3D 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? > =20 > I think 1.2 comes with parallel LC which should help the situation. = But we are not going to upgrade for a little while. >=20 > 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! >=20 > Sorry for the long email. Any suggestion or advice? >=20 > Thanks. > -Wei=20 >=20 > From: "aaron morton" > To: "Cassandra User" > Sent: Tuesday, January 29, 2013 12:59:42 PM > Subject: Re: Cassandra pending compaction tasks keeps increasing >=20 > * 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.=20 >=20 > Cheers >=20 > ----------------- > Aaron Morton > Freelance Cassandra Developer > New Zealand >=20 > @aaronmorton > http://www.thelastpickle.com >=20 > On 29/01/2013, at 9:36 PM, Wei Zhu wrote: >=20 > Thanks for the reply. Here is some information: >=20 > Do you have wide rows ? Are you seeing logging about "Compacting wide = rows" ?=20 >=20 > * I don't see any log about "wide rows" >=20 > Are you seeing GC activity logged or seeing CPU steal on a VM ?=20 >=20 > * There is some GC, but CPU general is under 20%. We have heap size of = 8G, RAM is at 72G. >=20 > Have you tried disabling multithreaded_compaction ?=20 >=20 > * 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. >=20 > Are you using Key Caches ? Have you tried disabling = compaction_preheat_key_cache?=20 >=20 > * We have fairly big Key caches, we set as 10% of Heap which is 800M. = Yes, compaction_preheat_key_cache is disabled.=20 >=20 > Can you enabled DEBUG level logging and make them available ?=20 >=20 > * Will try it tomorrow. Do I need to restart server to change the log = level? >=20 >=20 > -Wei >=20 >=20 > From: "aaron morton" > To: user@cassandra.apache.org > Sent: Monday, January 28, 2013 11:31:42 PM > Subject: Re: Cassandra pending compaction tasks keeps increasing >=20 >=20 >=20 >=20 >=20 >=20 >=20 > * 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?=20 > 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.=20 > 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.=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > * 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=20 >=20 >=20 > This is going to be tricky to diagnose, sorry for asking silly = questions...=20 >=20 >=20 > Do you have wide rows ? Are you seeing logging about "Compacting wide = rows" ?=20 > Are you seeing GC activity logged or seeing CPU steal on a VM ?=20 > Have you tried disabling multithreaded_compaction ?=20 > Are you using Key Caches ? Have you tried disabling = compaction_preheat_key_cache?=20 > Can you enabled DEBUG level logging and make them available ?=20 >=20 >=20 > Cheers=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > -----------------=20 > Aaron Morton=20 > Freelance Cassandra Developer=20 > New Zealand=20 >=20 >=20 > @aaronmorton=20 > http://www.thelastpickle.com=20 >=20 >=20 > On 29/01/2013, at 8:59 AM, Derek Williams < derek@fyrie.net > wrote:=20= >=20 >=20 >=20 > 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.=20 >=20 >=20 > As for speeding it up, no idea.=20 >=20 >=20 >=20 > On Mon, Jan 28, 2013 at 12:16 PM, Wei Zhu < wz1975@yahoo.com > wrote:=20= >=20 >=20 > Any thoughts?=20 >=20 >=20 > Thanks.=20 > -Wei=20 >=20 > ----- Original Message -----=20 >=20 > From: "Wei Zhu" < wz1975@yahoo.com >=20 > To: user@cassandra.apache.org=20 >=20 > Sent: Friday, January 25, 2013 10:09:37 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 >=20 >=20 > To recap the problem,=20 > 1.1.6 on SSD, 5 nodes, RF =3D 3, one CF only.=20 > 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 =3D 3.=20 > 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.=20 > Questions:=20 >=20 > * 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?=20 > * 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=20 >=20 >=20 > * compaction_throughput_mb_per_sec =3D 500=20 > * multithreaded_compaction: true=20 >=20 >=20 >=20 > Both Disk and CPU util are less than 10%. I understand LCS is single = threaded, any chance to speed it up?=20 >=20 >=20 > * 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.=20= >=20 >=20 > Any suggestion is very much appreciated.=20 >=20 > -Wei=20 >=20 >=20 > ----- Original Message -----=20 >=20 > From: "Wei Zhu" < wz1975@yahoo.com >=20 > To: user@cassandra.apache.org=20 >=20 > Sent: Thursday, January 24, 2013 11:46:04 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 > I believe I am running into this one:=20 >=20 > https://issues.apache.org/jira/browse/CASSANDRA-4765=20 >=20 > 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.=20 >=20 >=20 >=20 > ----- Original Message -----=20 >=20 > From: "Wei Zhu" < wz1975@yahoo.com >=20 > To: user@cassandra.apache.org=20 > Sent: Thursday, January 24, 2013 11:18:59 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 > Thanks Derek,=20 > in the cassandra-env.sh, it says=20 >=20 > # reduce the per-thread stack size to minimize the impact of Thrift=20 > # thread-per-client. (Best practice is for client connections to=20 > # be pooled anyway.) Only do so on Linux where it is known to be=20 > # supported.=20 > # u34 and greater need 180k=20 > JVM_OPTS=3D"$JVM_OPTS -Xss180k"=20 >=20 > What value should I use? Java defaults at 400K? Maybe try that first.=20= >=20 > Thanks.=20 > -Wei=20 >=20 > ----- Original Message -----=20 > From: "Derek Williams" < derek@fyrie.net >=20 > To: user@cassandra.apache.org , "Wei Zhu" < wz1975@yahoo.com >=20 > Sent: Thursday, January 24, 2013 11:06:00 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 >=20 > Increasing the stack size in cassandra-env.sh should help you get past = the stack overflow. Doesn't help with your original problem though.=20 >=20 >=20 >=20 > On Fri, Jan 25, 2013 at 12:00 AM, Wei Zhu < wz1975@yahoo.com > wrote:=20= >=20 >=20 > 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:=20 >=20 > at com.google.common.collect.Sets$1.iterator(Sets.java:578)=20 > at com.google.common.collect.Sets$1.iterator(Sets.java:578)=20 > at com.google.common.collect.Sets$1.iterator(Sets.java:578)=20 > at com.google.common.collect.Sets$1.iterator(Sets.java:578)=20 > at com.google.common.collect.Sets$3.iterator(Sets.java:667)=20 > at com.google.common.collect.Sets$3.size(Sets.java:670)=20 > at com.google.common.collect.Iterables.size(Iterables.java:80)=20 > at = org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:557= )=20 > at = org.apache.cassandra.db.compaction.CompactionController.(CompactionC= ontroller.java:69)=20 > at = org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.j= ava:105)=20 > at = org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCo= mpactionTask.java:50)=20 > at = org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(Compact= ionManager.java:154)=20 > at = org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)=20= > at java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)=20= > at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)=20 > at java.util.concurrent.FutureTask.run(Unknown Source)=20 > at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown = Source)=20 > at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)=20= > at java.lang.Thread.run(Unknown Source)=20 >=20 > Any suggestion is very much appreciated=20 >=20 > -Wei=20 >=20 >=20 >=20 > ----- Original Message -----=20 > From: "Wei Zhu" < wz1975@yahoo.com >=20 > To: user@cassandra.apache.org=20 > Sent: Thursday, January 24, 2013 10:55:07 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 > Do you mean 90% of the reads should come from 1 SSTable?=20 >=20 > 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.=20 >=20 >=20 > INFO [CompactionExecutor:108804] 2013-01-24 22:23:10,427 = CompactionTask.java (line 109) Compacting = [SSTableReader(path=3D'/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-= 753782-Data.db')]=20 > 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.=20 > INFO [CompactionExecutor:108805] 2013-01-24 22:23:11,617 = CompactionTask.java (line 109) Compacting = [SSTableReader(path=3D'/ssd/cassandra/data/zoosk/friends/zoosk-friends-hf-= 754880-Data.db')]=20 > 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.=20 > ERROR [CompactionExecutor:108806] 2013-01-24 22:23:13,048 = AbstractCassandraDaemon.java (line 135) Exception in thread = Thread[CompactionExecutor:108806,1,main]=20 > java.lang.StackOverflowError=20 > at java.util.AbstractList$Itr.hasNext(Unknown Source)=20 > at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)=20= > at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)=20= > at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)=20= > at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)=20= > at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)=20= > at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)=20= > at com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)=20= > at com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)=20= >=20 >=20 > ----- Original Message -----=20 > From: "aaron morton" < aaron@thelastpickle.com >=20 > To: user@cassandra.apache.org=20 > Sent: Wednesday, January 23, 2013 2:40:45 PM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 > The histogram does not look right to me, too many SSTables for an LCS = CF.=20 >=20 >=20 > It's a symptom no a cause. If LCS is catching up though it should be = more like the distribution in the linked article.=20 >=20 >=20 > Cheers=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > -----------------=20 > Aaron Morton=20 > Freelance Cassandra Developer=20 > New Zealand=20 >=20 >=20 > @aaronmorton=20 > http://www.thelastpickle.com=20 >=20 >=20 > On 23/01/2013, at 10:57 AM, Jim Cistaro < jcistaro@netflix.com > = wrote:=20 >=20 >=20 >=20 >=20 > What version are you using? Are you seeing any compaction related = assertions in the logs?=20 >=20 >=20 > Might be https://issues.apache.org/jira/browse/CASSANDRA-4411=20 >=20 >=20 > 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.=20 >=20 >=20 > As for timeouts etc, you will need to look at things like nodetool = tpstats to see if you have pending transactions queueing up.=20 >=20 >=20 > Jc=20 >=20 >=20 > From: Wei Zhu < wz1975@yahoo.com >=20 > Reply-To: " user@cassandra.apache.org " < user@cassandra.apache.org >, = Wei Zhu =20 > Date: Tuesday, January 22, 2013 12:56 PM=20 > To: " user@cassandra.apache.org " < user@cassandra.apache.org >=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 >=20 >=20 >=20 >=20 >=20 > 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.=20 >=20 >=20 > When I ran histogram,=20 >=20 >=20 >=20 > Offset SSTables Write Latency Read Latency Row Size Column Count=20 > 1 2644 0 0 0 18660057=20 > 2 8204 0 0 0 9824270=20 > 3 11198 0 0 0 6968475=20 > 4 4269 6 0 0 5510745=20 > 5 517 29 0 0 4595205=20 >=20 >=20 >=20 >=20 > 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.=20 >=20 >=20 > Is it normal for Cassandra to timeout some requests? We set rpc = timeout to be 1s, it shouldn't time out any of them?=20 >=20 >=20 > Thanks.=20 > -Wei=20 >=20 >=20 >=20 >=20 >=20 > From: aaron morton < aaron@thelastpickle.com >=20 > To: user@cassandra.apache.org=20 > Sent: Monday, January 21, 2013 12:21 AM=20 > Subject: Re: Cassandra pending compaction tasks keeps increasing=20 >=20 >=20 >=20 > The main guarantee LCS gives you is that most reads will only touch 1 = rowhttp://www.datastax.com/dev/blog/when-to-use-leveled-compaction=20 >=20 >=20 > If compaction is falling behind this may not hold.=20 >=20 >=20 > 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.=20= >=20 >=20 > Also, parallel levelled compaction in 1.2 = http://www.datastax.com/dev/blog/performance-improvements-in-cassandra-1-2= =20 >=20 >=20 > Cheers=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > -----------------=20 > Aaron Morton=20 > Freelance Cassandra Developer=20 > New Zealand=20 >=20 >=20 > @aaronmorton=20 > http://www.thelastpickle.com=20 >=20 >=20 > On 20/01/2013, at 7:49 AM, Jim Cistaro < jcistaro@netflix.com > wrote:=20= >=20 >=20 >=20 >=20 >=20 > 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.=20 > 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.=20 > 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.=20 > 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.=20 >=20 >=20 > 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.=20 >=20 >=20 > I hope this helps,=20 > jc=20 >=20 >=20 > From: Wei Zhu < wz1975@yahoo.com >=20 > Reply-To: " user@cassandra.apache.org " < user@cassandra.apache.org >, = Wei Zhu =20 > Date: Friday, January 18, 2013 12:10 PM=20 > To: Cassandr usergroup < user@cassandra.apache.org >=20 > Subject: Cassandra pending compaction tasks keeps increasing=20 >=20 >=20 >=20 >=20 >=20 >=20 > Hi,=20 > When I run nodetool compactionstats=20 >=20 >=20 > I see the number of pending tasks keep going up steadily.=20 >=20 >=20 > I tried to increase the compactionthroughput, by using=20 >=20 >=20 > nodetool setcompactionthroughput=20 >=20 >=20 > I even tried the extreme to set it to 0 to disable the throttling.=20 >=20 >=20 > 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%=20 >=20 >=20 > 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.=20 >=20 >=20 > Anything needs to be concerned? Or anything I can do to reduce the = number of pending compaction?=20 >=20 >=20 > Thanks.=20 > -Wei=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > --=20 >=20 > Derek Williams=20 >=20 >=20 >=20 >=20 >=20 >=20 >=20 > --=20 >=20 > Derek Williams