SST Inconsistency

2012-10-01 Thread Daniel Doubleday

Hi all

we are running c* 1.0.8 and found some strange row level tombstone problems.

Some rows (~50 in around 2B keys) have markedForDeleteAt timestamps in 
the future (so they 'drop' all writes) and 0 values as localDeletionTime.


A non-thorough check didn't bring up any code paths that could lead to a 
0 localDeletionTime and our client code is not capable to produce the 
markedForDeleteAt values we are seeing.
First we thought that it's file corruption. But we have multiple SSTs on 
different nodes that show the same pattern (though I did not check yet 
whether all belong to the same parition). Also all files are perfectly 
readable and the columns in the deleted rows are correct too.


I searched Jira but couldn't find anything related. So I was wondering 
if anyone is aware of a bug that could have led to this?


I'm going to patch scrub to fix the SSTs so it's not a big deal. Still 
strange ...


Cheers,
Daniel


Re: architectural understanding of write operation node flow

2012-01-23 Thread Daniel Doubleday
Your first thought was pretty much correct:

1. The node which is called by the client is the coordinator
2. The coordinator determines the nodes in the ring which can handle the 
request ordered by expected latency (via snitch). The coordinator may or may 
not be part of these nodes
3. Given the consistency level and read repair chance the coordinator 
calculates the min amount of node to ask and sends read requests to them
4. As soon as the minimum count (according to consistency) of responses is 
collected the coordinator will respond to the request. Mismatches will lead to 
repair write requests to the corresponding nodes

Thus the minimal depth is one (CL = 1 and coordinator can handle the request 
itself) or two otherwise.

Hope that helps

On Jan 23, 2012, at 4:47 PM, Peter Dijkshoorn wrote:

 
 Hi guys,
 
 I got an architectural question about how a write operation flows
 through the nodes.
 
 As far as I understand now, a client sends its write operation to
 whatever node it was set to use and if that node does not contain the
 data for this key K, then this node forwards the operation to the first
 node given by the hash function. This first node having key K then
 contacts the replication nodes depending on the selected consistency level.
 
 This means that in the unlucky event you always have a network call
 sequence depth of 2 (consistency level one), or 3 (assumed that the
 replication nodes are contacted in parallel)
 
 This is more than I expected, so I am not sure whether this is correct?
 can someone help me out?
 
 At first I thought that the receiver was the coordinator, and thus doing
 all further calls in parallel, the depth as described above would always
 be 2. But I just discovered that I was wrong and that it should be
 something like above.
 
 Another possibility would be that the client learnt the layout of the
 cluster at connection time and thereby tries per request to contact the
 coordinator directly, but I never read or see something like this happening.
 
 Remembering the picture of Dean about network and hard disk latencies,
 is this 3-sequential-network-call still faster?
 
 Thanks for any thoughts :)
 
 Peter
 
 -- 
 Peter Dijkshoorn
 Adyen - Payments Made Easy
 www.adyen.com
 
 Visiting Address: Mail Address:
 Stationsplein 57 - 4th floor  P.O. Box 10095
 1012 AB Amsterdam 1001 EB Amsterdam
 The Netherlands   The Netherlands
 
 Office +31.20.240.1240
 Email peter.dijksho...@adyen.com
 



Re: architectural understanding of write operation node flow

2012-01-23 Thread Daniel Doubleday
Ouch :-) you were asking write ...

Well kind of similar 

1. Coordinator calculates all nodes
2. If not enough (according to CL) nodes are alive it throughs unavailable
3. If nodes are down it writes and hh is enabled it writes a hint for that row
4. It sends write request to all nodes (including itself / shortcutting 
messaging)
5. If it receives enough (according to CL) acks before timeout everything is 
fine otherwise it throughs unavailable

errm .. I'm more confident in the read path though especially concerning hh 
handling so I'm happy to be corrected here. I.e. I'm not sure if hints are 
written when request time out but CL is reached.

On Jan 23, 2012, at 6:47 PM, Daniel Doubleday wrote:

 Your first thought was pretty much correct:
 
 1. The node which is called by the client is the coordinator
 2. The coordinator determines the nodes in the ring which can handle the 
 request ordered by expected latency (via snitch). The coordinator may or may 
 not be part of these nodes
 3. Given the consistency level and read repair chance the coordinator 
 calculates the min amount of node to ask and sends read requests to them
 4. As soon as the minimum count (according to consistency) of responses is 
 collected the coordinator will respond to the request. Mismatches will lead 
 to repair write requests to the corresponding nodes
 
 Thus the minimal depth is one (CL = 1 and coordinator can handle the request 
 itself) or two otherwise.
 
 Hope that helps
 
 On Jan 23, 2012, at 4:47 PM, Peter Dijkshoorn wrote:
 
 
 Hi guys,
 
 I got an architectural question about how a write operation flows
 through the nodes.
 
 As far as I understand now, a client sends its write operation to
 whatever node it was set to use and if that node does not contain the
 data for this key K, then this node forwards the operation to the first
 node given by the hash function. This first node having key K then
 contacts the replication nodes depending on the selected consistency level.
 
 This means that in the unlucky event you always have a network call
 sequence depth of 2 (consistency level one), or 3 (assumed that the
 replication nodes are contacted in parallel)
 
 This is more than I expected, so I am not sure whether this is correct?
 can someone help me out?
 
 At first I thought that the receiver was the coordinator, and thus doing
 all further calls in parallel, the depth as described above would always
 be 2. But I just discovered that I was wrong and that it should be
 something like above.
 
 Another possibility would be that the client learnt the layout of the
 cluster at connection time and thereby tries per request to contact the
 coordinator directly, but I never read or see something like this happening.
 
 Remembering the picture of Dean about network and hard disk latencies,
 is this 3-sequential-network-call still faster?
 
 Thanks for any thoughts :)
 
 Peter
 
 -- 
 Peter Dijkshoorn
 Adyen - Payments Made Easy
 www.adyen.com
 
 Visiting Address: Mail Address:
 Stationsplein 57 - 4th floor  P.O. Box 10095
 1012 AB Amsterdam 1001 EB Amsterdam
 The Netherlands   The Netherlands
 
 Office +31.20.240.1240
 Email peter.dijksho...@adyen.com
 
 



Re: Second Cassandra users survey

2011-11-08 Thread Daniel Doubleday
Ah cool - thanks for the pointer!

On Nov 7, 2011, at 5:25 PM, Ed Anuff wrote:

 This is basically what entity groups are about -
 https://issues.apache.org/jira/browse/CASSANDRA-1684
 
 On Mon, Nov 7, 2011 at 5:26 AM, Peter Lin wool...@gmail.com wrote:
 This feature interests me, so I thought I'd add some comments.
 
 Having used partition features in existing databases like DB2, Oracle
 and manual partitioning, one of the biggest challenges is keeping the
 partitions balanced. What I've seen with manual partitioning is that
 often the partitions get unbalanced. Usually the developers take a
 best guess and hope it ends up balanced.
 
 Some of the approaches I've used in the past were zip code, area code,
 state and some kind of hash.
 
 So my question related deterministic sharding is this, what rebalance
 feature(s) would be useful or needed once the partitions get
 unbalanced?
 
 Without a decent plan for rebalancing, it often ends up being a very
 painful problem to solve in production. Back when I worked mobile
 apps, we saw issues with how OpenWave WAP servers partitioned the
 accounts. The early versions randomly assigned a phone to a server
 when it is provisioned the first time. Once the phone was associated
 to that server, it was stuck on that server. If the load on that
 server was heavier than the others, the only choice was to scale up
 the hardware.
 
 My understanding of Cassandra's current sharding is consistent and
 random. Does the new feature sit some where in-between? Are you
 thinking of a pluggable API so that you can provide your own hash
 algorithm for cassandra to use?
 
 
 
 On Mon, Nov 7, 2011 at 7:54 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Allow for deterministic / manual sharding of rows.
 
 Right now it seems that there is no way to force rows with different row 
 keys will be stored on the same nodes in the ring.
 This is our number one reason why we get data inconsistencies when nodes 
 fail.
 
 Sometimes a logical transaction requires writing rows with different row 
 keys. If we could use something like this:
 
 prefix.uniquekey and let the partitioner use only the prefix the 
 probability that only part of the transaction would be written could be 
 reduced considerably.
 
 
 
 On Nov 1, 2011, at 11:59 PM, Jonathan Ellis wrote:
 
 Hi all,
 
 Two years ago I asked for Cassandra use cases and feature requests.
 [1]  The results [2] have been extremely useful in setting and
 prioritizing goals for Cassandra development.  But with the release of
 1.0 we've accomplished basically everything from our original wish
 list. [3]
 
 I'd love to hear from modern Cassandra users again, especially if
 you're usually a quiet lurker.  What does Cassandra do well?  What are
 your pain points?  What's your feature wish list?
 
 As before, if you're in stealth mode or don't want to say anything in
 public, feel free to reply to me privately and I will keep it off the
 record.
 
 [1] 
 http://www.mail-archive.com/cassandra-dev@incubator.apache.org/msg01148.html
 [2] 
 http://www.mail-archive.com/cassandra-user@incubator.apache.org/msg01446.html
 [3] http://www.mail-archive.com/dev@cassandra.apache.org/msg01524.html
 
 --
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of DataStax, the source for professional Cassandra support
 http://www.datastax.com
 
 
 



Re: Second Cassandra users survey

2011-11-07 Thread Daniel Doubleday
Allow for deterministic / manual sharding of rows.

Right now it seems that there is no way to force rows with different row keys 
will be stored on the same nodes in the ring.
This is our number one reason why we get data inconsistencies when nodes fail.

Sometimes a logical transaction requires writing rows with different row keys. 
If we could use something like this:

prefix.uniquekey and let the partitioner use only the prefix the probability 
that only part of the transaction would be written could be reduced 
considerably.



On Nov 1, 2011, at 11:59 PM, Jonathan Ellis wrote:

 Hi all,
 
 Two years ago I asked for Cassandra use cases and feature requests.
 [1]  The results [2] have been extremely useful in setting and
 prioritizing goals for Cassandra development.  But with the release of
 1.0 we've accomplished basically everything from our original wish
 list. [3]
 
 I'd love to hear from modern Cassandra users again, especially if
 you're usually a quiet lurker.  What does Cassandra do well?  What are
 your pain points?  What's your feature wish list?
 
 As before, if you're in stealth mode or don't want to say anything in
 public, feel free to reply to me privately and I will keep it off the
 record.
 
 [1] 
 http://www.mail-archive.com/cassandra-dev@incubator.apache.org/msg01148.html
 [2] 
 http://www.mail-archive.com/cassandra-user@incubator.apache.org/msg01446.html
 [3] http://www.mail-archive.com/dev@cassandra.apache.org/msg01524.html
 
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of DataStax, the source for professional Cassandra support
 http://www.datastax.com



Re: Second Cassandra users survey

2011-11-07 Thread Daniel Doubleday
Well - given the example in our case the prefix that determines the endpoints 
where a token should be routed to could be something like a user-id

so with 

key = userid + . + userthingid;

instead of

// this is happening right now
getEndpoints(hash(key))

you would have

getEndpoints(userid)

Since count(users) is much larger than number of nodes in the ring we would 
still have a balanced cluster.

I guess what we would need is something like a compound row key

You could almost do something like this with the current code base but I 
remember that there are certain assumptions about how keys translate to tokens 
on the ring make this impossible. 

But in essence this would result in another partitioner implementation. 
So you'd have OrderPreserverPartitioner, RandomPartitioner and maybe 
ShardedPartitioner


On Nov 7, 2011, at 2:26 PM, Peter Lin wrote:

 This feature interests me, so I thought I'd add some comments.
 
 Having used partition features in existing databases like DB2, Oracle
 and manual partitioning, one of the biggest challenges is keeping the
 partitions balanced. What I've seen with manual partitioning is that
 often the partitions get unbalanced. Usually the developers take a
 best guess and hope it ends up balanced.
 
 Some of the approaches I've used in the past were zip code, area code,
 state and some kind of hash.
 
 So my question related deterministic sharding is this, what rebalance
 feature(s) would be useful or needed once the partitions get
 unbalanced?
 
 Without a decent plan for rebalancing, it often ends up being a very
 painful problem to solve in production. Back when I worked mobile
 apps, we saw issues with how OpenWave WAP servers partitioned the
 accounts. The early versions randomly assigned a phone to a server
 when it is provisioned the first time. Once the phone was associated
 to that server, it was stuck on that server. If the load on that
 server was heavier than the others, the only choice was to scale up
 the hardware.
 
 My understanding of Cassandra's current sharding is consistent and
 random. Does the new feature sit some where in-between? Are you
 thinking of a pluggable API so that you can provide your own hash
 algorithm for cassandra to use?
 
 
 
 On Mon, Nov 7, 2011 at 7:54 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Allow for deterministic / manual sharding of rows.
 
 Right now it seems that there is no way to force rows with different row 
 keys will be stored on the same nodes in the ring.
 This is our number one reason why we get data inconsistencies when nodes 
 fail.
 
 Sometimes a logical transaction requires writing rows with different row 
 keys. If we could use something like this:
 
 prefix.uniquekey and let the partitioner use only the prefix the probability 
 that only part of the transaction would be written could be reduced 
 considerably.
 
 
 
 On Nov 1, 2011, at 11:59 PM, Jonathan Ellis wrote:
 
 Hi all,
 
 Two years ago I asked for Cassandra use cases and feature requests.
 [1]  The results [2] have been extremely useful in setting and
 prioritizing goals for Cassandra development.  But with the release of
 1.0 we've accomplished basically everything from our original wish
 list. [3]
 
 I'd love to hear from modern Cassandra users again, especially if
 you're usually a quiet lurker.  What does Cassandra do well?  What are
 your pain points?  What's your feature wish list?
 
 As before, if you're in stealth mode or don't want to say anything in
 public, feel free to reply to me privately and I will keep it off the
 record.
 
 [1] 
 http://www.mail-archive.com/cassandra-dev@incubator.apache.org/msg01148.html
 [2] 
 http://www.mail-archive.com/cassandra-user@incubator.apache.org/msg01446.html
 [3] http://www.mail-archive.com/dev@cassandra.apache.org/msg01524.html
 
 --
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of DataStax, the source for professional Cassandra support
 http://www.datastax.com
 
 



Re: data model for unique users in a time period

2011-11-01 Thread Daniel Doubleday
Hm - kind of hijacking this but since we have a similar problem I might throw 
in my idea:

We need consistent, idempotent counters. On the client side we can create 
unique (replayable) keys - like your user ids.

What we want to do is:

- add increment commands as columns such as [prefixByte.uniqueKey - +1] 
- use a custom compactor that sums up the commands and writes a single column 
[prefixByte.sstid - +6647] (make sure that keys dont clash)
- to read do a range query with the prefixByte

So you can have multiple counters in one row but max one column per counter per 
sst.

With leveled compaction this should work pretty nicely.

If you need fast access and want to use the row cache you will need to do some 
further patching though.

This is early brainstorming phase so any comments would be welcome

Cheers,

Daniel Doubleday
smeet.com

On Oct 31, 2011, at 7:08 PM, Ed Anuff wrote:

 Thanks, good point, splitting wide rows via sharding is a good
 optimization for the get_count approach.
 
 On Mon, Oct 31, 2011 at 10:58 AM, Zach Richardson
 j.zach.richard...@gmail.com wrote:
 Ed,
 
 I could be completely wrong about this working--I haven't specifically
 looked at how the counts are executed, but I think this makes sense.
 
 You could potentially shard across several rows, based on a hash of
 the username combined with the time period as the row key.  Run a
 count across each row and then add them up.  If your cluster is large
 enough this could spread the computation enough to make each query for
 the count a bit faster.
 
 Depending on how often this query would be hit, I would still
 recommend caching, but you could calculate reality a little more
 often.
 
 Zach
 
 
 On Mon, Oct 31, 2011 at 12:22 PM, Ed Anuff e...@anuff.com wrote:
 I'm looking at the scenario of how to keep track of the number of
 unique visitors within a given time period.  Inserting user ids into a
 wide row would allow me to have a list of every user within the time
 period that the row represented.  My experience in the past was that
 using get_count on a row to get the column count got slow pretty quick
 but that might still be the easiest way to get the count of unique
 users with some sort of caching of the count so that it's not
 expensive subsequently.  Using Hadoop is overkill for this scenario.
 Any other approaches?
 
 Ed
 
 



Re: Can not repair

2011-07-21 Thread Daniel Doubleday
Sounds like this one:

http://comments.gmane.org/gmane.comp.db.cassandra.user/15828

or

http://comments.gmane.org/gmane.comp.db.cassandra.user/15936

Hope you have a backup. That would make your life much easier ...


