>  So, one column represents a file in that directory and it has no value.
Just so I understand, the file contents are *not* stored in the column value ?

> Basically the heap fills up and if several queries happens simultaneously, 
> the heap is exhausted and the node stops.
Are you seeing the GCInspector log messages ? Are they ParNew or CMS 
compactions?
If you want to get more insight into what the JVM is doing enable the GC 
logging options in cassandra-env.sh. 

> Dumping the SSTables shows that there were a lot of tombstones between those 
> 2 columns.
How many is a lot ?

>  Normally I run with a 8GB heap and have no problems, but problematic queries 
> can fill up the heap even if I bump it up to 24GB. The machines have 32GB.
For queries like this it's (usually) not the overall size of the JVM heap, Xmx.
It's the size of the NEW_HEAP (in cassandra-env.sh) which sets Xmn. And the 
other new heap settings, SurvivorRatio and MaxTenuringThreshold. What settings 
do you have for those ?

> Of course, the problem goes away after gc_grace_seconds pass and I run a 
> manual compact on that CF, the tombstones are removed and queries to that row 
> are efficient again.
If you have a CF that has a high number of overwrites or deletions using 
Levelled Compaction can help. It does use up some more IO that sized tiered but 
it's designed for these sorts of situations. See 
http://www.datastax.com/dev/blog/leveled-compaction-in-apache-cassandra and 
http://www.datastax.com/dev/blog/when-to-use-leveled-compaction

Schema wise, you could try have multiple "directory" rows for each user. At 
certain times you can create a new row, which then receives all the writes. But 
you read (and delete if necessary) from all rows. Then migrate the data from 
the old rows to the new one and remove the old row.

Cheers


-----------------
Aaron Morton
Freelance Cassandra Developer
New Zealand

@aaronmorton
http://www.thelastpickle.com

On 11/01/2013, at 12:37 AM, André Cruz <andre.c...@co.sapo.pt> wrote:

> Hello.
> 
> I have a schema to represent a filesystem for my users. In this schema one of 
> the CF stores a directory listing this way:
> 
> CF DirList
> 
>   Dir1:    
>         File1:NOVAL File2:NOVAL ...
> 
> So, one column represents a file in that directory and it has no value. The 
> file metadata is stored elsewhere. When listing the contents of a directory I 
> fetch the row contents in batches (using pycassa's column_count and 
> column_start) and always limit the number of columns that I want returned, so 
> as not to occupy too much memory on the Cassandra server. However, if a 
> certain user has deleted a lot of files in that dir and so has a lot of 
> tombstones, even fetching with a column_count of 2 can pose problems to the 
> Cassandra server. Basically the heap fills up and if several queries happens 
> simultaneously, the heap is exhausted and the node stops. Dumping the 
> SSTables shows that there were a lot of tombstones between those 2 columns.
> 
> Is there anything, other than schema changes or throttling on the application 
> side, than I can do to prevent problems like these? Basically I would like 
> Cassandra to stop a query if the resultset already has X items whether they 
> are tombstones or not, and return an error. Or maybe it can stop if the 
> resultset already occupies more then Y bytes or the heap is almost full. Some 
> safety valve to prevent a DoS.
> 
> I should point out that I am using 1.1.5, but I have not seen anything in the 
> changelog that may reference this issue or more recent releases. Normally I 
> run with a 8GB heap and have no problems, but problematic queries can fill up 
> the heap even if I bump it up to 24GB. The machines have 32GB.
> 
> Of course, the problem goes away after gc_grace_seconds pass and I run a 
> manual compact on that CF, the tombstones are removed and queries to that row 
> are efficient again.
> 
> Thanks,
> André Cruz

Reply via email to