CPU hotspot at BloomFilterSerializer#deserialize

2013-01-30 Thread Takenori Sato
Hi all, We have a situation that CPU loads on some of our nodes in a cluster has spiked occasionally since the last November, which is triggered by requests for rows that reside on two specific sstables. We confirmed the followings(when spiked): version: 1.0.7(current) <- 0.8.6 <- 0.8.5 <- 0.7.8

Re: CPU hotspot at BloomFilterSerializer#deserialize

2013-02-03 Thread Takenori Sato
niques… > > Get a single named column. > Get the first 10 columns using the natural column order. > Get the last 10 columns using the reversed order. > > Hope that helps. > > - > Aaron Morton > Freelance Cassandra Developer > New Zealand > > @

A fix for those who suffer from GC storm by tombstones

2014-10-07 Thread Takenori Sato
Hi, I have filed a fix as CASSANDRA-8038, which would be a good news for those who has suffered from overwhelming GC or OOM by tombstones. Appreciate your feedbacks! Thanks, Takenori

Re: A fix for those who suffer from GC storm by tombstones

2014-10-07 Thread Takenori Sato
DuyHi and Rob, Thanks for your feedbacks. Yeah, that's exactly the point I found. Some may want to run read repair even on tombstones as before, but others not like Rob and us. Personally, I take read repaid as a nice to have feature, especially for tombstones, where a regular repair is anyway

Re: Re[2]: how wide can wide rows get?

2014-11-13 Thread Takenori Sato
We have up to a few hundreds of millions of columns in a super wide row. There are two major issues you should care about. 1. the wider the row is, the more memory pressure you get for every slice query 2. repair is row based, which means a huge row could be transferred at every repair 1 is not

Re: Cass 1.1.11 out of memory during compaction ?

2013-11-03 Thread Takenori Sato
Try increasing column_index_size_in_kb. A slice query to get some ranges(SliceFromReadCommand) requires to read all the column indexes for the row, thus could hit OOM if you have a very wide row. On Sun, Nov 3, 2013 at 11:54 PM, Oleg Dulin wrote: > Cass 1.1.11 ran out of memory on me with thi

Re: Cass 1.1.11 out of memory during compaction ?

2013-11-04 Thread Takenori Sato
I would go with cleanup. Be careful for this bug. https://issues.apache.org/jira/browse/CASSANDRA-5454 On Mon, Nov 4, 2013 at 9:05 PM, Oleg Dulin wrote: > If i do that, wouldn't I need to scrub my sstables ? > > > Takenori Sato wrote: > > Try increasing column_inde

Re: Tracing Queries at Cassandra Server

2013-11-10 Thread Takenori Sato
In addition to CassandraServer, add StorageProxy for details as follows. log4j.logger.org.apache.cassandra.service.StorageProxy=DEBUG log4j.logger.org.apache.cassandra.thrift.CassandraServer=DEBUG Hope it would help. On Mon, Nov 11, 2013 at 11:25 AM, Srinath Perera wrote: > I am talking to Ca

Re: Recommended amount of free disk space for compaction

2013-11-29 Thread Takenori Sato
Hi, > If Cassandra only compacts one table at a time, then I should be safe if I keep as much free space as there is data in the largest table. If Cassandra can compact multiple tables simultaneously, then it seems that I need as much free space as all the tables put together, which means no more

Re: need help with Cassandra 1.2 Full GCing -- output of jmap histogram

2014-03-09 Thread Takenori Sato
You have millions of org.apache.cassandra.db.DeletedColumn instances on the snapshot. This means you have lots of column tombstones, and I guess, which are read into memory by slice query. On Sun, Mar 9, 2014 at 10:55 PM, Oleg Dulin wrote: > I am trying to understand why one of my nodes keeps

Re: need help with Cassandra 1.2 Full GCing -- output of jmap histogram

2014-03-11 Thread Takenori Sato
-tools On Mon, Mar 10, 2014 at 7:41 PM, Oleg Dulin wrote: > I get that :) > > What I'd like to know is how to fix that :) > > > On 2014-03-09 20:24:54 +, Takenori Sato said: > > You have millions of org.apache.cassandra.db.DeletedColumn instances on >>

Re: Cleanup understastanding

2013-05-29 Thread Takenori Sato
gt; *Víctor Hugo Molinar - *@vhmolinar <http://twitter.com/#!/vhmolinar> > > > On Tue, May 28, 2013 at 9:55 PM, Takenori Sato(Cloudian) < > ts...@cloudian.com> wrote: > >> Hi Victor, >> >> As Andrey said, running cleanup doesn't work as you expect. >&