On Jul 21, 2011, at 4:54 PM, cbert...@libero.it wrote:

 Hi all,
 I can't get the repair in my production.
 We are out since 6 months but before we did not perform any delete do we 
 didn't need to run repair.
 Now we are out since 2 weeks with a new version of our software that performs 
 delete but we can not get the nodetool repair working,
 
 The first problem I see in the log is a: 
 
 ERROR 16:34:49,790 Fatal exception in thread Thread[CompactionExecutor:1,1,
 main]
 java.io.IOException: Keys must be written in ascending order.
at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend
 (SSTableWriter.java:111)
at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.
 java:128)
at org.apache.cassandra.db.CompactionManager.doCompaction
 (CompactionManager.java:451)
at org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.
 java:124)
at org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.
 java:94)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask
 (ThreadPoolExecutor.java:886)
at java.util.concurrent.ThreadPoolExecutor$Worker.run
 (ThreadPoolExecutor .java:908)
at java.lang.Thread.run(Thread.java:662)
 
 
 I've read something concerning a change of the partitioner but I've never 
 modified it. There is to say that we were online with 0.6.5 and now we are 
 with 
 0.7.5: to migrate from this version to another we followed the documentation 
 (conversion of the yaml, nodetool drain and so on) ... 
 
 The system is working even if there is this problem but not the repair. The 
 limit of FD of the user running the repair is unlimited but everytime we get 
 a 
 Too many open files.
 
 I'm little bit worried cause if some delete reappears the webapp will export 
 wrong data ...
 
 Best regards
 
 Carlo
 
 



Re: JNA to avoid swap but physical memory increase

2011-07-18 Thread Daniel Doubleday
http://permalink.gmane.org/gmane.comp.db.cassandra.user/14225

but given

https://issues.apache.org/jira/browse/CASSANDRA-2868

and me thinking 2 secs longer I guess it was the leaked native memory from gc 
inspector that has been swapped out.

(I didn't believe that mlockall is broken but at that time no one could explain 
why the process was swapping anyway)


On Jul 15, 2011, at 6:04 PM, Peter Schuller wrote:

 Also our experience shows that the jna call does not prevent swapping so the
 general advice is disable swap.
 
 That sounds extremely non-likely as it would imply the kernel fails to
 honor a successful mlockall(), unless other processes on the same
 machine are being swapped out. Did the process really have permission
 to do the mlockall()?
 
 (Not that I disagree in any way that swap should be disabled, +1 on that.)
 
 -- 
 / Peter Schuller (@scode on twitter)



Re: JNA to avoid swap but physical memory increase

2011-07-15 Thread Daniel Doubleday
When using jna the mlockall call will result in all pages locked in rss and 
thus reported there so you have either configured -Xms650M or you are running 
on a small box and the start script calculated it for you.

Also our experience shows that the jna call does not prevent swapping so the 
general advice is disable swap.

If you had data in you server and did read requests some (partially) loaded 
sstables would show up in rss too.

Also there is https://issues.apache.org/jira/browse/CASSANDRA-2868


On Jul 15, 2011, at 10:46 AM, Donna Li wrote:

 All:
 I download JNA jar and put it to cassandra lib directory. When restart 
 cassandra server, I found the physical memory highly increase. There is no 
 data saved in cassandra, why so much memory used by cassandra? How can I 
 decrease the memory usage by cassandra? My version is 0.7.6-2.
  
  
 Before
 PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND 
 11381 root  25   0  653m  57m 6272 S0  5.7   0:02.33 java
  
 After
 PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND 
 11130 root  25   0  653m 647m 9.9m S0 64.2   0:03.57 java  
  
  
 Best Regards
 Donna li



Re: Row cache

2011-07-04 Thread Daniel Doubleday
Just to make sure:

The yaml doesn't matter. The cache config is stored in the system tables. Its 
the CREATE ... WITH ... stuff you did via cassandra-cli to create the CF.

In Jconsole you see that the cache capacity is  0?

On Jul 4, 2011, at 11:18 AM, Shay Assulin wrote:

 Hi,
 
 The row cache capacity  0.
 
 
 after reading a row - the Caches.KeySpace.CFKeyCache.Requests attribute
 gets incremented but the ColumnFamilies.KeySpace.CF.ReadCount attribute
 remains zero and the Caches.KeySpace.CFRowCache.Size and Requsts
 attributes remain zero as well.
 
 It looks like the row-cache is disabled although the capacity is not zero.
 In addition the ColumnFamilies.KeySpace.CF.ReadCount does not reflect
 the fact that the row was fetched from SSTable.
 
 beside CF.rows_cached parameter (in the yaml) - should i configure anything
 else to enable the row-cache? 
 
 
 10x
 
 --
 View this message in context: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Row-cache-tp6532887p6545416.html
 Sent from the cassandra-u...@incubator.apache.org mailing list archive at 
 Nabble.com.



Re: Cassandra memory problem

2011-07-04 Thread Daniel Doubleday
Just to make sure: 
You were seeing that res mem was more than twice of max java heap and that did 
change after you tweaked GC settings?

Note that I am not having a heap / gc problem. The VM itself thinks everything 
is golden.

On Jul 4, 2011, at 3:41 PM, Sebastien Coutu wrote:

 We had an issue like that a short while ago here. This was mainly happening 
 under heavy load and we managed to stabilize it by tweaking the Young/Old 
 space ratio of the JVM and by also tweaking the tenuring thresholds/survivor 
 ratios. What kind of load to you have on your systems? Mostly reads, writes?
 
 SC
 
 On Mon, Jul 4, 2011 at 6:52 AM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Hi all,
 
 we have a mem problem with cassandra. res goes up without bounds (well until 
 the os kills the process because we dont have swap)
 
 I found a thread that's about the same problem but on OpenJDK: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Very-high-memory-utilization-not-caused-by-mmap-on-sstables-td5840777.html
 
 We are on Debian with Sun JDK.
 
 Resident mem is 7.4G while heap is restricted to 3G.
 
 Anyone else is seeing this with Sun JDK?
 
 Cheers,
 Daniel
 
 :/home/dd# java -version
 java version 1.6.0_24
 Java(TM) SE Runtime Environment (build 1.6.0_24-b07)
 Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02, mixed mode)
 
 :/home/dd# ps aux |grep java
 cass 28201  9.5 46.8 372659544 7707172 ?   SLl  May24 5656:21 
 /usr/bin/java -ea -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 
 -Xms3000M -Xmx3000M -Xmn400M ...
 
   PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND  
   
  
 28201 cass  20   0  355g 7.4g 1.4g S8 46.9   5656:25 java
 
 
 
 



Re: Cassandra memory problem

2011-07-04 Thread Daniel Doubleday
Yes thank you. 

I have read about the OpenJDK issue but unfortunately we are already on Sun JDK.

On Jul 4, 2011, at 6:04 PM, Sebastien Coutu wrote:

 It was among one of the issues we had. One of our hosts was using OpenJDK and 
 we've switched it to Sun and this part of the issue stabilized. The other 
 issues we had were Heap going through the roof and then OOM under load.
 
 
 On Mon, Jul 4, 2011 at 11:01 AM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Just to make sure: 
 You were seeing that res mem was more than twice of max java heap and that 
 did change after you tweaked GC settings?
 
 Note that I am not having a heap / gc problem. The VM itself thinks 
 everything is golden.
 
 On Jul 4, 2011, at 3:41 PM, Sebastien Coutu wrote:
 
 We had an issue like that a short while ago here. This was mainly happening 
 under heavy load and we managed to stabilize it by tweaking the Young/Old 
 space ratio of the JVM and by also tweaking the tenuring thresholds/survivor 
 ratios. What kind of load to you have on your systems? Mostly reads, writes?
 
 SC
 
 On Mon, Jul 4, 2011 at 6:52 AM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Hi all,
 
 we have a mem problem with cassandra. res goes up without bounds (well until 
 the os kills the process because we dont have swap)
 
 I found a thread that's about the same problem but on OpenJDK: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Very-high-memory-utilization-not-caused-by-mmap-on-sstables-td5840777.html
 
 We are on Debian with Sun JDK.
 
 Resident mem is 7.4G while heap is restricted to 3G.
 
 Anyone else is seeing this with Sun JDK?
 
 Cheers,
 Daniel
 
 :/home/dd# java -version
 java version 1.6.0_24
 Java(TM) SE Runtime Environment (build 1.6.0_24-b07)
 Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02, mixed mode)
 
 :/home/dd# ps aux |grep java
 cass 28201  9.5 46.8 372659544 7707172 ?   SLl  May24 5656:21 
 /usr/bin/java -ea -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 
 -Xms3000M -Xmx3000M -Xmn400M ...
 
   PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND 
  

 28201 cass  20   0  355g 7.4g 1.4g S8 46.9   5656:25 java
 
 
 
 
 
 



Re: Cassandra memory problem

2011-07-04 Thread Daniel Doubleday
Hi Sebastian,

one question: do you use jna.jar and do you see JNA mlockall successful in your 
logs.
There's that wild theory here that our problem might be related to mlockall and 
no swap. 
Maybe the JVM does some realloc stuff and the pinned pages are not cleared ... 

but that's really only wild guessing.

Also you are saying that on your servers res mem is not  max heap and the java 
process is not swapping?

Thanks,
Daniel

On Jul 4, 2011, at 6:04 PM, Sebastien Coutu wrote:

 It was among one of the issues we had. One of our hosts was using OpenJDK and 
 we've switched it to Sun and this part of the issue stabilized. The other 
 issues we had were Heap going through the roof and then OOM under load.
 
 
 On Mon, Jul 4, 2011 at 11:01 AM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Just to make sure: 
 You were seeing that res mem was more than twice of max java heap and that 
 did change after you tweaked GC settings?
 
 Note that I am not having a heap / gc problem. The VM itself thinks 
 everything is golden.
 
 On Jul 4, 2011, at 3:41 PM, Sebastien Coutu wrote:
 
 We had an issue like that a short while ago here. This was mainly happening 
 under heavy load and we managed to stabilize it by tweaking the Young/Old 
 space ratio of the JVM and by also tweaking the tenuring thresholds/survivor 
 ratios. What kind of load to you have on your systems? Mostly reads, writes?
 
 SC
 
 On Mon, Jul 4, 2011 at 6:52 AM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Hi all,
 
 we have a mem problem with cassandra. res goes up without bounds (well until 
 the os kills the process because we dont have swap)
 
 I found a thread that's about the same problem but on OpenJDK: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Very-high-memory-utilization-not-caused-by-mmap-on-sstables-td5840777.html
 
 We are on Debian with Sun JDK.
 
 Resident mem is 7.4G while heap is restricted to 3G.
 
 Anyone else is seeing this with Sun JDK?
 
 Cheers,
 Daniel
 
 :/home/dd# java -version
 java version 1.6.0_24
 Java(TM) SE Runtime Environment (build 1.6.0_24-b07)
 Java HotSpot(TM) 64-Bit Server VM (build 19.1-b02, mixed mode)
 
 :/home/dd# ps aux |grep java
 cass 28201  9.5 46.8 372659544 7707172 ?   SLl  May24 5656:21 
 /usr/bin/java -ea -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 
 -Xms3000M -Xmx3000M -Xmn400M ...
 
   PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND 
  

 28201 cass  20   0  355g 7.4g 1.4g S8 46.9   5656:25 java
 
 
 
 
 
 



Re: Row cache

2011-06-30 Thread Daniel Doubleday
Here's my understanding of things ... (this applies only for the regular heap 
implementation of row cache)

 Why Cassandra does not cache a row that was requested few times? 

What does the cache capacity read. Is it  0?

 What the ReadCount attribute in ColumnFamilies indicates and why it remains 
 zero. 

Hm I had that too one time (read count wont go up while there were reads). But 
I didn't have the time to debug.

 How can I know from where Cassandra read a row (from MEMTable,RowCache or 
 SSTable)? 

It will always read from 
row cache or 
memtable(s) and sstable(s)

jmx should tell you (hits go up)

 does the following correct? In read operation Cassandra looks for the row in 
 the MEMTable - if not found it looks in the row-cache - if not found it looks 
 in SSTable (after looking in the key-cache to optimize the access to the 
 SSTable)? 

No. 

If row cache capacity is  0 then a read will check if the row is in cache if 
not it read the entire row and cache it. Then / or if row was in cache already 
it will read from there and apply the respective filter to the cached CF.   
Writes update memtable and row cache when the row is cached. I must admit that 
I still dont quite understand why there's no race here. I haven't found any 
cache lock. So someone else should explain why a concurrent read / write cannot 
produce a lost update in the cached row.

If capacity is 0 then it will read from the current memtable, the memtable(s) 
that are being flushed and all sstables that may contain the row (filtered by 
bloom filter)

Hope that's correct and helps.

Cheers,
Daniel



Alternative Row Cache Implementation

2011-06-30 Thread Daniel Doubleday
Hi all - or rather devs

we have been working on an alternative implementation to the existing row 
cache(s)

We have 2 main goals:

- Decrease memory - get more rows in the cache without suffering a huge 
performance penalty
- Reduce gc pressure

This sounds a lot like we should be using the new serializing cache in 0.8. 
Unfortunately our workload consists of loads of updates which would invalidate 
the cache all the time.

The second unfortunate thing is that the idea we came up with doesn't fit the 
new cache provider api...

It looks like this:

Like the serializing cache we basically only cache the serialized byte buffer. 
we don't serialize the bloom filter and try to do some other minor compression 
tricks (var ints etc not done yet). The main difference is that we don't 
deserialize but use the normal sstable iterators and filters as in the regular 
uncached case.

So the read path looks like this:

return filter.collectCollatedColumns(memtable iter, cached row iter)

The write path is not affected. It does not update the cache

During flush we merge all memtable updates with the cached rows.

These are early test results:

- Depending on row width and value size the serialized cache takes between 30% 
- 50% of memory compared with cached CF. This might be optimized further
- Read times increase by 5 - 10%

We haven't tested the effects on gc but hope that we will see improvements 
there because we only cache a fraction of objects (in terms of numbers) in old 
gen heap which should make gc cheaper. Of course there's also the option to use 
native mem like serializing cache does.

We believe that this approach is quite promising but as I said it is not 
compatible with the current cache api.

So my question is: does that sound interesting enough to open a jira or has 
that idea already been considered and rejected for some reason?

Cheers,
Daniel
 

JRockit

2011-06-01 Thread Daniel Doubleday
Hi all

now that JRockit is available for free and the claims are there that it has 
better performance and gc I wanted to know if anybody out here has done any 
testing / benchmarking yet.
Also interested in deterministic gc ... maybe its worth the 300 bucks?

Cheers,
Daniel

Re: repair question

2011-05-25 Thread Daniel Doubleday
Ok - obviously these haven't been my brightest days.

The stream request sent to the neighbors doesn't contain the CF for which the 
ranges have been determined to mismatch.
So every diff in every CF will result in getting that range from every CF of 
the neighbor.

That explains everything.

So I guess my next repair will be scheduled in 0.8.1.

But I don't understand why this did not hit others so hard that it is 
considered more critical.
We seem to use cassandra in unusual ways.

Thanks again.

