Re: Very slow writes in Cassandra

2011-10-31 Thread Adrian Cockcroft
You are using replication factor of one and the Lustre clustered filesystem over the network. Not good practice. Try RF=3 and local disks. Lustre duplicates much of the functionality of Cassandra, there is no point using both. Make your Lustre server nodes into Cassandra nodes instead. Adrian On

[RELEASE] Apache Cassandra 0.7.10 released

2011-10-31 Thread Sylvain Lebresne
The Cassandra team is pleased to announce the release of Apache Cassandra version 0.7.10. Cassandra is a highly scalable second-generation distributed database, bringing together Dynamo's fully distributed design and Bigtable's ColumnFamily-based data model. You can read more here: http://cassan

Re: data model for unique users in a time period

2011-10-31 Thread Ed Anuff
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 wrote: > Ed, > > I could be completely wrong about this working--I haven't specifically > looked at how the counts are executed, but I think th

Re: data model for unique users in a time period

2011-10-31 Thread Zach Richardson
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 r

Re : Best way to search content in Cassandra

2011-10-31 Thread Laurent Aufrechter
Hello, One good way to manage such things is to give your columns a name that will allow you to make some slices query... Your column name could be something like: image-png-other_identifier1 image-gif-other_identifier2 In your slice query, you could do a search for "image-png-A" to "image-png

data model for unique users in a time period

2011-10-31 Thread Ed Anuff
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

Re: Cassandra Cluster Admin - phpMyAdmin for Cassandra

2011-10-31 Thread Ertio Lew
Thanks so much SebWajam for this great piece of work! Is there a way to set a data type for displaying the column names/ values of a CF ? It seems that your project always uses String Serializer for any piece of data however most of the times in real world cases this is not true so can we anyhow

Re: OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Sylvain Lebresne
On Mon, Oct 31, 2011 at 2:58 PM, Sylvain Lebresne wrote: > On Mon, Oct 31, 2011 at 1:10 PM, Mick Semb Wever wrote: >> On Mon, 2011-10-31 at 13:05 +0100, Mick Semb Wever wrote: >>> Given a 60G sstable, even with 64kb chunk_length, to read just that one >>> sstable requires close to 8G free heap me

Re: OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Sylvain Lebresne
On Mon, Oct 31, 2011 at 1:10 PM, Mick Semb Wever wrote: > On Mon, 2011-10-31 at 13:05 +0100, Mick Semb Wever wrote: >> Given a 60G sstable, even with 64kb chunk_length, to read just that one >> sstable requires close to 8G free heap memory... > > Arg, that calculation was a little off... >  (a lon

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Jonathan Ellis
Cleanup would have the same effect I think, in exchange for a minor amount of extra CPU used. On Mon, Oct 31, 2011 at 4:08 AM, Sylvain Lebresne wrote: > On Mon, Oct 31, 2011 at 9:07 AM, Mick Semb Wever wrote: >> On Mon, 2011-10-31 at 08:00 +0100, Mick Semb Wever wrote: >>> After an upgrade to ca

Re: Newbie question - fetching multiple columns of different datatypes and conversion from byte[]

2011-10-31 Thread Ertio Lew
Should the different datatype col values or names be first read as byte buffer & then converted to appropriate type using Hector's provided serializers api like the way shown below ? ByteBuffer bb; .. String s= StringSerializer.get().fromByteBuffer(bb); Or are there any better ways ?

Re: OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
On Mon, 2011-10-31 at 13:05 +0100, Mick Semb Wever wrote: > Given a 60G sstable, even with 64kb chunk_length, to read just that one > sstable requires close to 8G free heap memory... Arg, that calculation was a little off... (a long isn't exactly 8K...) But you get my concern... ~mck -- "Whe

Re: OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
On Mon, 2011-10-31 at 09:07 +0100, Mick Semb Wever wrote: > The read pattern of these rows is always in bulk so the chunk_length > could have been much higher so to reduce memory usage (my largest > sstable is 61G). Isn't CompressionMetadata.readChunkOffsets(..) rather dangerous here? Given a 60

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Sylvain Lebresne
On Mon, Oct 31, 2011 at 11:41 AM, Mick Semb Wever wrote: > On Mon, 2011-10-31 at 10:08 +0100, Sylvain Lebresne wrote: >> you can >> trigger a "user defined compaction" through JMX on each of the sstable >> you want to rebuild. > > May i ask how? > Everything i see from NodeProbe to StorageProxy is

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Sylvain Lebresne
On Mon, Oct 31, 2011 at 11:35 AM, Mick Semb Wever wrote: > On Mon, 2011-10-31 at 10:08 +0100, Sylvain Lebresne wrote: >> >> I set chunk_length_kb to 16 as my rows are very skinny (typically 100b) >> > >> > >> > I see now this was a bad choice. >> > The read pattern of these rows is always in bulk

[RELEASE] Apache Cassandra 1.0.1 released

2011-10-31 Thread Sylvain Lebresne
The Cassandra team is pleased to announce the release of Apache Cassandra version 1.0.1. Cassandra is a highly scalable second-generation distributed database, bringing together Dynamo's fully distributed design and Bigtable's ColumnFamily-based data model. You can read more here: http://cassand

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
On Mon, 2011-10-31 at 10:08 +0100, Sylvain Lebresne wrote: > you can > trigger a "user defined compaction" through JMX on each of the sstable > you want to rebuild. May i ask how? Everything i see from NodeProbe to StorageProxy is ks and cf based. ~mck -- “Anyone who lives within their means s

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
On Mon, 2011-10-31 at 10:08 +0100, Sylvain Lebresne wrote: > >> I set chunk_length_kb to 16 as my rows are very skinny (typically 100b) > > > > > > I see now this was a bad choice. > > The read pattern of these rows is always in bulk so the chunk_length > > could have been much higher so to reduce

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Sylvain Lebresne
On Mon, Oct 31, 2011 at 9:07 AM, Mick Semb Wever wrote: > On Mon, 2011-10-31 at 08:00 +0100, Mick Semb Wever wrote: >> After an upgrade to cassandra-1.0 any get_range_slices gives me: >> >> java.lang.OutOfMemoryError: Java heap space >>       at >> org.apache.cassandra.io.compress.CompressionMeta

Re: get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
On Mon, 2011-10-31 at 08:00 +0100, Mick Semb Wever wrote: > After an upgrade to cassandra-1.0 any get_range_slices gives me: > > java.lang.OutOfMemoryError: Java heap space > at > org.apache.cassandra.io.compress.CompressionMetadata.readChunkOffsets(CompressionMetadata.java:93) > at

get_range_slices OOM on CompressionMetadata.readChunkOffsets(..)

2011-10-31 Thread Mick Semb Wever
After an upgrade to cassandra-1.0 any get_range_slices gives me: java.lang.OutOfMemoryError: Java heap space at org.apache.cassandra.io.compress.CompressionMetadata.readChunkOffsets(CompressionMetadata.java:93) at org.apache.cassandra.io.compress.CompressionMetadata.(CompressionM