Re: Reduce Cassandra GC

2013-06-15 Thread Takenori Sato
> INFO [ScheduledTasks:1] 2013-04-15 14:00:02,749 GCInspector.java (line 122) GC for ParNew: 338798 ms for 1 collections, 592212416 used; max is 1046937600 This says GC for New Generation took so long. And this is usually unlikely. The only situation I am aware of is when a fairly large object is

Re: Reduce Cassandra GC

2013-06-15 Thread Takenori Sato
that we can compare it? > > Quick thing to do would be to do a histo live at 2 points and compare > > Sent from my iPhone > > On Jun 15, 2013, at 6:57 AM, Takenori Sato wrote: > > > INFO [ScheduledTasks:1] 2013-04-15 14:00:02,749 GCInspector.java (line > 122) GC for Pa

Re: Reduce Cassandra GC

2013-06-15 Thread Takenori Sato
13 at 4:44 AM, Mohit Anchlia wrote: > Can you paste you gc config? Also can you take a heap dump at 2 diff > points so that we can compare it? > > Quick thing to do would be to do a histo live at 2 points and compare > > Sent from my iPhone > > On Jun 15, 2013, at 6:57

Re: Reduce Cassandra GC

2013-06-17 Thread Takenori Sato
Find "promotion failure". Bingo if it happened at the time. Otherwise, post the relevant portion of the log here. Someone may find a hint. On Mon, Jun 17, 2013 at 5:51 PM, Joel Samuelsson wrote: > Just got a very long GC again. What am I to look for in the logging I just > enabled? > > > 2013/6

Re: Reduce Cassandra GC

2013-06-18 Thread Takenori Sato
> INFO [ScheduledTasks:1] 2013-06-17 08:13:47,518 StatusLogger.java (line > 116) OpsCenter.rollups300 2516,570931 > INFO [ScheduledTasks:1] 2013-06-17 08:13:47,519 StatusLogger.java (line > 116) OpsCenter.pdps9072,160850 > INFO [ScheduledTasks:1] 2013

Re: Reduce Cassandra GC

2013-06-19 Thread Takenori Sato
GC options are not set. You should see the followings. -XX:+PrintGCDateStamps -XX:+PrintPromotionFailure -Xloggc:/var/log/cassandra/gc-1371603607.log > Is it normal to have two processes like this? No. You are running two processes. On Wed, Jun 19, 2013 at 4:16 PM, Joel Samuelsson wrote: > M

Re: Alternate "major compaction"

2013-07-11 Thread Takenori Sato
Hi, I think it is a common headache for users running a large Cassandra cluster in production. Running a major compaction is not the only cause, but more. For example, I see two typical scenario. 1. backup use case 2. active wide row In the case of 1, say, one data is removed a year later. Thi

Re: Alternate "major compaction"

2013-07-11 Thread Takenori Sato
e tool provides some good info that people can use. It would > be great if you can share it with the community. > > > > > On Thu, Jul 11, 2013 at 6:51 AM, Takenori Sato wrote: > >> Hi, >> >> I think it is a common headache for users running a large Cassandra >&

Re: Alternate "major compaction"

2013-07-12 Thread Takenori Sato
It's light. Without -v option, you can even run it against just a SSTable file without needing the whole Cassandra installation. - Takenori On Sat, Jul 13, 2013 at 6:18 AM, Robert Coli wrote: > On Thu, Jul 11, 2013 at 9:43 PM, Takenori Sato wrote: > >> I made the repositor

Fp chance for column level bloom filter

2013-07-17 Thread Takenori Sato
Hi, I thought memory consumption of column level bloom filter will become a big concern when a row becomes very wide like more than tens of millions of columns. But I read from source(1.0.7) that fp chance for column level bloom filter is hard-coded as 0.160, which is very high. So seems not. Is

Random Distribution, yet Order Preserving Partitioner

2013-08-22 Thread Takenori Sato
Hi, I am trying to implement a custom partitioner that evenly distributes, yet preserves order. The partitioner returns a token by BigInteger as RandomPartitioner does, while does a decorated key by string as OrderPreservingPartitioner does. * for now, since IPartitioner does not support differen

Re: Random Distribution, yet Order Preserving Partitioner