Daniel

 
On May 24, 2011, at 9:05 PM, Daniel Doubleday wrote:

 Ok thanks for your help Sylvain - much appreciated 
 
 In short: I believe that most of this is me not looking clearly yesterday. 
 There are only one / two points that i don't get. 
 Maybe you could help me out there.
 
 First the ~500MB thing is BS. The closer neighbors recieved around 80G and 
 the other 2 aroung 40G.
 Sorry about that but I got your attention :-)
 
 My missing pieces are:
 
 1. Why was I running out of space. I checked again and found that I started 
 with 761G free disc space?
 
 To make it simple I will only look at one CF 'BlobStore' which is the evil 
 large one which makes up for 80%.
 
 I greped for the streaming metadata in the log and summed it up: Total 
 streaming file size was 279G.
 This comes as a real surprise but still ...
 
 2. The file access times are strange: why does the node receive data before 
 differencing has finished?
 
 On the repairing node I see first differencing for that one ended 13:02:
 
 grep streaming /var/log/cassandra/system.log
 
  INFO [AntiEntropyStage:1] 2011-05-23 13:02:52,990 AntiEntropyService.java 
 (line 491) Performing streaming repair of 2088 ranges for #TreeRequest 
 manual-repair-ab469cff-98fb-46fa-9ad4-476a77860ed8, /172.24.0.190, 
 (Smeet,ObjectRepository)
 
 
 a listing I did on that node in the data dir shows that data files arrive 
 much earlier
 
 ls -al *tmp*
 ...
 -rw-r--r-- 1 cass cass   146846246 May 23 12:14 BlobStore-tmp-f-16356-Data.db
 -rw-r--r-- 1 cass cass  701291 May 23 12:14 BlobStore-tmp-f-16357-Data.db
 -rw-r--r-- 1 cass cass 6628735 May 23 12:14 BlobStore-tmp-f-16358-Data.db
 -rw-r--r-- 1 cass cass9991 May 23 12:14 BlobStore-tmp-f-16359-Data.db
 ...
 
 The youngest file for every CF was written at 12:14 which is the time the 
 first differencing ended:
 
  INFO [AntiEntropyStage:1] 2011-05-23 12:14:36,255 AntiEntropyService.java 
 (line 491) Performing streaming repair of 71 ranges for #TreeRequest 
 manual-repair-ab469cff-98fb-46fa-9ad4-476a77860ed8, /172.24.0.230, 
 (Smeet,Rooms
 
 I thought that cassandra would stream directly from the sstables without tmp 
 files and that these are the files received from the other nodes?
 
 3. That's only loosely related but how could a repairing node ever receive 
 data that is not requested because of a merkle tree diff. 
 
 If you look at https://issues.apache.org/jira/browse/CASSANDRA-2670: Only one 
 tree request was generated but still the repairing node got all that data 
 from the other CFs.
 That's in fact one of the reasons why I thought that there might be a bug 
 that sends to much data in the first place.
 
 Thanks for reading this book
 Daniel
 
 If your interested here's the log: 
 http://dl.dropbox.com/u/5096376/system.log.gz
 
 
 
 
 
 I also lied about total size of one node. It wasn't 320 but 280. All nodes 
 
 On May 24, 2011, at 3:41 PM, Sylvain Lebresne wrote:
 
 On Tue, May 24, 2011 at 12:40 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 We are performing the repair on one node only. Other nodes receive 
 reasonable amounts of data (~500MB).  It's only the repairing node itself 
 which 'explodes'.
 
 That, for instance, is a bit weird. That the node on which the repair
 is performed get more data is expected, since it is repair with all
 it's neighbor while the neighbors themselves get repaired only
 against that given node. But when differences between two A and B are
 computed, the ranges to repair are streaming both from A to B and for
 B to A. Unless A and B are widely out of sync (like A has no data and
 B has tons of it), around the same amount of data should transit in
 both way. So with RF=3, the node on with repair was started should get
 around 4 times (up to 6 times if you have weird topology) as much data
 than any neighboring node, but that's is. While if I'm correct, you
 are reporting that the neighboring node gets ~500MB and the
 coordinator gets  700GB ?!
 Honestly I'm not sure an imprecision of the merkle tree could account
 for that behavior.
 
 Anyway, Daniel, would you be able to share the logs of the nodes (at
 least the node on which repair is started) ? I'm not sure how much
 that could help but that cannot hurt.
 
 --
 Sylvain
 
 
 I must admit that I'm a noob when it comes to aes/repair. Its just strange 
 that a cluster that is up and running with no probs is doing that. But I 
 understand that its not supposed to do what its doing. I

Re: Database grows 10X bigger after running nodetool repair

2011-05-25 Thread Daniel Doubleday
We are having problems with repair too. 

It sounds like yours are the same. From today:
http://permalink.gmane.org/gmane.comp.db.cassandra.user/16619

On May 25, 2011, at 4:52 PM, Dominic Williams wrote:

 Hi,
 
 I've got a strange problem, where the database on a node has inflated 10X 
 after running repair. This is not the result of receiving missed data.
 
 I didn't perform repair within my usual 10 day cycle, so followed recommended 
 practice:
 http://wiki.apache.org/cassandra/Operations#Dealing_with_the_consequences_of_nodetool_repair_not_running_within_GCGraceSeconds
 
 The sequence of events was like this:
 
 1) set GCGraceSeconds to some huge value
 2) perform rolling upgrade from 0.7.4 to 0.7.6-2
 3) run nodetool repair on the first node in cluster ~10pm. It has a ~30G 
 database
 3) 2.30am decide to leave it running all night and wake up 9am to find still 
 running
 4) late morning investigation shows that db size has increased to 370G. The 
 snapshot folder accounts for only 30G
 5) node starts to run out of disk space http://pastebin.com/Sm0B7nfR
 6) decide to bail! Reset GCGraceSeconds to 864000 and restart node to stop 
 repair
 7) as node restarts it deletes a bunch of tmp files, reducing db size from 
 370G to 270G
 8) node now constantly performing minor compactions and du rising slightly 
 then falling by a greater amount after minor compaction deletes sstable
 9) gradually disk usage is coming down. Currently at 254G (3pm)
 10) performance of node obviously not great!
 
 Investigation of the database reveals the main problem to have occurred in a 
 single column family, UserFights. This contains millions of fight records 
 from our MMO, but actually exactly the same number as the MonsterFights cf. 
 However, the comparative size is
 
   Column Family: MonsterFights
   SSTable count: 38
   Space used (live): 13867454647
   Space used (total): 13867454647 (13G)
   Memtable Columns Count: 516
   Memtable Data Size: 598770
   Memtable Switch Count: 4
   Read Count: 514
   Read Latency: 157.649 ms.
   Write Count: 4059
   Write Latency: 0.025 ms.
   Pending Tasks: 0
   Key cache capacity: 20
   Key cache size: 183004
   Key cache hit rate: 0.0023566218452145135
   Row cache: disabled
   Compacted row minimum size: 771
   Compacted row maximum size: 943127
   Compacted row mean size: 3208
 
   Column Family: UserFights
   SSTable count: 549
   Space used (live): 185355019679
   Space used (total): 219489031691 (219G)
   Memtable Columns Count: 483
   Memtable Data Size: 560569
   Memtable Switch Count: 8
   Read Count: 2159
   Read Latency: 2589.150 ms.
   Write Count: 4080
   Write Latency: 0.018 ms.
   Pending Tasks: 0
   Key cache capacity: 20
   Key cache size: 20
   Key cache hit rate: 0.03357770764288416
   Row cache: disabled
   Compacted row minimum size: 925
   Compacted row maximum size: 12108970
   Compacted row mean size: 503069
 
 These stats were taken at 3pm, and at 1pm UserFights was using 224G total, so 
 overall size is gradually coming down. 
 
 Another observation is the following appearing in the logs during the minor 
 compactions:
 Compacting large row 536c69636b5061756c (121235810 bytes) incrementally
 
 The largest number of fights any user has performed on our MMO that I can 
 find is short of 10,000. Each fight record is smaller than 1K... so it looks 
 like these rows have grown +10X somehow.
 
 The size of UserFights on another replica node, which actually has a slightly 
 higher proportion of ring is
 
   Column Family: UserFights
   SSTable count: 14
   Space used (live): 17844982744
   Space used (total): 17936528583 (18G)
   Memtable Columns Count: 767
   Memtable Data Size: 891153
   Memtable Switch Count: 6
   Read Count: 2298
   Read Latency: 61.020 ms.
   Write Count: 4261
   Write Latency: 0.104 ms.
   Pending Tasks: 0
   Key cache capacity: 20
   Key cache size: 55172
   Key cache hit rate: 0.8079570484581498
   Row cache: disabled
   Compacted row minimum size: 925
   Compacted row maximum size: 12108970
   Compacted row mean size: 846477
 ...
 
 All ideas and suggestions greatly appreciated as always!
 
 Dominic
 ria101.wordpress.com



Re: Database grows 10X bigger after running nodetool repair

2011-05-25 Thread Daniel Doubleday




Firstly any ideas for a quick fix because this is giving me big 
production problems. Write/read with QUORUM is reportedly producing 
unpredictable results (people have called support regarding monsters 
in my MMO appearing and disappearing magically) and many operations 
are just failing with SocketTimeoutException I guess because of the 
continuing compactions over huge sstables. I'm going to have to try 
making adjustments to client timeout settings etc but this feels like 
using a hanky to protect oneself from a downpour.



Monsters appear magically? That doesn't sound so bad ...

Sorry ... well this is actually strange. You can get 'inconsistent' 
results with quorum because of timed out writes that are written on one 
server only but not on the other two (given a rf=3). Some reads will 
return with the old value until one read will eventually get the one 
newer 'failed' write. If you're really out of luck read repair can then 
fail and you will get the old value again. But this is eventually going 
to be ok.


Secondly does anyone know if this is just a waiting game - will the 
node eventually correct itself and shrink back down?

I'm down to 204G now from 270G.

One thing you have to make sure is that the repair actually stopped. If 
you killed the repairing node the other nodes will continue to retry to 
send data. I think 7 or 8 times with increasing pauses. Check the log 
files on the other nodes. If your sure that you don't get anymore data I 
would recommend a major (forced) compaction. But this will take a couple 
of hours depending on you drives.


If your using dynamic snitch the one slow node should not kill you. If 
it does and you can take it out of the ring. I think there was some 
nodetool way but I can't remember. Easiest way is to configure it to 
listen only on localhost and restart.
Thirdly does anyone know if the problem is contagious i.e. should I 
consider decommissioning the whole node and try to rebuild from replicas?



No. That should not be necessary

Good luck

Thanks, Dominic

On 25 May 2011 17:16, Daniel Doubleday daniel.double...@gmx.net 
mailto:daniel.double...@gmx.net wrote:


We are having problems with repair too.

It sounds like yours are the same. From today:
http://permalink.gmane.org/gmane.comp.db.cassandra.user/16619

On May 25, 2011, at 4:52 PM, Dominic Williams wrote:


Hi,

I've got a strange problem, where the database on a node has
inflated 10X after running repair. This is not the result of
receiving missed data.

I didn't perform repair within my usual 10 day cycle, so followed
recommended practice:

http://wiki.apache.org/cassandra/Operations#Dealing_with_the_consequences_of_nodetool_repair_not_running_within_GCGraceSeconds

The sequence of events was like this:

1) set GCGraceSeconds to some huge value
2) perform rolling upgrade from 0.7.4 to 0.7.6-2
3) run nodetool repair on the first node in cluster ~10pm. It has
a ~30G database
3) 2.30am decide to leave it running all night and wake up 9am to
find still running
4) late morning investigation shows that db size has increased to
370G. The snapshot folder accounts for only 30G
5) node starts to run out of disk space http://pastebin.com/Sm0B7nfR
6) decide to bail! Reset GCGraceSeconds to 864000 and restart
node to stop repair
7) as node restarts it deletes a bunch of tmp files, reducing db
size from 370G to 270G
8) node now constantly performing minor compactions and du rising
slightly then falling by a greater amount after minor compaction
deletes sstable
9) gradually disk usage is coming down. Currently at 254G (3pm)
10) performance of node obviously not great!

Investigation of the database reveals the main problem to have
occurred in a single column family, UserFights. This contains
millions of fight records from our MMO, but actually exactly the
same number as the MonsterFights cf. However, the comparative size is

Column Family: MonsterFights
SSTable count: 38
Space used (live): 13867454647
Space used (total): 13867454647 (13G)
Memtable Columns Count: 516
Memtable Data Size: 598770
Memtable Switch Count: 4
Read Count: 514
Read Latency: 157.649 ms.
Write Count: 4059
Write Latency: 0.025 ms.
Pending Tasks: 0
Key cache capacity: 20
Key cache size: 183004
Key cache hit rate: 0.0023566218452145135
Row cache: disabled
Compacted row minimum size: 771
Compacted row maximum size: 943127
Compacted row mean size: 3208

Column Family: UserFights
SSTable count: 549
Space used (live): 185355019679
Space used (total): 219489031691 (219G)
Memtable Columns Count: 483
Memtable Data Size: 560569
Memtable Switch Count: 8
Read Count: 2159
Read Latency: 2589.150 ms.
Write Count: 4080
Write Latency: 0.018 ms.
Pending Tasks: 0
Key cache capacity: 20

Re: repair question

2011-05-24 Thread Daniel Doubleday
Ok thanks for your help Sylvain - much appreciated 

In short: I believe that most of this is me not looking clearly yesterday. 
There are only one / two points that i don't get. 
Maybe you could help me out there.

First the ~500MB thing is BS. The closer neighbors recieved around 80G and the 
other 2 aroung 40G.
Sorry about that but I got your attention :-)

My missing pieces are:

1. Why was I running out of space. I checked again and found that I started 
with 761G free disc space?

To make it simple I will only look at one CF 'BlobStore' which is the evil 
large one which makes up for 80%.

I greped for the streaming metadata in the log and summed it up: Total 
streaming file size was 279G.
This comes as a real surprise but still ...

2. The file access times are strange: why does the node receive data before 
differencing has finished?

On the repairing node I see first differencing for that one ended 13:02:

grep streaming /var/log/cassandra/system.log

 INFO [AntiEntropyStage:1] 2011-05-23 13:02:52,990 AntiEntropyService.java 
(line 491) Performing streaming repair of 2088 ranges for #TreeRequest 
manual-repair-ab469cff-98fb-46fa-9ad4-476a77860ed8, /172.24.0.190, 
(Smeet,ObjectRepository)


a listing I did on that node in the data dir shows that data files arrive much 
earlier

ls -al *tmp*
...
-rw-r--r-- 1 cass cass   146846246 May 23 12:14 BlobStore-tmp-f-16356-Data.db
-rw-r--r-- 1 cass cass  701291 May 23 12:14 BlobStore-tmp-f-16357-Data.db
-rw-r--r-- 1 cass cass 6628735 May 23 12:14 BlobStore-tmp-f-16358-Data.db
-rw-r--r-- 1 cass cass9991 May 23 12:14 BlobStore-tmp-f-16359-Data.db
...

The youngest file for every CF was written at 12:14 which is the time the first 
differencing ended:

 INFO [AntiEntropyStage:1] 2011-05-23 12:14:36,255 AntiEntropyService.java 
(line 491) Performing streaming repair of 71 ranges for #TreeRequest 
manual-repair-ab469cff-98fb-46fa-9ad4-476a77860ed8, /172.24.0.230, (Smeet,Rooms

I thought that cassandra would stream directly from the sstables without tmp 
files and that these are the files received from the other nodes?

3. That's only loosely related but how could a repairing node ever receive data 
that is not requested because of a merkle tree diff. 

If you look at https://issues.apache.org/jira/browse/CASSANDRA-2670: Only one 
tree request was generated but still the repairing node got all that data from 
the other CFs.
That's in fact one of the reasons why I thought that there might be a bug that 
sends to much data in the first place.

Thanks for reading this book
Daniel

If your interested here's the log: http://dl.dropbox.com/u/5096376/system.log.gz





I also lied about total size of one node. It wasn't 320 but 280. All nodes 

On May 24, 2011, at 3:41 PM, Sylvain Lebresne wrote:

 On Tue, May 24, 2011 at 12:40 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 We are performing the repair on one node only. Other nodes receive 
 reasonable amounts of data (~500MB).  It's only the repairing node itself 
 which 'explodes'.
 
 That, for instance, is a bit weird. That the node on which the repair
 is performed get more data is expected, since it is repair with all
 it's neighbor while the neighbors themselves get repaired only
 against that given node. But when differences between two A and B are
 computed, the ranges to repair are streaming both from A to B and for
 B to A. Unless A and B are widely out of sync (like A has no data and
 B has tons of it), around the same amount of data should transit in
 both way. So with RF=3, the node on with repair was started should get
 around 4 times (up to 6 times if you have weird topology) as much data
 than any neighboring node, but that's is. While if I'm correct, you
 are reporting that the neighboring node gets ~500MB and the
 coordinator gets  700GB ?!
 Honestly I'm not sure an imprecision of the merkle tree could account
 for that behavior.
 
 Anyway, Daniel, would you be able to share the logs of the nodes (at
 least the node on which repair is started) ? I'm not sure how much
 that could help but that cannot hurt.
 
 --
 Sylvain
 
 
 I must admit that I'm a noob when it comes to aes/repair. Its just strange 
 that a cluster that is up and running with no probs is doing that. But I 
 understand that its not supposed to do what its doing. I just hope that I 
 find out why soon enough.
 
 
 
 On 23.05.2011, at 21:21, Peter Schuller peter.schul...@infidyne.com wrote:
 
 I'm a bit lost: I tried a repair yesterday with only one CF and that 
 didn't really work the way I expected but I thought that would be a bug 
 which only affects that special case.
 
 So I tried again for all CFs.
 
 I started with a nicely compacted machine with around 320GB of load. Total 
 disc space on this node was 1.1TB.
 
 Did you do repairs simultaneously on all nodes?
 
 I have seen very significant disk space increases under some
 circumstances. While I haven't filed a ticket about it because

repair question

2011-05-23 Thread Daniel Doubleday
Hi all

I'm a bit lost: I tried a repair yesterday with only one CF and that didn't 
really work the way I expected but I thought that would be a bug which only 
affects that special case.

So I tried again for all CFs.

I started with a nicely compacted machine with around 320GB of load. Total disc 
space on this node was 1.1TB.

After it went out of disc space (meaning I received around 700GB of data) I had 
a very brief look at the repair code again and it seems to me that the 
repairing node will get all data for its range from all its neighbors.

Is that true and if so is it the intended behavior? If so one would rather need 
5-6 times of disc space given that compactions that need to run after the 
sstable rebuild also need temp disc space.

Cheers,
Daniel

Re: repair question

2011-05-23 Thread Daniel Doubleday
Thanks Sylvain

well no I don't really understand it at all. We have all 

Wide rows / small val to single larger column in one row.

The problem hits every CF. RF = 3 Read / Write with Quorum. 

The CF that is killing me right now is one col thats never updated (its WORM - 
updates are reinserts under a new key and a delete of the old one - to avoid 
updates of large CF). 250GB per node.
Unfortunately restarting the node doesn't stop repair so the repair started 
again. I deleted all tmp files before restarting but its out of space again. du 
-hcs shows 780GB for that CF now.

Guess I have to restart all nodes to stop repair?

To answer the question: yes the cluster might be a little out of synch but not 
that much. 

What I dont understand: I saw that the repairing node was still doing a 
validation compaction on that major sstable file (200GB) but it already 
received loads of data for that CF from the other nodes.

Sigh...


On May 23, 2011, at 7:48 PM, Sylvain Lebresne wrote:

 On Mon, May 23, 2011 at 7:17 PM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Hi all
 
 I'm a bit lost: I tried a repair yesterday with only one CF and that didn't 
 really work the way I expected but I thought that would be a bug which only 
 affects that special case.
 
 So I tried again for all CFs.
 
 I started with a nicely compacted machine with around 320GB of load. Total 
 disc space on this node was 1.1TB.
 
 After it went out of disc space (meaning I received around 700GB of data) I 
 had a very brief look at the repair code again and it seems to me that the 
 repairing node will get all data for its range from all its neighbors.
 
 The repaired node is supposed to get only data from it's
 neighbors for rows it is not in sync with. That is all supposed
 to depend on how much the node is out of sync compared to
 the other nodes.
 
 Now there is a number of things that could make it repair more
 that what you would hope. For instance:
  1) even if one column is different for a row, the full row is
  repaired. If you have a small number of huge rows, that
  can amount for quite some data useless transfered.
  2) The other one is that the merkle tree (that allows to say
  whether 2 rows are in sync) doesn't necessarily have one
  hash by row, so in theory one column not in sync may imply
  the repair of more than one row.
  3) https://issues.apache.org/jira/browse/CASSANDRA-2324 (which
  is fixed in 0.8)
 
 Fortunately, the chance to get hit by 1) is proportionally inverse
 to the change of getting hit by 2) and vice versa.
 
 Anyway, the kind of excess data your seeing is not something
 I would expect unless the node is really completely out of sync
 with all the other nodes.
 So in the light of this, do you have more info on your own case ?
 (do you lots of small row, few of large ones ? Did you expected
 the node to be widely out of sync with the other nodes ? Etc..)
 
 
 --
 Sylvain
 
 
 Is that true and if so is it the intended behavior? If so one would rather 
 need 5-6 times of disc space given that compactions that need to run after 
 the sstable rebuild also need temp disc space.
 
 Cheers,
 Daniel



