Some thoughts…

> non replicated Key Space
Not sure what you mean here. Do you mean RF 1 ? I would consider using 3. 
Consider what happens you want to install a rolling upgrade to the cluster. 

> single Column Family, where key is session ID and each column within row 
> stores single key/value - (Map<String,Set<String,String>>)

Consider storing the session data as a single blob in a single column, it will 
reduce the memory and disk overhead and run a bit faster. Assuming the blobs 
are not too big.

> write CL = ONE
> read CL = ONE
Consider testing at QUORUM and then use ONE if you think it helps with your 
availability requirements. 

> 2.000 writes/s
> 5.000 reads/s
Fine and dandy. If you really want to squeeze the most out of the reads go down 
the netflix path and use the external Memcache row cache provider. So yo can 
have reads out of a very large cache outside of the JVM, have cassandra persist 
the data. 

With 3 reasonably spec'd machines I would guess this throughput is achievable 
without too much tuning. Depending on how big the working set is. 

> In this case consistency is not a problem, but the performance could be, 
> especially disk IO.
> 

Wait and see, but if you can disable the commit log or use a longer periodic 
sync. Of course the simple solution is add more machines. 

> If column expires in Memtable before flushing it to SSTable, will Cassandra 
> anyway store such column in SSTable (flush it to HDD)?
Yes, for technical reasons they need to hit the disk. Otherwise the column 
instance will not be used when reconciling against other copies of the column 
already on disk. 

> Each CF hat max 10 columns. In such case I would enable row cache and disable 
> key cache. But I am expecting my data to be still available in Memtable, in 
> this case I could disable whole cache, right?
Keep the row cache. Now days (0.8 sort of and 1.0 definitely) there is no way 
to control how long data stays in the memtable table. This is a good thing as 
you will get it wrong. 
 
> Any Cassandra configuration hints for such session-store use case would be 
> really appreciated :)
Inline above. It is important to understand is how big the working set may be, 
basically estimate concurrent users * session size. Do some tests and don't 
bother tuning until they show you need to. 

Have fun. 

-----------------
Aaron Morton
Freelance Cassandra Developer
@aaronmorton
http://www.thelastpickle.com

On 11/10/2011, at 11:49 PM, Maciej Miklas wrote:

> Hi *,
> 
> I would like to use Cassandra to store session related informations. I do not 
> have real HTTP session - it's different protocol, but the same concept.
> 
> Memcached would be fine, but I would like to additionally persist data.
> 
> Cassandra setup:
> 
> non replicated Key Space
> single Column Family, where key is session ID and each column within row 
> stores single key/value - (Map<String,Set<String,String>>)
> column TTL = 10 minutes
> write CL = ONE
> read CL = ONE
> 2.000 writes/s
> 5.000 reads/s
> Data example:
> 
> session1:{ // CF row key
>    {prop1:val1, TTL:10 min},
>    {prop2:val2, TTL:10 min},
> .....
>    {propXXX:val3, TTL:10 min}
> },
> session2:{ // CF row key
>    {prop1:val1, TTL:10 min},
>    {prop2:val2, TTL:10 min},
> },
> ......
> sessionXXXX:{ // CF row key
>    {prop1:val1, TTL:10 min},
>    {prop2:val2, TTL:10 min},
> }
> In this case consistency is not a problem, but the performance could be, 
> especially disk IO.
> 
> Since data in my session leaves for short time, I would like to avoid storing 
> it on hard drive - except for commit log.
> 
> I have some questions:
> 
> If column expires in Memtable before flushing it to SSTable, will Cassandra 
> anyway store such column in SSTable (flush it to HDD)?
> Replication is disabled for my Key Space, in this case storing such expired 
> column in SSTable would not be necessary, right?
> Each CF hat max 10 columns. In such case I would enable row cache and disable 
> key cache. But I am expecting my data to be still available in Memtable, in 
> this case I could disable whole cache, right?
> Any Cassandra configuration hints for such session-store use case would be 
> really appreciated :)
> Thank you, 
> 
> Maciej
> 

Reply via email to