2013-08-22 Thread Takenori Sato
, all cities in US and so on. > however in this way things may get pretty non-ballanced > > Nick > > > > > > On Thu, Aug 22, 2013 at 11:19 AM, Takenori Sato wrote: > >> Hi, >> >> I am trying to implement a custom partitioner that evenly distributes, >

OrderPreservingPartitioner in 1.2

2013-08-23 Thread Takenori Sato
Hi, I know it has been depreciated, but OrderPreservingPartitioner still works with 1.2? Just wanted to know how it works, but I got a couple of exceptions as below: ERROR [GossipStage:2] 2013-08-23 07:03:57,171 CassandraDaemon.java (line 175) Exception in thread Thread[GossipStage:2,5,main] jav

Re: Random Distribution, yet Order Preserving Partitioner

2013-08-27 Thread Takenori Sato
dered fashion. Then you can do multi_get to get you actual data. The downside is you have to do two read queries, and depending on how much data you have you will end up with a wide metadata row. Manoj On Fri, Aug 23, 2013 at 8:47 AM, Takenori Sato wrote: > Hi Nick, > > > token and key

/proc/sys/vm/zone_reclaim_mode

2013-09-09 Thread Takenori Sato
Hi, I am investigating NUMA issues. I have been aware that bin/cassandra tries to use interleave all policy if available. https://issues.apache.org/jira/browse/CASSANDRA-2594 https://issues.apache.org/jira/browse/CASSANDRA-3245 So what about /proc/sys/vm/zone_reclaim_mode? Any recommendations?

Re: questions related to the SSTable file

2013-09-17 Thread Takenori Sato
> So in fact, incremental backup of Cassandra is just hard link all the new SSTable files being generated during the incremental backup period. It could contain any data, not just the data being update/insert/delete in this period, correct? Correct. But over time, some old enough SSTable files ar

Re: questions related to the SSTable file

2013-09-17 Thread Takenori Sato
> > Yong > > -- > Date: Tue, 17 Sep 2013 18:01:45 -0700 > > Subject: Re: questions related to the SSTable file > From: rc...@eventbrite.com > To: user@cassandra.apache.org > > > On Tue, Sep 17, 2013 at 5:46 PM, Takenori Sato wrote: > > &g

Re: CPU hotspot at BloomFilterSerializer#deserialize

2013-02-05 Thread Takenori Sato(Cloudian)
teresting the press c* got about having 2 billion columns in a row. You *can* do it but it brings to light some realities of what that means. On Sun, Feb 3, 2013 at 8:09 AM, Takenori Sato <mailto:ts...@cloudian.com>> wrote: Hi Aaron, Thanks for your answers. That helped me get a big pict

Re: -pr vs. no -pr

2013-02-28 Thread Takenori Sato(Cloudian)
Hi, Please note that I confirmed on v1.0.7. > I mean a repair involves all three nodes and pushes and pulls data, right? Yes, but that's how -pr works. A repair without -pr does more. For example, suppose you have a ring with RF=3 like this. A - B - C - D - E - F Then, a repair on A withou

Re: Cleanup understastanding

2013-05-28 Thread Takenori Sato(Cloudian)
Hi Victor, As Andrey said, running cleanup doesn't work as you expect. > The reason I need to clean things is that I wont need most of my inserted data on the next day. Deleted objects(columns/records) become deletable from sstable file when they get expired(after gc_grace_seconds). Such d

Re: OrderPreservingPartitioner in 1.2

2013-08-25 Thread Takenori Sato(Cloudian)
For the first exception: OPP was not working in 1.2. It has been fixed but not yet there in latest 1.2.8 version. Jira issue about it: https://issues.apache.org/jira/browse/CASSANDRA-5793 On Fri, Aug 23, 2013 at 12:51 PM, Takenori Sato <mailto:ts...@cloudian.com>> wrote: Hi, I

Re: questions related to the SSTable file

2013-09-16 Thread Takenori Sato(Cloudian)
Hi, > 1) I will expect same row key could show up in both sstable2json output, as this one row exists in both SSTable files, right? Yes. > 2) If so, what is the boundary? Will Cassandra guarantee the column level as the boundary? What I mean is that for one column's data, it will be guarant

Re: questions related to the SSTable file

2013-09-17 Thread Takenori Sato(Cloudian)
Thanks, Rob, for clarifying! - Takenori (2013/09/18 10:01), Robert Coli wrote: On Tue, Sep 17, 2013 at 5:46 PM, Takenori Sato <mailto:ts...@cloudian.com>> wrote: > So in fact, incremental backup of Cassandra is just hard link all the new SSTable files being generate