Re: repair question

2011-05-23 Thread Daniel Doubleday
We are performing the repair on one node only. Other nodes receive reasonable 
amounts of data (~500MB).  It's only the repairing node itself which 
'explodes'. 

I must admit that I'm a noob when it comes to aes/repair. Its just strange that 
a cluster that is up and running with no probs is doing that. But I understand 
that its not supposed to do what its doing. I just hope that I find out why 
soon enough. 



On 23.05.2011, at 21:21, Peter Schuller peter.schul...@infidyne.com wrote:

 I'm a bit lost: I tried a repair yesterday with only one CF and that didn't 
 really work the way I expected but I thought that would be a bug which only 
 affects that special case.
 
 So I tried again for all CFs.
 
 I started with a nicely compacted machine with around 320GB of load. Total 
 disc space on this node was 1.1TB.
 
 Did you do repairs simultaneously on all nodes?
 
 I have seen very significant disk space increases under some
 circumstances. While I haven't filed a ticket about it because there
 was never time to confirm, I believe two things were at play:
 
 (1) nodes were sufficiently out a sync in a sufficiently spread out
 fashion that the granularity of the merkle tree (IIRC, and if I read
 correctly, it divides the ring into up to 2^15 segments but no more)
 became ineffective so that repair effectively had to transfer all the
 data. at first I thought there was an outright bug, but after looking
 at the code I suspected it was just the merkle tree granularity.
 
 (2) I suspected at the time that a contributing factor was also that
 as one repair might cause a node to significantly increase it's live
 sstables temporarily until they are compacted, another repair on
 another node may start and start validating compaction and streaming
 of that data - leading to disk space bload essentially being
 contagious; the third node streaming from the node that was
 temporarily bloated, will receive even more data from that node than
 it normally would.
 
 We're making sure to only run one repair at a time between any hosts
 that are neighbors of each other (meaning that at RF=3, that's 1
 concurrent repair per 6 nodes in the cluster).
 
 I'd be interested in hearing anyone confirm or deny whether my
 understanding of (1) in particular is correct. To connect it to
 reality: a 20 GB CF divided into 2^15 segments implies each segment is
 600 kbyte in size. For CF:s with tens or hundreds of millions of
 small rows and a fairly random (with respect to partitioner) update
 pattern, it's not very difficult to end up in a situation where most
 600 kbyte chunks contain out-of-synch data. Particularly in a
 situation with lots of dropped messages.
 
 I'm getting the 2^15 from AntiEntropyService.Validator.Validator()
 which passes a maxsize of 2^15 to the MerkelTree constructor.
 
 -- 
 / Peter Schuller


Documentation of Known Issues

2011-05-20 Thread Daniel Doubleday
Hi all

I was wondering if there might be some way to better communicate known issues. 

We do try to track jira issues but at times some slip through or we miss 
implications.

