You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Maciej Miklas <ma...@googlemail.com> on 2011/10/11 12:49:13 UTC

Cassandra as session store under heavy load

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:

   1. If column expires in Memtable before flushing it to SSTable, will
   Cassandra anyway store such column in SSTable (flush it to HDD)?
   2. Replication is disabled for my Key Space, in this case storing such
   expired column in SSTable would not be necessary, right?
   3. 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?
   4. Any Cassandra configuration hints for such session-store use case
   would be really appreciated :)

Thank you,

Maciej

Re: Cassandra as session store under heavy load

Posted by Jonathan Ellis <jb...@gmail.com>.
Or upgrade to 1.0 and use leveled compaction
(http://www.datastax.com/dev/blog/leveled-compaction-in-apache-cassandra)

On Thu, Oct 13, 2011 at 4:28 PM, aaron morton <aa...@thelastpickle.com> wrote:
> They only have a minimum time, gc_grace_seconds for deletes.
>
> If you want to be really watch disk space reduce the compaction thresholds on the CF.
>
> Or run a major compaction as part of maintenance.
>
> cheers
>
> -----------------
> Aaron Morton
> Freelance Cassandra Developer
> @aaronmorton
> http://www.thelastpickle.com
>
> On 13/10/2011, at 10:50 PM, Maciej Miklas wrote:
>
>> durable_writes sounds great - thank you! I really do not need commit log here.
>>
>> Another question: it is possible to configure live time of Tombstones?
>>
>>
>> Regards,
>> Maciej
>
>



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

Re: Cassandra as session store under heavy load

Posted by aaron morton <aa...@thelastpickle.com>.
They only have a minimum time, gc_grace_seconds for deletes.

If you want to be really watch disk space reduce the compaction thresholds on the CF. 

Or run a major compaction as part of maintenance. 

cheers

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

On 13/10/2011, at 10:50 PM, Maciej Miklas wrote:

> durable_writes sounds great - thank you! I really do not need commit log here.
> 
> Another question: it is possible to configure live time of Tombstones? 
> 
> 
> Regards,
> Maciej


Re: Cassandra as session store under heavy load

Posted by Maciej Miklas <ma...@googlemail.com>.
durable_writes sounds great - thank you! I really do not need commit log
here.

Another question: it is possible to configure live time of Tombstones?


Regards,
Maciej

Re: Cassandra as session store under heavy load

Posted by aaron morton <aa...@thelastpickle.com>.
> - Serializing data is not an option, because I would like to have possibility to access data using console
fair enough, but I would do some tests to see the difference in performance and disk space

> - Using Cassandra to build something like "HTTP session store" with short TTL is not an anti-pattern ?
Heavy delete work loads, such as a queue, is more of a problem. You may have to may some extra attention to disk space, because it not clean up instantly. But I would not go so far to say anti-pattern.

- There is really no way to tell Cassandra, that particular Key Space should be stored "mostly" in RAM and only asynchronous backup on HDD (JMS has something like that)?

See the durable_writes option for create keyspace in the CLI. Turn if off and the commit log is not used, writes will only be committed to disk when the memtable is flushed. 

Have fun. 


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

On 12/10/2011, at 7:28 PM, Maciej Miklas wrote:

> - RF is 1. We have few KeySpaces, only this one is not replicated - this data is not that very important. In case of error customer will have to execute process again. But again, I would like to persist it.
> - Serializing data is not an option, because I would like to have possibility to access data using console
> - I will keep row cache - you are right, there is no guarantee, that my data is still in Memtable
> 
> I will get my hardware soon (3 servers) and we will see ;) In this worst case I will switch my session storage to memcached, and leave all other data in Cassandra (no TTL, or very long)
> 
> Another questions:
> - Using Cassandra to build something like "HTTP session store" with short TTL is not an anti-pattern ?
> - There is really no way to tell Cassandra, that particular Key Space should be stored "mostly" in RAM and only asynchronous backup on HDD (JMS has something like that)?
> 
> 
> Thanks,
> Maciej
> 


Re: Cassandra as session store under heavy load

Posted by Maciej Miklas <ma...@googlemail.com>.
- RF is 1. We have few KeySpaces, only this one is not replicated - this
data is not that very important. In case of error customer will have to
execute process again. But again, I would like to persist it.
- Serializing data is not an option, because I would like to have
possibility to access data using console
- I will keep row cache - you are right, there is no guarantee, that my data
is still in Memtable

I will get my hardware soon (3 servers) and we will see ;) In this worst
case I will switch my session storage to memcached, and leave all other data
in Cassandra (no TTL, or very long)

Another questions:
- Using Cassandra to build something like "HTTP session store" with short
TTL is not an anti-pattern ?
- There is really no way to tell Cassandra, that particular Key Space should
be stored "mostly" in RAM and only asynchronous backup on HDD (JMS has
something like that)?


Thanks,
Maciej

Re: Cassandra as session store under heavy load

Posted by aaron morton <aa...@thelastpickle.com>.
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
>