Things like the broken repair of specific CFs. 
(https://issues.apache.org/jira/browse/CASSANDRA-2670). I know that this 
potentially dups jira but maybe tasks could get tagged and some magic filter 
could show a list. Or something like a simple wiki page that lists the known 
issues that might not be critical such as the mentioned bug but still are a 
pain if it happens to you.

Cheers,
Daniel



Berlin Buzzword Hackathon

2011-05-18 Thread Daniel Doubleday
Hi all

was wondering if there's anybody here planning to go to the Berlin Buzzwords 
and attend the cassandra hackathon.

I'm still indecisive but it might be good to have the chance to talk about 
experiences in more detail.

Cheers,
Daniel

Dynamic Snitch Problem

2011-05-17 Thread Daniel Doubleday
Hi all

after upgrading to 0.7 we have a small problem with dynamic snitch:

we have rf=3, quorum read/write and read repair prop set to 0. Thus cassandra 
always shortcuts reads to only 2 hosts.

Problem is that one of our nodes get ignored unless using a little patch and 
initialize the scores.

Anybody else sees this?

Cheers,
Daniel

btw: we had to do this on 0.6 too when using the 'shortcut' read path.

Index: src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
===
--- src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
(revision 1100429)
+++ src/java/org/apache/cassandra/locator/DynamicEndpointSnitch.java
(revision )
@@ -178,8 +178,16 @@
 {
 Double scored1 = scores.get(a1);
 Double scored2 = scores.get(a2);
-
+
-if (scored1 == null || scored2 == null || scored1.equals(scored2))
+if (scored1 == null) {
+scored1 = 0.0;
+}
+
+if (scored2 == null) {
+scored2 = 0.0;
+}
+
+if (scored1.equals(scored2))
 return subsnitch.compareEndpoints(target, a1, a2);
 if (scored1  scored2)
 return -1;







Re: Monitoring bytes read per cf

2011-05-13 Thread Daniel Doubleday
Thanks - yes I agree. Didn't want to judge solely based on this figure. 

It should just add to the picture. But since we know access patterns and other 
stats like key and row cache hit ratios we hope to be able to make a more 
educated guess whats going on.

On May 13, 2011, at 9:08 AM, Peter Schuller wrote:

 It seems to work fine but maybe someone here knows that this is not a good 
 idea 
 
 It depends. If you're trying to figure out what generates actual iops
 to disk, it won't really tell you since you're counting all I/O
 including that which is cached. Since whatever goes down to disk tends
 to be, due to the LRU:ishness of caches, the less frequently accessed
 data that tends to make it difficult to judge by numbers that include
 all I/O.
 
 -- 
 / Peter Schuller



Monitoring bytes read per cf

2011-05-12 Thread Daniel Doubleday
Hi all

got a question for folks with some code insight again.

To be able to better understand where our IO load is coming from we want to 
monitor the number of bytes read from disc per cf. (we love stats)

What I have done is wrapping the FileDataInput in SSTableReader to sum the 
bytes read in CFS. This will only record data file access but that would be 
good enough for us.

It seems to work fine but maybe someone here knows that this is not a good idea 


Cheers,
Daniel

Some code:

SSTableReader:
private static final boolean KEEP_IO_STATISICS = 
Boolean.getBoolean(cassandra.keepIOStats);
public FileDataInput getFileDataInput(DecoratedKey decoratedKey, int 
bufferSize)
{
long position = getPosition(decoratedKey, Operator.EQ);
if (position  0)
return null;

FileDataInput segment = dfile.getSegment(position, bufferSize);
return (KEEP_IO_STATISICS) ? new MonitoringFileDataIInput(metadata, 
segment) : segment; 
}

with MonitoringFileDataIInput

public class MonitoringFileDataIInput implements FileDataInput, Closeable
{

private final FileDataInput fileDataInput;
private final ColumnFamilyStore columnFamilyStore;

public MonitoringFileDataIInput(CFMetaData cfMetaData, FileDataInput 
fileDataInput)
{
columnFamilyStore = 
Table.open(cfMetaData.tableName).getColumnFamilyStore(cfMetaData.cfId);
this.fileDataInput = fileDataInput;
}

@Override
public boolean readBoolean() throws IOException
{
columnFamilyStore.addBytesRead(1);
return fileDataInput.readBoolean();
}

// ... etc

and ColumnFamilyStore
private final AtomicLong bytesRead = new AtomicLong(0L);

@Override // ColumnFamilyStoreMBean
public long getBytesRead()
{
return bytesRead.get();
}

public void addBytesRead(int num) 
{
bytesRead.addAndGet(num);
}

  

Re: Unicode key encoding problem when upgrading from 0.6.13 to 0.7.5

2011-05-05 Thread Daniel Doubleday
Thats UTF-8 not UTF-16.

On May 5, 2011, at 1:57 PM, aaron morton wrote:

 The hard core way to fix the data is export to json with sstable2json, hand 
 edit, and then json2sstable it back. 
 
 Also to confirm, this only happens when data is written in 0.6 and then tried 
 to read back in 0.7?
 
 And you what partitioner are you using ? You can still see the keys ?
 
 Can you use sstable2json agains tthe 0.6 data ?
 
 Looking at you last email something looks fishy about the encoding...
 
 My two keys that I send in my test program are 0xe695b0e69982e99693 and 
 0x666f6f, which decodes to 数時間 and foo respectively.
 
 
 There are 9 bytes encoded there I would expect a multiple of 2 for each 
 character. (using UTF-16 surrogate pairs 
 http://en.wikipedia.org/wiki/UTF-16/UCS-2 )
 
 I looked the characters up and their encoding is different here 
 æ•° 0x6570 http://www.fileformat.info/info/unicode/char/6570/index.htm
 時 0x6642 http://www.fileformat.info/info/unicode/char/6642/index.htm 
 é–“ 0x9593 http://www.fileformat.info/info/unicode/char/9593/index.htm
 
 Am I missing something ?
 
 Hope that helps. 
 -
 Aaron Morton
 Freelance Cassandra Developer
 @aaronmorton
 http://www.thelastpickle.com
 
 On 5 May 2011, at 23:09, Henrik Schröder wrote:
 
 Yes, the keys were written to 0.6, but when I looked through the thrift 
 client code for 0.6, it explicitly converts all string keys to UTF8 before 
 sending them over to the server so the encoding *should* be right, and after 
 the upgrade to 0.7.5, sstablekeys prints out the correct byte values for 
 those keys, but Cassandra itself is unable to get those rows.
 
 I ran some more tests yesterday with a clean database where I only wrote two 
 rows, one with an ascii key and one with a unicode key, upgraded to 0.7.5, 
 ran nodetool cleanup, and that actually fixed it. After cleanup, the server 
 could fetch both rows correctly.
 
 However, when I tried to do the same thing with a snapshot of our live 
 database where we have ~2 million keys, out of which ~1000 are unicode, 
 cleanup failed with a lot of Keys must be written in descending order 
 exceptions. I've tried various combinations of cleanup and scrub, running 
 cleanup before upgrading, etc, but I've yet to find something that fixes all 
 the problems without losing those rows.
 
 
 /Henrik
 
 On Thu, May 5, 2011 at 12:48, aaron morton aa...@thelastpickle.com wrote:
 I take it back, the problem started in 0.6 where keys were strings. Looking 
 into how 0.6 did it's thing
 
 
 -
 Aaron Morton
 Freelance Cassandra Developer
 @aaronmorton
 http://www.thelastpickle.com
 
 On 5 May 2011, at 22:36, aaron morton wrote:
 
 Interesting but as we are dealing with keys it should not matter as they 
 are treated as byte buffers. 
 
 -
 Aaron Morton
 Freelance Cassandra Developer
 @aaronmorton
 http://www.thelastpickle.com
 
 On 5 May 2011, at 04:53, Daniel Doubleday wrote:
 
 This is a bit of a wild guess but Windows and encoding and 0.7.5 sounds 
 like
 
 https://issues.apache.org/jira/browse/CASSANDRA-2367
 
  
 On May 3, 2011, at 5:15 PM, Henrik Schröder wrote:
 
 Hey everyone,
 
 We did some tests before upgrading our Cassandra cluster from 0.6 to 0.7, 
 just to make sure that the change in how keys are encoded wouldn't cause 
 us any dataloss. Unfortunately it seems that rows stored under a unicode 
 key couldn't be retrieved after the upgrade. We're running everything on 
 Windows, and we're using the generated thrift client in C# to access it.
 
 I managed to make a minimal test to reproduce the error consistently:
 
 First, I started up Cassandra 0.6.13 with an empty data directory, and a 
 really simple config with a single keyspace with a single bytestype 
 columnfamily.
 I wrote two rows, each with a single column with a simple column name and 
 a 1-byte value of 1. The first row had a key using only ascii chars 
 ('foo'), and the second row had a key using unicode chars ('ドメインウ').
 
 Using multi_get, and both those keys, I got both columns back, as 
 expected.
 Using multi_get_slice and both those keys, I got both columns back, as 
 expected.
 I also did a get_range_slices to get all rows in the columnfamily, and I 
 got both columns back, as expected.
 
 So far so good. Then I drain and shut down Cassandra 0.6.13, and start up 
 Cassandra 0.7.5, pointing to the same data directory, with a config 
 containing the same keyspace, and I run the schematool import command.
 
 I then start up my test program that uses the new thrift api, and run 
 some commands.
 
 Using multi_get_slice, and those two keys encoded as UTF8 byte-arrays, I 
 only get back one column, the one under the key 'foo'. The other row I 
 simply can't retrieve.
 
 However, when I use get_range_slices to get all rows, I get back two 
 rows, with the correct column values, and the byte-array keys are 
 identical to my encoded keys, and when I decode the byte-arrays as UTF8 
 drings, I get back my

Re: Unicode key encoding problem when upgrading from 0.6.13 to 0.7.5

2011-05-05 Thread Daniel Doubleday
Don't know if that helps you but since we had the same SSTable corruption I 
have been looking into that very code the other day:

If you could afford to drop these rows and are able to recognize them the 
easiest way would be patching:

SSTableScanner:162

public IColumnIterator next()
{
try
{
if (row != null)
file.seek(finishedAt);
assert !file.isEOF();

DecoratedKey key = SSTableReader.decodeKey(sstable.partitioner,
   sstable.descriptor,
   
ByteBufferUtil.readWithShortLength(file));
long dataSize = SSTableReader.readRowSize(file, 
sstable.descriptor);
long dataStart = file.getFilePointer();
finishedAt = dataStart + dataSize;

if (filter == null)
{
row = new SSTableIdentityIterator(sstable, file, key, 
dataStart, dataSize);
return row;
}
else
{
return row = filter.getSSTableColumnIterator(sstable, file, 
key);
}
}
catch (IOException e)
{
throw new RuntimeException(SSTableScanner.this +  failed to 
provide next columns from  + this, e);
}
}

The string key is new String(ByteBufferUtil.getArray(key.key), UTF-8)
If you find one that you don't like just skip it.

This way compaction goes through but obviously you'll loose data.

On May 5, 2011, at 1:12 PM, Henrik Schröder wrote:

 Yeah, I've seen that one, and I'm guessing that it's the root cause of my 
 problems, something something encoding error, but that doesn't really help 
 me. :-)
 
 However, I've done all my tests with 0.7.5, I'm gonna try them again with 
 0.7.4, just to see how that version reacts.
 
 
 /Henrik
 
 On Wed, May 4, 2011 at 18:53, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 This is a bit of a wild guess but Windows and encoding and 0.7.5 sounds like
 
 https://issues.apache.org/jira/browse/CASSANDRA-2367
 
  
 On May 3, 2011, at 5:15 PM, Henrik Schröder wrote:
 
 Hey everyone,
 
 We did some tests before upgrading our Cassandra cluster from 0.6 to 0.7, 
 just to make sure that the change in how keys are encoded wouldn't cause us 
 any dataloss. Unfortunately it seems that rows stored under a unicode key 
 couldn't be retrieved after the upgrade. We're running everything on 
 Windows, and we're using the generated thrift client in C# to access it.
 
 I managed to make a minimal test to reproduce the error consistently:
 
 First, I started up Cassandra 0.6.13 with an empty data directory, and a 
 really simple config with a single keyspace with a single bytestype 
 columnfamily.
 I wrote two rows, each with a single column with a simple column name and a 
 1-byte value of 1. The first row had a key using only ascii chars ('foo'), 
 and the second row had a key using unicode chars ('ドメインウ').
 
 Using multi_get, and both those keys, I got both columns back, as expected.
 Using multi_get_slice and both those keys, I got both columns back, as 
 expected.
 I also did a get_range_slices to get all rows in the columnfamily, and I got 
 both columns back, as expected.
 
 So far so good. Then I drain and shut down Cassandra 0.6.13, and start up 
 Cassandra 0.7.5, pointing to the same data directory, with a config 
 containing the same keyspace, and I run the schematool import command.
 
 I then start up my test program that uses the new thrift api, and run some 
 commands.
 
 Using multi_get_slice, and those two keys encoded as UTF8 byte-arrays, I 
 only get back one column, the one under the key 'foo'. The other row I 
 simply can't retrieve.
 
 However, when I use get_range_slices to get all rows, I get back two rows, 
 with the correct column values, and the byte-array keys are identical to my 
 encoded keys, and when I decode the byte-arrays as UTF8 drings, I get back 
 my two original keys. This means that both my rows are still there, the keys 
 as output by Cassandra are identical to the original string keys I used when 
 I created the rows in 0.6.13, but it's just impossible to retrieve the 
 second row.
 
 To continue the test, I inserted a row with the key 'ドメインウ' encoded as UTF-8 
 again, and gave it a similar column as the original, but with a 1-byte value 
 of 2.
 
 Now, when I use multi_get_slice with my two encoded keys, I get back two 
 rows, the 'foo' row has the old value as expected, and the other row has the 
 new value as expected.
 
 However, when I use get_range_slices to get all rows, I get back *three* 
 rows, two of which have the *exact same* byte-array key, one has the old 
 column, one has the new column. 
 
 
 How is this possible? How can there be two different rows with the exact 
 same key? I'm guessing that it's related

Re: Unicode key encoding problem when upgrading from 0.6.13 to 0.7.5

2011-05-04 Thread Daniel Doubleday
This is a bit of a wild guess but Windows and encoding and 0.7.5 sounds like

https://issues.apache.org/jira/browse/CASSANDRA-2367

 
On May 3, 2011, at 5:15 PM, Henrik Schröder wrote:

 Hey everyone,
 
 We did some tests before upgrading our Cassandra cluster from 0.6 to 0.7, 
 just to make sure that the change in how keys are encoded wouldn't cause us 
 any dataloss. Unfortunately it seems that rows stored under a unicode key 
 couldn't be retrieved after the upgrade. We're running everything on Windows, 
 and we're using the generated thrift client in C# to access it.
 
 I managed to make a minimal test to reproduce the error consistently:
 
 First, I started up Cassandra 0.6.13 with an empty data directory, and a 
 really simple config with a single keyspace with a single bytestype 
 columnfamily.
 I wrote two rows, each with a single column with a simple column name and a 
 1-byte value of 1. The first row had a key using only ascii chars ('foo'), 
 and the second row had a key using unicode chars ('ドメインウ').
 
 Using multi_get, and both those keys, I got both columns back, as expected.
 Using multi_get_slice and both those keys, I got both columns back, as 
 expected.
 I also did a get_range_slices to get all rows in the columnfamily, and I got 
 both columns back, as expected.
 
 So far so good. Then I drain and shut down Cassandra 0.6.13, and start up 
 Cassandra 0.7.5, pointing to the same data directory, with a config 
 containing the same keyspace, and I run the schematool import command.
 
 I then start up my test program that uses the new thrift api, and run some 
 commands.
 
 Using multi_get_slice, and those two keys encoded as UTF8 byte-arrays, I only 
 get back one column, the one under the key 'foo'. The other row I simply 
 can't retrieve.
 
 However, when I use get_range_slices to get all rows, I get back two rows, 
 with the correct column values, and the byte-array keys are identical to my 
 encoded keys, and when I decode the byte-arrays as UTF8 drings, I get back my 
 two original keys. This means that both my rows are still there, the keys as 
 output by Cassandra are identical to the original string keys I used when I 
 created the rows in 0.6.13, but it's just impossible to retrieve the second 
 row.
 
 To continue the test, I inserted a row with the key 'ドメインウ' encoded as UTF-8 
 again, and gave it a similar column as the original, but with a 1-byte value 
 of 2.
 
 Now, when I use multi_get_slice with my two encoded keys, I get back two 
 rows, the 'foo' row has the old value as expected, and the other row has the 
 new value as expected.
 
 However, when I use get_range_slices to get all rows, I get back *three* 
 rows, two of which have the *exact same* byte-array key, one has the old 
 column, one has the new column. 
 
 
 How is this possible? How can there be two different rows with the exact same 
 key? I'm guessing that it's related to the encoding of string keys in 0.6, 
 and that the internal representation is off somehow. I checked the generated 
 thrift client for 0.6, and it UTF8-encodes all keys before sending them to 
 the server, so it should be UTF8 all the way, but apparently it isn't.
 
 Has anyone else experienced the same problem? Is it a platform-specific 
 problem? Is there a way to avoid this and upgrade from 0.6 to 0.7 and not 
 lose any rows? I would also really like to know which byte-array I should 
 send in to get back that second row, there's gotta be some key that can be 
 used to get it, the row is still there after all.
 
 
 /Henrik Schröder



Re: Strange corrupt sstable

2011-05-02 Thread Daniel Doubleday
Just for the record:

The problem had nothing to do with bad memory. After some more digging it 
turned out that due to a bug we wrote invalid utf-8 sequences as row keys. In 
0.6 the key tokens are constructed from string decoded bytes. This does not 
happen anymore in 0.7 files. So what apparently happened during compaction was 

1. read sst and generate string based order rows
2. write the new file based on that order
3. read the compacted file based on raw bytes order - crash

That bug never made it to production so we are fine.
 
On Apr 29, 2011, at 10:32 AM, Daniel Doubleday wrote:

 Bad == Broken
 
 That means you cannot rely on 1 == 1. In such a scenario everything can 
 happen including data loss. 
 That's why you want ECC mem on production servers. Our cheapo dev boxes dont.
 
 On Apr 28, 2011, at 7:46 PM, mcasandra wrote:
 
 What do you mean by Bad memory? Is it less heap size, OOM issues or something
 else? What happens in such scenario, is there a data loss?
 
 Sorry for many questions just trying to understand since data is critical
 afterall :)
 
 --
 View this message in context: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Strange-corrupt-sstable-tp6314052p6314218.html
 Sent from the cassandra-u...@incubator.apache.org mailing list archive at 
 Nabble.com.
 



Re: Strange corrupt sstable

2011-04-29 Thread Daniel Doubleday
Bad == Broken

That means you cannot rely on 1 == 1. In such a scenario everything can happen 
including data loss. 
That's why you want ECC mem on production servers. Our cheapo dev boxes dont.

On Apr 28, 2011, at 7:46 PM, mcasandra wrote:

 What do you mean by Bad memory? Is it less heap size, OOM issues or something
 else? What happens in such scenario, is there a data loss?
 
 Sorry for many questions just trying to understand since data is critical
 afterall :)
 
 --
 View this message in context: 
 http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Strange-corrupt-sstable-tp6314052p6314218.html
 Sent from the cassandra-u...@incubator.apache.org mailing list archive at 
 Nabble.com.



Re: best way to backup

2011-04-29 Thread Daniel Doubleday
What we are about to set up is a time machine like backup. This is more like an 
add on to the s3 backup.

Our boxes have an additional larger drive for local backup. We create a new 
backup snaphot every x hours which hardlinks the files in the previous snapshot 
(bit like cassandras incremental_backups thing) and than we sync that snapshot 
dir with the cassandra data dir. We can do archiving / backup to external 
system from there without impacting the main data raid.

But the main reason to do this is to have an 'omg we screwed up big time and 
deleted / corrupted data' recovery.

On Apr 28, 2011, at 9:53 PM, William Oberman wrote:

 Even with N-nodes for redundancy, I still want to have backups.  I'm an 
 amazon person, so naturally I'm thinking S3.  Reading over the docs, and 
 messing with nodeutil, it looks like each new snapshot contains the previous 
 snapshot as a subset (and I've read how cassandra uses hard links to avoid 
 excessive disk use).  When does that pattern break down?  
 
 I'm basically debating if I can do a rsync like backup, or if I should do a 
 compressed tar backup.  And I obviously want multiple points in time.  S3 
 does allow file versioning, if a file or file name is changed/resused over 
 time (only matters in the rsync case).  My only concerns with compressed tars 
 is I'll have to have free space to create the archive and I get no delta 
 space savings on the backup (the former is solved by not allowing the disk 
 space to get so low and/or adding more nodes to bring down the space, the 
 latter is solved by S3 being really cheap anyways).
 
 -- 
 Will Oberman
 Civic Science, Inc.
 3030 Penn Avenue., First Floor
 Pittsburgh, PA 15201
 (M) 412-480-7835
 (E) ober...@civicscience.com



Strange corrupt sstable

2011-04-28 Thread Daniel Doubleday
Hi all

on one of our dev machines we ran into this:

INFO [CompactionExecutor:1] 2011-04-28 15:07:35,174 SSTableWriter.java (line 
108) Last written key : DecoratedKey(12707736894140473154801792860916528374, 
74657374)
 INFO [CompactionExecutor:1] 2011-04-28 15:07:35,174 SSTableWriter.java (line 
109) Current key : DecoratedKey(9639328335678452015784116654185025244, 
61c080c080c080c080c080c0803a60)
 INFO [CompactionExecutor:1] 2011-04-28 15:07:35,175 SSTableWriter.java (line 
110) Writing into file /var/lib/cassandra/data/Smeet/Tracker-tmp-f-752-Data.db
ERROR [CompactionExecutor:1] 2011-04-28 15:07:35,179 
AbstractCassandraDaemon.java (line 113) Fatal exception in thread 
Thread[CompactionExecutor:1,1,main]
java.io.IOException: Keys must be written in ascending order.
at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:111)
at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:128)
at 
org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:451)
at 
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:124)
at 
org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:94)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
at java.util.concurrent.FutureTask.run(FutureTask.java:166)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:636)


After looking into this we found that one SSTable was corrupt:

INFO 18:10:23,767 checking sstable 
/Users/dd/Temp/cassandra/data/Smeet/Tracker-f-21-Data.db
 INFO 18:10:23,773 DecoratedKey(12707736894140473154801792860916528374, 
74657374)
 INFO 18:10:23,777 DecoratedKey(9639328335678452015784116654185025244, 
61c080c080c080c080c080c0803a60)
 INFO 18:10:23,781 DecoratedKey(14745077953608664589374317792467471221, 
7265706f2d333631343039)
 INFO 18:10:23,783 DecoratedKey(20959981347795136420050187639971803838, 
6576656e746c6f672d333631383838)
 INFO 18:10:23,785 DecoratedKey(21549594967496343960898734735584772470, 
6576656e746c6f672d333631383837)
 INFO 18:10:23,787 DecoratedKey(21604488955263763122920767208831708343, 
6576656e746c6f672d333631383835)
 INFO 18:10:23,790 DecoratedKey(50100329769829417461462475380721306089, 
61c080c080c080c080c080c0803a5f)
 INFO 18:10:23,792 DecoratedKey(46089976107344533463199626515706634206, 
7265706f2d333631303436)
 INFO 18:10:23,794 DecoratedKey(50825113246224326722027014732583502298, 
6c6f674576656e742d333631383431)
 INFO 18:10:23,797 DecoratedKey(146509745693074257726394585872920772980, 
61c080c080c080c080c080c0803a59)
 INFO 18:10:23,801 DecoratedKey(63662277073629697060427801149510996510, 
6c6f674576656e742d333631383432)
 INFO 18:10:23,803 DecoratedKey(79706531464725069255477768601476202854, 
6576656e746c6f672d333631383834)
 INFO 18:10:23,808 DecoratedKey(99662740527281958972877086277408797556, 
6c6f674576656e742d333631383430)
 INFO 18:10:23,810 DecoratedKey(24216004522287050570821311008920284627, 
61c080c080c080c080c080c0803a5a)
 INFO 18:10:23,816 DecoratedKey(130819691046705791898531759480356609485, 
61c080c080c080c080c080c0803a5e)
 INFO 18:10:24,121 DecoratedKey(107312182597869061361893705497084751672, 
61c080c080c080c080c080c0803a58)
 INFO 18:10:24,123 DecoratedKey(142873550623537979564886292314205248041, 
7265706f2d333631363731)
 INFO 18:10:24,124 DecoratedKey(147476468396567670371575440897348610391, 
6576656e746c6f672d333631383836)

I couldn't figure out how SSTableWriter could possible write such a file so ... 
Just wanted to report. Maybe it rings a bell somewhere.

It seems really strange that some data corruption would leave the file readable 
for the SSTableReader.

History of this server is 0.6.5 - 0.7.4. But this file was obviously written 
by 0.7.4



Cheers,
Daniel




Re: Advice on mmap related swapping issue

2011-03-23 Thread Daniel Doubleday
FWIW: For whatever reason jna memlockall does not work for us. jna call is 
successful but cassandra process swaps anyway.

see: http://www.mail-archive.com/user@cassandra.apache.org/msg11235.html

We disabled swap entirely.


On Mar 22, 2011, at 8:56 PM, Chris Goffinet wrote:

 The easiest way to get memlock to work after putting the jna jar in your 
 classpath is just run this before:
 
 ulimit -a unlimited
 
 in your init script or before starting cassandra. The default for max locked 
 memory is 32KB on older kernels, and 64KB on newer ones.
 
 -Chris
 
 On Mar 22, 2011, at 12:52 PM, Adi wrote:
 
 On Tue, Mar 22, 2011 at 3:44 PM, ruslan usifov ruslan.usi...@gmail.com 
 wrote:
 
 
 2011/3/22 Adi adi.pan...@gmail.com
 I have been going through the mailing list and compiling suggestions to 
 address the swapping due to mmap issue. 
 
 1) Use JNA (done but)
 Are these steps also required:
 - Start Cassandra with CAP_IPC_LOCK (or as root). (not done)
 
 And what is CAP_IPC_LOCK?
 
 I saw that suggestion in 
 https://issues.apache.org/jira/browse/CASSANDRA-1214. 
 
 I do not yet know how to run cassandra or a java process with that 
 privilege, still researching and hoping my sysadmin knows better.
 
 http://www.lids.org/lids-howto/node50.html
 Allow locking of shared memory segments
 Allow mlock and mlockall (which doesn't really have anything to do with IPC)
 
 



Re: cassandra nodes with mixed hard disk sizes

2011-03-22 Thread Daniel Doubleday

On Mar 22, 2011, at 5:09 AM, aaron morton wrote:

 1) You should use nodes with the same capacity (CPU, RAM, HDD), cassandra 
 assumes they are all equal. 

Care to elaborate? While equal node will certainly make life easier I would 
have thought that  dynamic snitch would take care of performance differences 
and manual assignment of token ranges can yield to any data distribution. 
Obviously if a node has twice as much data will probably get twice the load. 
But if that is no problem ...

Where does cassandra assume that all are equal?  

Cheers Daniel


 
 2) Not sure what exactly would happen. Am guessing either the node would 
 shutdown or writes would eventually block, probably the former. If the node 
 was up read performance may suffer (if there were more writes been sent in). 
 If you really want to know more let me know and I may find time to dig into 
 it. 
 
 Also a node is be responsible for storing it's token range and acting as a 
 replica for other token ranges. So reducing the token range may not have a 
 dramatic affect on the storage requirements. 
 
 Hope that helps. 
 Aaron
 
 On 22 Mar 2011, at 09:50, Jonathan Colby wrote:
 
 
 This is a two part question ...
 
 1. If you have cassandra nodes with different sized hard disks,  how do you 
 deal with assigning the token ring such that the nodes with larger disks get 
 more data?   In other words, given equally distributed token ranges, when 
 the smaller disk nodes run out of space, the larger disk nodes with still 
 have unused capacity.Or is installing a mixed hardware cluster a no-no?
 
 2. What happens when a cassandra node runs out of disk space for its data 
 files?  Does it continue serving the data while not accepting new data?  Or 
 does the node break and require manual intervention?
 
 This info has alluded me elsewhere.
 Jon
 



jna and swapping

2011-03-15 Thread Daniel Doubleday
Hi all

strange things here: we are using jna. Log file says mlockall was successful. 
We start with -Xms2000M -Xmx2000M and run cassandra as root process so 
RLIMIT_MEMLOCK limit should have no relevance. Still cassandra is swapping ...

Used swap varies between 100MB - 800MB

We removed the swap partition altogether now but I still dont understand why 
this happens.

We see this on nodes with a longer uptime ( 2 weeks). 

Here's some process info: 

top - 14:27:35 up 146 days,  3:02,  1 user,  load average: 0.89, 0.97, 0.93
Tasks: 122 total,   1 running, 121 sleeping,   0 stopped,   0 zombie
Cpu(s):  1.4%us,  0.6%sy,  0.0%ni, 85.5%id, 12.6%wa,  0.0%hi,  0.0%si,  0.0%st
Mem:   6128360k total,  5852408k used,   275952k free, 4472k buffers
Swap:  1951892k total,   231008k used,  1720884k free,  1576720k cached

  PID USER  PR  NI  VIRT  RES  SHR S %CPU %MEMTIME+  COMMAND

   
29757 root  18  -2  251g 3.7g 298m S6 63.8   1590:17 java 


blnrzh019:/var/log/cassandra# ps axxx|grep 29757
29757 ?SLl 1589:56 /usr/bin/java -ea -Xms2000M -Xmx2000M 
-XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled 
-XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=1 
-XX:CMSInitiatingOccupancyFraction=75 -XX:+UseCMSInitiatingOccupancyOnly 
-Dcassandra.compaction.priority=1 -Dcassandra.dynamic_snitch=true 
-Dcom.sun.management.jmxremote.port=8080 
-Dcom.sun.management.jmxremote.ssl=false 
-Dcom.sun.management.jmxremote.authenticate=false 
-Dstorage-config=/opt/smeet-cassandra/bin/../conf -cp 
/opt/smeet-cassandra/bin/../conf:/opt/smeet-cassandra/bin/../build/classes:/opt/smeet-cassandra/bin/../lib/antlr-3.1.3.jar:/opt/smeet-cassandra/bin/../lib/apache-cassandra-0.6.12-patched.jar:/opt/smeet-cassandra/bin/../lib/clhm-production.jar:/opt/smeet-cassandra/bin/../lib/commons-cli-1.1.jar:/opt/smeet-cassandra/bin/../lib/commons-codec-1.2.jar:/opt/smeet-cassandra/bin/../lib/commons-collections-3.2.1.jar:/opt/smeet-cassandra/bin/../lib/commons-lang-2.4.jar:/opt/smeet-cassandra/bin/../lib/google-collections-1.0.jar:/opt/smeet-cassandra/bin/../lib/hadoop-core-0.20.1.jar:/opt/smeet-cassandra/bin/../lib/high-scale-lib.jar:/opt/smeet-cassandra/bin/../lib/ivy-2.1.0.jar:/opt/smeet-cassandra/bin/../lib/jackson-core-asl-1.4.0.jar:/opt/smeet-cassandra/bin/../lib/jackson-mapper-asl-1.4.0.jar:/opt/smeet-cassandra/bin/../lib/jline-0.9.94.jar:/opt/smeet-cassandra/bin/../lib/jna-3.2.7.jar:/opt/smeet-cassandra/bin/../lib/jna.jar:/opt/smeet-cassandra/bin/../lib/json-simple-1.1.jar:/opt/smeet-cassandra/bin/../lib/libthrift-r917130.jar:/opt/smeet-cassandra/bin/../lib/log4j-1.2.14.jar:/opt/smeet-cassandra/bin/../lib/slf4j-api-1.5.8.jar:/opt/smeet-cassandra/bin/../lib/slf4j-log4j12-1.5.8.jar:/opt/smeet-cassandra/bin/../lib/smeet-cassandra-contrib.jar
 org.apache.cassandra.thrift.CassandraDaemon

blnrzh019:/var/log/cassandra# cat /proc/29757/smaps |grep -i swap| awk '{SUM += 
$2} END {print SUM:  SUM  kB ( SUM/1024  MB)}'
SUM: 207844 kB (202.973 MB)

blnrzh019:/var/log/cassandra# grep JNA /var/log/cassandra/system.log*
/var/log/cassandra/system.log.1: INFO [main] 2011-01-27 17:38:11,201 
CLibrary.java (line 86) JNA mlockall successful
/var/log/cassandra/system.log.1: INFO [main] 2011-02-16 07:47:24,788 
CLibrary.java (line 86) JNA mlockall successful
/var/log/cassandra/system.log.1: INFO [main] 2011-02-18 12:29:39,958 
CLibrary.java (line 86) JNA mlockall successful
/var/log/cassandra/system.log.1: INFO [main] 2011-02-25 11:59:42,318 
CLibrary.java (line 86) JNA mlockall successful



Re: nodetool repair on cluster

2011-03-15 Thread Daniel Doubleday
At least if you are using RackUnawareStrategy

Cheers,
Daniel

On Mar 15, 2011, at 6:44 PM, Huy Le wrote:

 Hi,
 
 We have a cluster with 12 servers and use RF=3.  When running nodetool 
 repair, do we have to run it on all nodes on the cluster or can we run on 
 every 3rd node?  Thanks!
 
 Huy
 
 -- 
 Huy Le 
 Spring Partners, Inc.
 http://springpadit.com 



mixed cluster 0.6.9 and 0.6.12

2011-03-09 Thread Daniel Doubleday
Hi all

we are still on 0.6.9 and plan to upgrade to 0.6.12 but are a little concerned 
about:

https://issues.apache.org/jira/browse/CASSANDRA-2170

I thought of upgrading only one node (of 5) to .12 and monitor for a couple of 
days.

Is this a bad idea?

Thanks,
Daniel

Re: Alternative to repair

2011-03-08 Thread Daniel Doubleday
Thanks for the reply!

 Not really:
 
 - range scans do not perform read repair

Ok I obviously overlooked that RangeSliceResponseResolver does not repair rows 
on nodes that never saw a write for a given key at all. But that's not a big 
problem for us since we are mainly interested in fixing missed deletions. And 
read repairs for conflicting updates seem to work fine too.

 - if you converted it to range scan + [multi]get, the RR messages are
 fair game to drop to cope with load (active repair messages are
 never dropped in 0.6.7+)

We actually have a little hack for that: In the special case of CL_ALL on range 
slice queries we perform synchronous mutations as read repairs (instead of read 
repair messages). This way we get timeouts for the read when a repair fails. In 
that case we restart at the given token and continue from there when the 
cluster load is lower.

For the time being I guess thats good enough and we hope that 0.7 works a 
little smoother when doing repairs.

Cheers,
Daniel


On Mar 7, 2011, at 7:22 PM, Jonathan Ellis wrote:

 On Mon, Mar 7, 2011 at 11:18 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Since we already have a very simple hadoopish framework in place which 
 allows us to do token range walks with multiple workers and restart at a 
 given position in case of failure I created a simple worker that would read 
 everything with CL_ALL. With only one worker and almost no performance 
 impact one scan took 7h.
 
 My understanding is that at that point due to read repair I got the same as 
 I would have achieved with repair runs.
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of DataStax, the source for professional Cassandra support
 http://www.datastax.com



Alternative to repair

2011-03-07 Thread Daniel Doubleday
Hi all

we're still on 0.6 and are facing problems with repairs. 

I.e. a repair for one CF takes around 60h and we have to do that twice (RF=3, 5 
nodes). During that time the cluster is under pretty heavy IO load. It kinda 
works but during peek times we see lots of dropped messages (including writes). 
So we are actually creating inconsistencies that we are trying to fix with the 
repair.

Since we already have a very simple hadoopish framework in place which allows 
us to do token range walks with multiple workers and restart at a given 
position in case of failure I created a simple worker that would read 
everything with CL_ALL. With only one worker and almost no performance impact 
one scan took 7h.

My understanding is that at that point due to read repair I got the same as I 
would have achieved with repair runs.

Is that true or am I missing something?

Cheers,
Daniel



Re: Does variation in no of columns in rows over the column family has any performance impact ?

2011-02-07 Thread Daniel Doubleday
It depends a little on your write pattern:

- Wide rows tend to get distributed over more sstables so more disk reads are 
necessary. This will become noticeable when you have high io load and reads 
actually hit the discs.
- If you delete a lot slice query performance might suffer: extreme example: 
create 2M cols, delete the first 1M and then ask for the first 10.


On Feb 7, 2011, at 7:07 AM, Aditya Narayan wrote:

 Does huge variation in no. of columns in rows, over the column family
 has *any* impact on the performance ?
 
 Can I have like just 100 columns in some rows and like hundred
 thousands of columns in another set of rows, without any downsides ?



Re: Using Cassandra to store files

2011-02-04 Thread Daniel Doubleday
We are doing this with cassandra.

But we cache a lot. We get around 20 writes/s and 1k reads/s (~ 100Mbit/s) for 
that particular CF but only 1% of them hit our cassandra cluster (5 nodes, 
rf=3).

/Daniel

On Feb 4, 2011, at 9:37 AM, Brendan Poole wrote:

 Hi Daniel
  
 When you say We are doing this do you mean via NFS or Cassandra.
  
 Thanks
  
 Brendan
  
  
  
 
 
 
 Signature.jpg Brendan Poole
  Systems Developer
   NewLaw Solicitors
  Helmont House  
  Churchill Way
  Cardiff
  brendan.po...@new-law.co.uk
  029 2078 4283
  www.new-law.co.uk
 
 
 
 
 
 From: Daniel Doubleday [mailto:daniel.double...@gmx.net] 
 Sent: 03 February 2011 17:21
 To: user@cassandra.apache.org
 Subject: Re: Using Cassandra to store files
 
 Hundreds of thousands doesn't sound too bad. Good old NFS would do with an ok 
 directory structure.
 
 We are doing this. Our documents are pretty small though (a few kb). We have 
 around 40M right now with around 300GB total.
 
 Generally the problem is that much data usually means that cassandra becomes 
 io bound during repairs and compactions even if your hot dataset would fit in 
 the page cache. There are efforts to overcome this and 0.7 will help with 
 repair problems but for the time being you have to have quite some headroom 
 in terms of io performance to handle these situations.  
 
 Here is a related post:
 
 http://comments.gmane.org/gmane.comp.db.cassandra.user/11190
 
 On Feb 3, 2011, at 1:33 PM, Brendan Poole wrote:
 
 Hi
  
 Would anyone recommend using Cassandra for storing hundreds of thousands of 
 documents in Word/PDF format? The manual says it can store documents under 
 64MB with no issue but was wondering if anyone is using it for this specific 
 perpose.  Would it be efficient/reliable and is there anything I need to 
 bear in mind?
  
 Thanks in advance
  
 
 Signature.jpg Brendan Poole
  Systems Developer
   NewLaw Solicitors
  Helmont House  
  Churchill Way
  Cardiff
  brendan.po...@new-law.co.uk
  029 2078 4283
  www.new-law.co.uk
 
 
  
 
 
 P Please consider the environment before printing this e-mail
 Important - The information contained in this email (and any attached files) 
 is confidential and may be legally privileged and protected by law.
 The intended recipient is authorised to access it. If you are not the 
 intended recipient, please notify the sender immediately and delete or 
 destroy all copies. You must not disclose the contents of this email to 
 anyone. Unauthorised use, dissemination, distribution, publication or 
 copying of this communication is prohibited.
 NewLaw Solicitors does not accept any liability for any inaccuracies or 
 omissions in the contents of this email that may have arisen as a result of 
 transmission. This message and any attachments are believed to be free of 
 any virus or defect that might affect any computer system into which it is 
 received and opened. However, it is the responsibility of the recipient to 
 ensure that it is virus free; therefore, no responsibility is accepted for 
 any loss or damage in any way arising from its use.
 NewLaw Solicitors is the trading name of NewLaw Legal Ltd, a limited company 
 registered in England and Wales with registered number 07200038.
 NewLaw Legal Ltd is regulated by the Solicitors Regulation Authority whose 
 website is http://www.sra.org.uk
 The registered office of NewLaw Legal Ltd is at Helmont House, Churchill 
 Way, Cardiff, CF10 2HE. Tel: 0845 756 6870, Fax: 0845 756 6871, Email: 
 i...@new-law.co.uk. www.new-law.co.uk.
 We use the word ‘partner’ to refer to a shareowner or director of the 
 company, or an employee or consultant of the company who is a lawyer with 
 equivalent standing and qualifications. A list of the directors is displayed 
 at the above address, together with a list of those persons who are 
 designated as partners.
 
  
 
 
 P Please consider the environment before printing this e-mail
 Important - The information contained in this email (and any attached files) 
 is confidential and may be legally privileged and protected by law.
 The intended recipient is authorised to access it. If you are not the 
 intended recipient, please notify the sender immediately and delete or 
 destroy all copies. You must not disclose the contents of this email to 
 anyone. Unauthorised use, dissemination, distribution, publication or copying 
 of this communication is prohibited.
 NewLaw Solicitors does not accept any liability for any inaccuracies or 
 omissions in the contents of this email that may have arisen as a result of 
 transmission. This message and any attachments are believed to be free of any 
 virus or defect that might affect any computer system into which it is 
 received and opened. However, it is the responsibility of the recipient to 
 ensure that it is virus free; therefore, no responsibility is accepted for 
 any loss or damage in any way arising from its use

Problematic usage pattern

2010-12-22 Thread Daniel Doubleday
Hi all

wanted to share a cassandra usage pattern you might want to avoid (if you can).

The combinations of 

- heavy rows,
- large volume and
- many updates (overwriting columns)

will lead to a higher count of live ssts (at least if you're not starting mayor 
compactions a lot) with many ssts actually containing the same hot rows. 
This will lead to loads of multiple reads that will increase latency and io 
pressure by itself and making the page cache less effective because it will 
contains loads of 'invalid' data.

In our case we could reduce reads by ~40%. Our rows contained one large column 
(1-4k) and some 50 - 100 small columns.
We splitted into 2 CFs and stored the large column in the other CF with a UUID 
as row key which we store in CF1. We cache the now light-weight rows in the row 
cache which eliminates the update problem and instead of updating the large 
column we create a new row and delete the other one. That way the bloom filter 
prevents unnecessary reads. 

The downside is that to read the large column from CF2 we have to read CF1 
first but since that one is in the row cache that still way better.

To monitor this we did a very small patch which records the file scans for a CF 
in a histogram in a similar way as the latency stats.

If someone's interested - here is the patch agains 0.6.8:

https://gist.github.com/751601

Cheers,
Daniel
smeet.com, Berlin





Re: Virtual IP / hardware load balancing for cassandra nodes

2010-12-20 Thread Daniel Doubleday
You will loose part of the retry / fallback functionality offered by hector.

The job of the client lib is not only load-balancing. I.e. if a node is 
bootstrapping it will accept TCP connections but throw an exception which will 
be communicated via thrift. The client lib is supposed to handle that.
 
On Dec 20, 2010, at 3:25 PM, Jonathan Colby wrote:

 I was unable to find example or documentation on my question.  I'd like to 
 know what the best way to group a cluster of cassandra nodes behind a virtual 
 ip.
 
 For example, can cassandra nodes be placed behind a Citrix Netscaler hardware 
 load balancer?   
 
 I can't imagine it being a problem, but in doing so would you break any 
 cassandra functionality?
 
 The goal is to have the application talk to a single virtual ip  and be 
 directed to a random node in the cluster.
 
 I heard a little about adding the node addresses to Hector's load-balancing 
 mechanism, but this doesn't seem too robust or easy to maintain.
 
 Thanks in advance.



Re: Dynamic Snitch / Read Path Questions

2010-12-17 Thread Daniel Doubleday
 the purpose of your thread is: How far are you away from being I/O
 bound (say in terms of % utilization - last column of iostat -x 1 -
 assuming you don't have a massive RAID underneath the block device)

No my cheap boss didn't want to by me a stack of these 
http://www.ocztechnology.com/products/solid-state-drives/pci-express/z-drive-r2/mlc-performance-series/ocz-z-drive-r2-p88-pci-express-ssd.html

But seriously: we don't know yet what the best way in terms of TCO is. Maybe 
its worth investing 2k in SSDs if that machine could than handle the load of 3.


 when compaction/AESis *not* running? I.e., how much in relative terms,
 in terms of time spent by disks servicing requests is added by
 compaction/AES?
 

Can't really say in terms of util% because we only monitor IO waits in zabbix. 
Now with our cluster running smoothly I'd say compactions adds around 15-20%.
In terms of IO waits we saw our graphs jumped during compactions

- from 20 - 30% to 50%  with 'ok' load (reqs where handled at around 100ms max 
and no messages dropped) and
- from 50% - 80/90% during peak hours. Things got ugly then

 Are your values in generally largish (say a few kb or some such)or
 very small (5-50 bytes) or somewhere in between? I've been trying to
 collect information when people report compaction/repair killing their
 performance. My hypothesis is that most sever issues are for data sets
 where compaction becomes I/O bound rather than CPU bound (for those
 that have seen me say this a gazillion times I must be sounding like
 I'm a stuck LP record); and this would tend to be expected with larger
 and fewer values as opposed to smaller and more numerous values as the
 latter is much more expensive in terms of CPU cycles per byte
 compacted. Further I expect CPU bound compaction to be a problem very
 infrequently in comparison. I'm trying to confirm or falsify the
 hypothesis.

Well we have 4 CFs with different characteristics but it seems that what made 
things go wrong was a CF with ~2k cols. I have never seen CPU user time over 
30% on any of the nodes. So I second your hypothesis

 
 -- 
 / Peter Schuller



Re: Read Latency Degradation

2010-12-17 Thread Daniel Doubleday
 How much ram is dedicated to cassandra? 12gb heap (probably too high?)
 What is the hit rate of caches? high, 90%+

If your heap allows it I would definitely try to give more ram for fs cache. 
Your not using row cache so I don't see what cassandra would gain from so much 
memory.

A question about your tests:

I assume that they run isolated (you load test one cf at a time) and the 
results are the same byte-wise?
So the only difference is that one time you are reading from a larger file?

Do you see the same IO load in both tests? Do you use mem-mapped io? And if so 
are the number of page faults the same in both tests?

In the end it could just be more physical movements of the disc heads with 
larger files ...


On Dec 17, 2010, at 5:46 PM, Wayne wrote:

 Below are some answers to your questions. We have wide rows (what we like 
 about Cassandra) and I wonder if that plays into this? We have been loading 1 
 keyspace in our cluster heavily in the last week so it is behind in 
 compaction for that keyspace. I am not even looking at those read latency 
 times as there are as many as 100+ sstables. Compaction will run tomorrow for 
 all nodes (weekend is our slow time) and I will test the read latency there. 
 For the keyspace/CFs that are already well compacted we are seeing a steady 
 increase in read latency as the total sstable size grows and a linear 
 relationship between our different keyspaces cfs sizes and the read latency 
 for reads.
 
 How many nodes? 10 - 16 cores each (2 x quad ht cpus)
 How much ram per node? 24gb
 What disks and how many? SATA 7200rpm 1x1tb for commit log, 4x1tb (raid0) for 
 data 
 Is your ring balanced? yes, random partitioned very evenly
 How many column families? 4 CFs x 3 Keyspaces
 How much ram is dedicated to cassandra? 12gb heap (probably too high?)
 What type of caching are you using? Key caching
 What are the sizes of caches? 500k-1m values for 2 of the CFs
 What is the hit rate of caches? high, 90%+
 What does your disk utiliztion|CPU|Memory look like at peak times? Disk goes 
 to 90%+ under heavy read load. CPU load high as well. Latency does not change 
 that much for single reads vs. under load (30 threads). We can keep current 
 read latency up to 25-30 read threads if no writes or compaction is going on. 
 We are worried about what we see in terms of latency for a single read.
 What are your average mean|max row size from cfstats? 30k avg/5meg max for 
 one CF and 311k avg/855k max for the other.
 On average for a given sstable how large is the data bloom and index files? 
 30gig data, 189k filter, 5.7meg index for one CF, 98gig data, 587k filter, 
 18meg index for the other.
 
 Thanks.
 
 
 
 On Fri, Dec 17, 2010 at 10:58 AM, Edward Capriolo edlinuxg...@gmail.com 
 wrote:
 On Fri, Dec 17, 2010 at 8:21 AM, Wayne wav...@gmail.com wrote:
  We have been testing Cassandra for 6+ months and now have 10TB in 10 nodes
  with rf=3. It is 100% real data generated by real code in an almost
  production level mode. We have gotten past all our stability issues,
  java/cmf issues, etc. etc. now to find the one thing we assumed may not be
  true. Our current production environment is mysql with extensive
  partitioning. We have mysql tables with 3-4 billion records and our query
  performance is the same as with 1 million records ( 100ms).
 
  For those of us really trying to manage large volumes of data memory is not
  an option in any stretch of the imagination. Our current data volume once
  placed within Cassandra ignoring growth should be around 50 TB. We run
  manual compaction once a week (absolutely required to keep ss table counts
  down) and it is taking a very long amount of time. Now that our nodes are
  past 1TB I am worried it will take more than a day. I was hoping everyone
  would respond to my posting with something must be wrong, but instead I am
  hearing you are off the charts good luck and be patient. Scary to say the
  least given our current investment in Cassandra. Is it true/expected that
  read latency will get worse in a linear fashion as the ss table size grows?
 
  Can anyone talk me off the fence here? We have 9 MySQL servers that now
  serve up 15+TB of data. Based on what we have seen we need 100 Cassandra
  nodes with rf=3 to give us good read latency (by keeping the node data sizes
  down). The cost/value equation just does not add up.
 
  Thanks in advance for any advice/experience you can provide.
 
 
  On Fri, Dec 17, 2010 at 5:07 AM, Daniel Doubleday daniel.double...@gmx.net
  wrote:
 
  On Dec 16, 2010, at 11:35 PM, Wayne wrote:
 
   I have read that read latency goes up with the total data size, but to
   what degree should we expect a degradation in performance? What is the
   normal read latency range if there is such a thing for a small slice of
   scol/cols? Can we really put 2TB of data on a node and get good read 
   latency
   querying data off of a handful of CFs? Any experience or explanations 
   would
   be greatly

Re: org.apache.cassandra.service.ReadResponseResolver question

2010-12-15 Thread Daniel Doubleday

On Dec 14, 2010, at 9:20 PM, Jonathan Ellis wrote:

 Correct.  https://issues.apache.org/jira/browse/CASSANDRA-1830 is open to fix 
 that.  If you'd like to review the patch there, that would be very helpful. :)

That patch looks good to me :-) Should have checked jira first ...

Speaking of which, https://issues.apache.org/jira/browse/CASSANDRA-982 is 
referenced there and seems to be pretty close to something I was trying to do 
last 2 days.

I'm not going to repeat the reasoning here, but its this thread: 
http://thread.gmane.org/gmane.comp.db.cassandra.user/10927/focus=10977

Just wanted to mention that I implemented my idea and did some functional 
testing and load testing. Though certainly not enough ...

But I was able to test 
- normal read behavior (all nodes up, two nodes up)
- normal failure behavior (not enough nodes up)
- behavior when the environment changes (affecting cores by controlling latency 
in the ReadVerbHandler, Timeouts in the read path, Exceptions in read path of a 
selected node, nodes going down during a read)

So far it looks pretty promising. Everything worked as expected.

The only real draw back I found is when a read fails on a selected node (such 
as an exception). As far as I understand it there's no way to signal the 
readresolvehandler to return early in this case. Thus you have to wait for the 
timeout until the rest of the nodes are consulted. But I hope that failure 
detection + scores should be good enough to prevent this from happening to 
often.

I did some load testing and compared with vanilla cassandra. It's one of our 
use cases we have in production. Its a chat app. So it writes and reads 
messages and offline notifications. It's of limited use though since I was not 
able to reproduce our IO overload yet.

But to give a first impression: In this rather cpu bound test the patched 
version did ~20 - 25% more tests. Test was on 3 nodes, rf 3, quorum read / 
writes. reproduced many times.

I am currently working on a load test to reproduce the problem in our 
production environment last week.

If someone's interested (note that this makes only sense - if at all - for 
quorum reads with the dynamic snitch):

That's the patch I did to 0.6.8:  https://gist.github.com/742280

And of course I'd be glad to get feedback if someone feels that I am about to 
lose my job... 

Thanks,

Daniel
smeet.com, Berlin


 On Tue, Dec 14, 2010 at 1:55 PM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Hi
 
 I'm sorry - don't want to be a pain in the neck with source questions. So 
 please just ignore me if this is stupid:
 
 Isn't org.apache.cassandra.service.ReadResponseResolver suposed to throw a 
 DigestMismatchException if it receives a digest wich does not match the 
 digest of a read message?
 
 If messages contains multiple digest responses it will drop all but one. So 
 if any of the dropped digest are a mismatch to the version that mismatch is 
 simply ignored.
 It can cope with multiple reads (versions) but not with multiple digests and 
 that's what it gets from quorum reads.
 
 It might be an edge case, but I think that would break quorum promise with rf 
  3 because you could have 1 broken data message, 1 broken digest message and 
 2 good digest messages. If the 2 good messages were dropped than the quorum 
 read that should have triggered repair and conflict resolution would return 
 old data.
 
 I just can't see what I'm not seeing here.
 
 Cheers,
 Daniel
 
 
 
 
 
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com



Re: Dynamic Snitch / Read Path Questions

2010-12-14 Thread Daniel Doubleday

On Dec 14, 2010, at 2:29 AM, Brandon Williams wrote:

 On Mon, Dec 13, 2010 at 6:43 PM, Daniel Doubleday daniel.double...@gmx.net 
 wrote:
 Oh - well but I see that the coordinator is actually using its own score for 
 ordering. I was only concerned that dropped messages are ignored when 
 calculating latencies but that seems to be the case for local or remote 
 responses. And even than I guess you can assume that enough slow messages 
 arrive to destroy the score.
  
 That's odd, since it should only be tracking READ_RESPONSE messages... I'm 
 not sure how a node would send one to itself.

As far as I understand it the MessagingService is always user in the strong 
read path. Local messages will be shortcutted transport-wise but 
MessageDeliveryTask will still be used which in turn calls the 
ResponseVerbHandler which notifies the snitch about latencies.

It's only in the weak read path where the MessagingService is not used at all. 
And it will always use the local data and (I think) latencies are not recorded. 
 

 
 Maybe I misunderstand but that would not really lead to less load right. I 
 don't think that inconsistency / read repairs are the problem which leads to 
 high io load but the digest requests. Turning off read repair would also lead 
 to inconsistent reads which invalidates the whole point of quorum reads (at 
 least in 0.6. I think rr probability has no effect in strong reads in 0.7) . 
 Again assuming I am not misinterpreting the code.
 
 Ah, I see what you want to do: take a chance that you pick the two replicas 
 (at RF=3, at least) that should agree, and only send the last checksum 
 request if you lose (at the price of latency.)
 

Yes exactly. I want to use the two endpoints with the best score (according to 
dynamic snitch). As soon as I have test results I'll post them here.

Thanks,
Daniel

org.apache.cassandra.service.ReadResponseResolver question

2010-12-14 Thread Daniel Doubleday
Hi

I'm sorry - don't want to be a pain in the neck with source questions. So 
please just ignore me if this is stupid:

Isn't org.apache.cassandra.service.ReadResponseResolver suposed to throw a 
DigestMismatchException if it receives a digest wich does not match the digest 
of a read message?

If messages contains multiple digest responses it will drop all but one. So if 
any of the dropped digest are a mismatch to the version that mismatch is simply 
ignored.
It can cope with multiple reads (versions) but not with multiple digests and 
that's what it gets from quorum reads. 

It might be an edge case, but I think that would break quorum promise with rf  
3 because you could have 1 broken data message, 1 broken digest message and 2 
good digest messages. If the 2 good messages were dropped than the quorum read 
that should have triggered repair and conflict resolution would return old data.

I just can't see what I'm not seeing here.

Cheers,
Daniel




Re: Dynamic Snitch / Read Path Questions

2010-12-13 Thread Daniel Doubleday

On 13.12.10 21:15, Brandon Williams wrote:
On Sun, Dec 12, 2010 at 10:49 AM, Daniel Doubleday 
daniel.double...@gmx.net mailto:daniel.double...@gmx.net wrote:


Hi again.

It would be great if someone could comment whether the following
is true or not.
I tried to understand the consequences of using
|-Dcassandra.dynamic_snitch=true for the read path |and that's
what I came up with:

1) If using CL  1 than using the dynamic snitch will result in a
data read from node with the lowest latency (little simplified)
even if the proxy node contains the data but has a higher latency
that other possible nodes which means that it is not necessary to
do load-based balancing on the client side.


No.  If the coordinator node is part of the replica set, the dynamic 
snitch will fall back to the wrapped snitch for ordering, since it 
does not track latencies of itself.  This likely means it will return 
the data.
Oh - well but I see that the coordinator is actually using its own score 
for ordering. I was only concerned that dropped messages are ignored 
when calculating latencies but that seems to be the case for local or 
remote responses. And even than I guess you can assume that enough slow 
messages arrive to destroy the score.


The next one goes a little further:

We read / write with quorum / rf = 3.

It seems to me that it wouldn't be hard to patch the StorageProxy
to send only one read request and one digest request. Only if one
of the requests fail we would have to query the remaining node. We
don't need read repair because we have to repair once a week
anyways and quorum guarantees consistency. This way we could
reduce read load significantly which should compensate for latency
increase by failing reads. Am I missing something?


Just turn off read repair in that case.
Maybe I misunderstand but that would not really lead to less load right. 
I don't think that inconsistency / read repairs are the problem which 
leads to high io load but the digest requests. Turning off read repair 
would also lead to inconsistent reads which invalidates the whole point 
of quorum reads (at least in 0.6. I think rr probability has no effect 
in strong reads in 0.7) . Again assuming I am not misinterpreting the code.


-Brandon




Dynamic Snitch / Read Path Questions

2010-12-12 Thread Daniel Doubleday

Hi again.

It would be great if someone could comment whether the following is true 
or not.
I tried to understand the consequences of using 
|-Dcassandra.dynamic_snitch=true for the read path |and that's what I 
came up with:


1) If using CL  1 than using the dynamic snitch will result in a data 
read from node with the lowest latency (little simplified) even if the 
proxy node contains the data but has a higher latency that other 
possible nodes which means that it is not necessary to do load-based 
balancing on the client side.


2) If using CL =1 than the proxy node will always return the data itself 
even when there is another node with less load.


3) Digest requests will be sent to all other living peer nodes for that 
key and will result in a data read on all nodes to calculate the digest. 
The only difference is that the data is not sent back but IO-wise it is 
just as expensive.



The next one goes a little further:

We read / write with quorum / rf = 3.

It seems to me that it wouldn't be hard to patch the StorageProxy to 
send only one read request and one digest request. Only if one of the 
requests fail we would have to query the remaining node. We don't need 
read repair because we have to repair once a week anyways and quorum 
guarantees consistency. This way we could reduce read load significantly 
which should compensate for latency increase by failing reads. Am I 
missing something?



Best,
Daniel





Re: Stuck with adding nodes

2010-12-10 Thread Daniel Doubleday
Thanks for your help Peter.

We gave up and rolled back to our mysql implementation (we did all writes to 
our old store in parallel so we did not lose anything).
Problem was that every solution we came up with would require at least on major 
compaction before the new nodes could join and our cluster could not survive 
this (in terms of serving requests at reasonable latencies).

But thanks anyway,
Daniel

On Dec 9, 2010, at 8:25 PM, Peter Schuller wrote:

 Currently I am copying all data files (thats all existing data) from one 
 node to the new nodes in hope that I could than manually assign them their 
 new tokenrange (nodetool move) and do cleanup.
 
 Unless I'm misunderstanding you I believe you should be setting the
 initial token. nodetool move would be for a node already in the ring.
 And keep in mind that a nodetool move is currently a
 decommission+bootstrap - so if you're teetering on the edge of
 overload you will want to keep that in mind when moving a node to
 avoid ending up in a worse situation as another node temporarily
 receives more load than usual as a result of increased ring ownership.
 
 Obviously I will try this tomorrow (it's been a long day) on a test system 
 but any advice would be highly appreciated.
 
 One possibility if you have additional hardware to spare temporarily,
 is to add more nodes than you actually need and then, once you are
 significantly over capacity, you have the flexibility to move nodes
 around to an optimum position and then decommission those machines
 that were only borrowed. I.e., initial bootstrap of nodes takes a
 shorter amount of time because you're giving them less token space per
 new node. And once all are in the ring, you're free to move things
 around and then free up the hardware.
 
 (Another option may be to implement throttling of the anti-compaction
 so that it runs very slowly during peak hours, but that requires
 patching cassandra or else firewall/packet filtering fu and is
 probably likely to be more risky than it's worth.)
 
 -- 
 / Peter Schuller



Stuck with adding nodes

2010-12-09 Thread Daniel Doubleday
Hi good people.

I underestimated load during peak times and now I'm stuck with our production 
cluster. 
Right now its 3 nodes, rf 3 so everything is everywhere. We have ~300GB data 
load. ~10MB/sec incoming traffic and ~50 (peak) reads/sec to the cluster

The problem derives from our quorum read / writes: At peak hours one of the 
machines (thats random) will fall behind because its a little slower than the 
others and than shortly after that it will drop most read requests. So right 
now the only way to survive is to take one machine down making every read / 
write a ALL operation. It's necessary to take one machine down because 
otherwise users will wait for timeouts from that overwhelmed machine when the 
client lib chooses it. Since we are a real time oriented thing thats a killer.

So now we tried to add 2 more nodes. Problem is that anticompaction takes to 
long. Meaning it is not done when peak hour arrives and the machine that would 
stream the data to the new node must be taken down. We tried to block the ports 
7000 and 9160 to that machine because we hoped that would stop traffic and let 
the machine end anticompaction. But that did not work because we could not cut 
the already existing connections to the other nodes.

Currently I am copying all data files (thats all existing data) from one node 
to the new nodes in hope that I could than manually assign them their new 
tokenrange (nodetool move) and do cleanup.

Obviously I will try this tomorrow (it's been a long day) on a test system but 
any advice would be highly appreciated.

Sighs and thanks.
Daniel

smeet.com
Berlin

Dont bogart that connection my friend

2010-12-03 Thread Daniel Doubleday
Hi all

I have found an anti pattern the other day which I wanted to share, although 
its pretty special case.

Special case because our production cluster is somewhat strange: 3 servers, rf 
= 3. We do consistent reads/writes with quorum.

I did a long running read series (loads of reads as fast as I can) with one 
connection. Since all queries could be handled by that node the overall latency 
is determined by its own and the fastest second node (cause the quorum is 
satisfied with 2 reads). What will happen than is that after a couple of 
minutes one of the other two nodes will go in 100% io wait and will drop most 
of its read messages. Leaving it practically dead while the other 2 nodes keep 
responding at an average of ~10ms. The node that died was only a little slower 
~13ms average but it will inevitably queue up messages. Average response time 
increases to timeout (10 secs) flat. It never recovers.

It happened all the time. And it wasn't the same node that would die.

The solution was that I return the connection to the pool and get a new one for 
every read to balance the load on the client side.

Obviously this will not happen in a cluster where the percentage of all rows on 
one node is enough. But the same thing will probably happen if you scan by 
continuos tokens (meaning that you will read from the same node a long time).

Cheers,

Daniel Doubleday
smeet.com, Berlin

Re: Best Practice for Data Center Migration

2010-12-03 Thread Daniel Doubleday
Hm - 

assuming that you have configured your initial tokens in a way that every next 
start token lives in the other datacenter wouldn't it suffice to decrease rf to 
2 switch to simple replication strategy switch off the old dc and start 
repairs/cleanup?

every row should live in either primary node or the node after the the primary 
(when the primary was located in the switched off dc)

Daniel Doubleday
smeet.com, Berlin

On Dec 2, 2010, at 6:11 PM, Jonathan Ellis wrote:

 On Thu, Dec 2, 2010 at 4:08 AM, Jake Maizel j...@soundcloud.com wrote:
 Hello,
 
 We have a ring of 12 nodes with 6 in one data center and 6 in another.
  We want to shutdown all 6 nodes in data center 1 in order to close
 it down.  We are using a replication factor of 3 and are using
 RackAwareStrategy with version 0.6.6.
 
 We have been thinking that using decomission on each of the nodes in
 the old data center one at a time would do the trick.  Does this sound
 reasonable?
 
 That is the simplest approach.  The major downside is that
 RackAwareStrategy guarantees you will have at least one copy of _each_
 row in both DCs, so when you are down to 1 node in dc1 it will have a
 copy of all the data.  If you have a small enough data volume to make
 this feasible then that is the option I would go with.
 
 We have also been considering increasing the replication factor to 4
 and then just shutting down all the old nodes.  Would that work as far
 as data availability would go?
 
 Not sure what you are thinking of there, but probably not. :)
 
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com



Re: Dont bogart that connection my friend

2010-12-03 Thread Daniel Doubleday

Yes.

I thought that would make sense, no? I guessed that the quorum read 
forces the slowest of the 3 nodes to keep the pace of the faster ones. 
But it cant. No matter how small the performance diff is. So it will 
just fill up.


Also when saying 'practically dead' and 'never recovers' I meant for the 
time I kept the reads up. As soon as I stopped the scan it recovered. It 
just was not able to recover during the load because for that it would 
have to become faster that the other nodes and with full queues that 
just wouldn't happen.


By changing the node for every read I would hit the slower node every 
couple of reads. This forced the client to wait for the slower node.


I guess to change that behavior you would need to use something like 
dynamic snitch and ask only as many peer nodes as necessary to satisfy 
quorum and only ask other nodes when reads fail. But that would probably 
increase latency and cause whatever other problems. Since you probably 
don't want to run the cluster at a load at which the weakest node of a 
replication group can't keep up I don't think this is an issue at all.


Just wanted to prevent others shooting their own foot as I did.

On 03.12.10 23:36, Jonathan Ellis wrote:

Am I understanding correctly that you had all connections going to one
cassandra node, which caused one of the *other* nodes to die, and
spreading the connections around the cluster fixed it?

On Fri, Dec 3, 2010 at 4:00 AM, Daniel Doubleday
daniel.double...@gmx.net  wrote:

Hi all

I have found an anti pattern the other day which I wanted to share, although 
its pretty special case.

Special case because our production cluster is somewhat strange: 3 servers, rf 
= 3. We do consistent reads/writes with quorum.

I did a long running read series (loads of reads as fast as I can) with one 
connection. Since all queries could be handled by that node the overall latency 
is determined by its own and the fastest second node (cause the quorum is 
satisfied with 2 reads). What will happen than is that after a couple of 
minutes one of the other two nodes will go in 100% io wait and will drop most 
of its read messages. Leaving it practically dead while the other 2 nodes keep 
responding at an average of ~10ms. The node that died was only a little slower 
~13ms average but it will inevitably queue up messages. Average response time 
increases to timeout (10 secs) flat. It never recovers.

It happened all the time. And it wasn't the same node that would die.

The solution was that I return the connection to the pool and get a new one for 
every read to balance the load on the client side.

Obviously this will not happen in a cluster where the percentage of all rows on 
one node is enough. But the same thing will probably happen if you scan by 
continuos tokens (meaning that you will read from the same node a long time).

Cheers,

Daniel Doubleday
smeet.com, Berlin







Re: High BloomFilterFalseRation

2010-11-02 Thread Daniel Doubleday
Hi all

had some time yesterday to dig a lil deeper. And maybe this saves someone who 
made the same mistake the time so ...

After trying to reproduce the problem in unit tests with the same data which 
led nowhere because every single result was almost exactly what the math 
promised and incidentally stumbling upon this one: 
http://sites.google.com/site/murmurhash/murmurhash2flaw thinking omg all is 
lost ... I finally found that everything is just fine.

Turns out that the jmx BloomFilterFalseRation simply does not show what I 
expected it to be. I thought it would provide a quality measure how good the 
bloom filter works in terms of hit rate. Which would be (Unnecessary File 
Lookups / Total Lookups) but it is ( False Positives / ( False + True 
Positives) ) which means it does not count all hits that where rejected by the 
filter.

So if you would only ask for rows that do not exist this ration will always 
show 1.0

Meaning it is rather a measure of how many of your queries ask for non existing 
values.

Cheers,
Daniel
 

On Oct 28, 2010, at 1:10 PM, Daniel Doubleday wrote:

 Hi Ryan
 
 I took a sample of one sstable (just flushed, not compacted). 
 
 I compared 2 samples of sstables. One that is showing fine false positive 
 ratios and the problem one. 
 And yes both look the same to me. Both have the expected 15 buckets per row 
 and the cardinality of the bitsets are the same.
 
 But I am pretty sure that it is indeed as suggested a problem with skewed 
 query pattern. I stopped the import and started a random read test and things 
 look better.
 
 I'll try to reproduce this with a patched cassandra to get more debug info to 
 figure out why this is happening. Because I still don't understand it.
 
 Thanks for your time everyone
 
 == Sample of problem CD ==
 
 DATA FILE
 
 file size: 68804626 bytes
 rows: 7432 
 
 FILTER FILE
 
 file size: 14013 bytes
 bloom filter bitset size: 111488
 bloom filter bitset cardinalaity: 54062
 
 
 == Sample of working CF ==
 
 DATA FILE
 
 file size: 110730565 bytes
 rows: 47432
 
 FILTER FILE
 
 file size: 96565 bytes
 bloom filter bitset size: 771904
 bloom filter bitset cardinalaity: 354610
 
 
 On Oct 27, 2010, at 6:41 PM, Ryan King wrote:
 
 On Wed, Oct 27, 2010 at 3:24 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Hi people
 
 We are currently moving our second use case from mysql to cassandra. While 
 importing the data (ongoing) I noticed that the BloomFilterFalseRation 
 seems to be pretty high compared to another CF which is in used in 
 production right now.
 
 Its a hierarchical data model and I cannot avoid to do a read before 
 inserting multiple columns.
 
 I see a false positive ration of 0.28 while in my other CF it is 0.00025.
 
 The CF has 5 live sstables whiel I read that ratio. At that time I inserted 
 ~ 200k rows with a total of 1M cols. Row keys are pretty large 
 unfortunately (key.length() ~ 60)
 
 Just wanted to check if this value is to be expected.
 
 This is not expected. How big are the bloom filters on disk?
 
 -ryan
 



High BloomFilterFalseRation

2010-10-27 Thread Daniel Doubleday
Hi people

We are currently moving our second use case from mysql to cassandra. While 
importing the data (ongoing) I noticed that the BloomFilterFalseRation seems to 
be pretty high compared to another CF which is in used in production right now.

Its a hierarchical data model and I cannot avoid to do a read before inserting 
multiple columns.
 
I see a false positive ration of 0.28 while in my other CF it is 0.00025.

The CF has 5 live sstables whiel I read that ratio. At that time I inserted ~ 
200k rows with a total of 1M cols. Row keys are pretty large unfortunately 
(key.length() ~ 60)

Just wanted to check if this value is to be expected. 



Thanks,
Daniel

Re: High BloomFilterFalseRation

2010-10-27 Thread Daniel Doubleday
Hm -

not sure if I understand the random question. We are using RP. But I wouldn't 
know why that should matter.
I thought that the bloom filter hash function should evenly distribute no 
matter what keys come in.
 
Keys are '/' separated strings (aka paths :-))

I do bulk inserts like: (1000 rows at a time, with ~ 50 cols each)

[
{'a/b/foo': cols},
{'a/b/bar': cols},
{'a/b/baz': cols}
]

and before that I would query for 'a/b'. Recursively as in mkdir -p

If parent paths are missing they would be inserted with the bulk insert.

The value for BloomFilterFalseRatio has been in the range of 0.19 - 0.59 in the 
last couple of hours. Mostly around 0.3

We're on 0.6.6 btw


On Oct 27, 2010, at 3:58 PM, Jonathan Ellis wrote:

 This is not expected, no.  How random are your queries?  If you have a
 couple outlier rows causing the false positives that are being queried
 over and over then that could just be the luck of the draw.
 
 On Wed, Oct 27, 2010 at 5:24 AM, Daniel Doubleday
 daniel.double...@gmx.net wrote:
 Hi people
 
 We are currently moving our second use case from mysql to cassandra. While 
 importing the data (ongoing) I noticed that the BloomFilterFalseRation seems 
 to be pretty high compared to another CF which is in used in production 
 right now.
 
 Its a hierarchical data model and I cannot avoid to do a read before 
 inserting multiple columns.
 
 I see a false positive ration of 0.28 while in my other CF it is 0.00025.
 
 The CF has 5 live sstables whiel I read that ratio. At that time I inserted 
 ~ 200k rows with a total of 1M cols. Row keys are pretty large unfortunately 
 (key.length() ~ 60)
 
 Just wanted to check if this value is to be expected.
 
 
 
 Thanks,
 Daniel
 
 
 
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com



Re: High BloomFilterFalseRation

2010-10-27 Thread Daniel Doubleday

Ah of course - question makes total sense.

But no: this is not the case: I am not constantly asking the same 
question since the tree is deep enough. Most data nodes are level 5 from 
the root. So the parents getting queried will be different most of the time.


Since the parent nodes are created the queries stop there and don't 
propagate toward the root.


And I am seeing the high values all the time. Best that it gets is 0.15.

Daniel

On 27.10.10 18:37, Mike Malone wrote:
I think he was asking about queries, not data. The data may be 
randomly distributed by way of a hash on the key, but if your queries 
are heavily skewed (e.g., if you query for foo a lot more than 
foo/bar, and foo randomly happens to trigger a false positive) the 
skew in your query pattern could cause a seemingly strange spike in 
false positives.


With a hierarchical data model it's not unlikely that this sort of 
skew exists since you'd tend to query for items towards the root of 
the hierarchy more frequently.


Mike

On Wed, Oct 27, 2010 at 2:14 PM, Daniel Doubleday 
daniel.double...@gmx.net mailto:daniel.double...@gmx.net wrote:


Hm -

not sure if I understand the random question. We are using RP. But
I wouldn't know why that should matter.
I thought that the bloom filter hash function should evenly
distribute no matter what keys come in.

Keys are '/' separated strings (aka paths :-))

I do bulk inserts like: (1000 rows at a time, with ~ 50 cols each)

[
   {'a/b/foo': cols},
   {'a/b/bar': cols},
   {'a/b/baz': cols}
]

and before that I would query for 'a/b'. Recursively as in mkdir -p

If parent paths are missing they would be inserted with the bulk
insert.

The value for BloomFilterFalseRatio has been in the range of 0.19
- 0.59 in the last couple of hours. Mostly around 0.3

We're on 0.6.6 btw


On Oct 27, 2010, at 3:58 PM, Jonathan Ellis wrote:

 This is not expected, no.  How random are your queries?  If you
have a
 couple outlier rows causing the false positives that are being
queried
 over and over then that could just be the luck of the draw.

 On Wed, Oct 27, 2010 at 5:24 AM, Daniel Doubleday
 daniel.double...@gmx.net mailto:daniel.double...@gmx.net wrote:
 Hi people

 We are currently moving our second use case from mysql to
cassandra. While importing the data (ongoing) I noticed that the
BloomFilterFalseRation seems to be pretty high compared to another
CF which is in used in production right now.

 Its a hierarchical data model and I cannot avoid to do a read
before inserting multiple columns.

 I see a false positive ration of 0.28 while in my other CF it
is 0.00025.

 The CF has 5 live sstables whiel I read that ratio. At that
time I inserted ~ 200k rows with a total of 1M cols. Row keys are
pretty large unfortunately (key.length() ~ 60)

 Just wanted to check if this value is to be expected.



 Thanks,
 Daniel



 --
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com






Re: nodetool repair

2010-09-22 Thread Daniel Doubleday
Hi all,
just wanted to make sure that I get this right:
What this means is that I have to schedule repairs only on every RFs node?
So with 4 nodes and RF=2 I would repair nodes 1 and 3
and with 6 nodes and RF=3 I would repair nodes 1 and 4
and that would lead to a synched cluster?

 On Thu, Jul 15, 2010 at 1:54 PM, B. Todd Burruss bburr...@real.com wrote:
  if i have N=3 and run nodetool repair on node X.  i assume that merkle
  trees (at a minimum) are calculated on nodes X, X+1, and X+2 (since
  N=3).  when the repair is finished are nodes X, X+1, and X+2 all in sync
  with respect to node X's data?
 
 yes.
 
 -- 
 Jonathan Ellis
 Project Chair, Apache Cassandra
 co-founder of Riptano, the source for professional Cassandra support
 http://riptano.com



Read before Write

2010-08-27 Thread Daniel Doubleday
Hi people

I was wondering if anyone already benchmarked such a situation:

I have:

day of year (row key) - SomeId (column key) - byte[0]

I need to make sure that I write SomeId, but in around 80% of the cases it will 
be already present (so I would essentially replace it with itself). RF will be 
2.

So should I rather just write all the time (given that cassandra is so fast on 
write) or should I read and write only if not present?

Cheers,
Daniel