incubator-cassandra-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From aaron morton <aa...@thelastpickle.com>
Subject Re: Collecting of tombstones columns during read query fills up heap
Date Tue, 15 Jan 2013 03:55:37 GMT
>> Just so I understand, the file contents are *not* stored in the column value ?
> 
> No, on that particular CF the columns are SuperColumns with 5 sub columns (size, is_dir,
hash, name, revision). Each super column is small, I didn't mention super columns before because
they don't seem to be related to the problem at hand.
I strongly recommend you not use super columns. While they are still supported they are not
accessible via CQL and perform poorly compared to standard CF's. 

Still confused, are the contents of a file stored in cassandra ? If you are storing the contents
in separating them from the meta data will make the operations on the meta data much faster.


> Millions. I have dumped the SSTables to JSON, but have yet to figure out a way to parse
and obtain more information like an exact number since the files are so big.
That's too many. 
You'll need to look at some of the previous suggestion for schema or compaction strategy changes.


> For example, using your suggestion, it would imply some sort of synchronisation whenever
an operation decided that a new row should be written otherwise I could lose some updates.

I think you can do it without synchronisation or ZK. 

Cheers

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

@aaronmorton
http://www.thelastpickle.com

On 11/01/2013, at 12:10 PM, André Cruz <andre.cruz@co.sapo.pt> wrote:

> On Jan 10, 2013, at 8:01 PM, aaron morton <aaron@thelastpickle.com> wrote:
> 
>>>  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 ?
> 
> No, on that particular CF the columns are SuperColumns with 5 sub columns (size, is_dir,
hash, name, revision). Each super column is small, I didn't mention super columns before because
they don't seem to be related to the problem at hand.
> 
>>> 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. 
> 
> I see a lot of messages regarding SliceQueryFilter:
> 
> DEBUG [ReadStage:53] 2013-01-08 18:08:36,451 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(edbc633e-3f09-11e2-8f7d-e0db55018fa4 -delete at 1357508861622915- [hash:false:36@1354732265022159,is_dir:false:1@1354732265022159,mtime:false:4@1354732265022159,name:false:57@1354732265022159,revision:false:16@1354732265022159,])
> DEBUG [ReadStage:62] 2013-01-08 18:08:36,467 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(75869f16-3f0d-11e2-a935-e0db550199f4 -delete at 1357543298946499- [hash:false:36@1354733781339045,is_dir:false:1@1354733781339045,mtime:false:4@1354733781339045,name:false:56@1354733781339045,revision:false:16@1354733781339045,])
> DEBUG [ReadStage:64] 2013-01-08 18:08:36,449 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(6b3323de-3f0a-11e2-93b7-e0db55018fa4 -delete at 1357543981711099- [hash:false:36@1354732475524213,is_dir:false:1@1354732475524213,mtime:false:4@1354732475524213,name:false:56@1354732475524213,revision:false:16@1354732475524213,])
> DEBUG [ReadStage:51] 2013-01-08 18:08:36,448 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(2e2ccb66-3f0f-11e2-9f34-e0db5501ca40 -delete at 1357548656930340- [hash:false:36@1354734520625161,is_dir:false:1@1354734520625161,mtime:false:4@1354734520625161,name:false:54@1354734520625161,revision:false:16@1354734520625161,])
> DEBUG [ReadStage:62] 2013-01-08 18:08:36,468 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(758c5f3c-3f0d-11e2-a935-e0db550199f4 -delete at 1357543303722497- [hash:false:36@1354733781376479,is_dir:false:1@1354733781376479,mtime:false:4@1354733781376479,name:false:56@1354733781376479,revision:false:16@1354733781376479,])
> DEBUG [ReadStage:61] 2013-01-08 18:08:36,447 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(be15520e-3f08-11e2-843b-e0db550199f4 -delete at 1357508704355097- [hash:false:36@1354731755577230,is_dir:false:1@1354731755577230,mtime:false:4@1354731755577230,name:false:57@1354731755577230,revision:false:16@1354731755577230,])
> DEBUG [ReadStage:52] 2013-01-08 18:08:36,446 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(463b877e-3f0a-11e2-b990-e0db55018fa4 -delete at 1357543038078223- [hash:false:36@1354732413504338,is_dir:false:1@1354732413504338,mtime:false:4@1354732413504338,name:false:57@1354732413504338,revision:false:16@1354732413504338,])
> DEBUG [ReadStage:52] 2013-01-08 18:08:36,471 SliceQueryFilter.java (line 124) collecting
1 of 102: SuperColumn(463ef5c6-3f0a-11e2-b990-e0db55018fa4 -delete at 1357543038078223- [hash:false:36@1354732413523782,is_dir:false:1@1354732413523782,mtime:false:4@1354732413523782,name:false:57@1354732413523782,revision:false:16@1354732413523782,])
> 
> 
> GC related messages:
> 
>  INFO [ScheduledTasks:1] 2013-01-09 12:11:17,554 GCInspector.java (line 122) GC for ParNew:
426 ms for 2 collections, 6138212856 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:11:19,819 GCInspector.java (line 122) GC for ConcurrentMarkSweep:
324 ms for 1 collections, 6136066400 used; max is 8357150720
>  WARN [ScheduledTasks:1] 2013-01-09 12:11:19,820 GCInspector.java (line 145) Heap is
0.7342294767181129 full.  You may need to reduce memtable and/or cache sizes.  Cassandra will
now flush up to the two largest memtables to free up memory.  Adjust flush_largest_memtables_at
threshold in cassandra.yaml if you don't want Cassandra to do this automatically
>  WARN [ScheduledTasks:1] 2013-01-09 12:11:19,821 StorageService.java (line 2855) Flushing
CFS(Keyspace='Disco', ColumnFamily='FilesPerBlock') to relieve memory pressure
>  INFO [ScheduledTasks:1] 2013-01-09 12:11:19,821 ColumnFamilyStore.java (line 659) Enqueuing
flush of Memtable-FilesPerBlock@271892815(3190888/38297827 serialized/live bytes, 24184 ops)
>  INFO [FlushWriter:5] 2013-01-09 12:11:19,822 Memtable.java (line 264) Writing Memtable-FilesPerBlock@271892815(3190888/38297827
serialized/live bytes, 24184 ops)
>  INFO [FlushWriter:5] 2013-01-09 12:11:20,118 Memtable.java (line 305) Completed flushing
/servers/storage/cassandra-data/Disco/FilesPerBlock/Disco-FilesPerBlock-he-3793-Data.db (3414379
bytes) for commitlog position ReplayPosition(segmentId=1357730243521, position=32441486)
>  INFO [ScheduledTasks:1] 2013-01-09 12:11:38,413 GCInspector.java (line 122) GC for ParNew:
536 ms for 2 collections, 7337859152 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:11:39,414 GCInspector.java (line 122) GC for ParNew:
253 ms for 1 collections, 7746234936 used; max is 8357150720
> 
> …
> 
>  INFO [ScheduledTasks:1] 2013-01-09 12:12:16,688 GCInspector.java (line 122) GC for ConcurrentMarkSweep:
1966 ms for 2 collections, 2690214576 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:12:25,035 GCInspector.java (line 122) GC for ParNew:
404 ms for 2 collections, 3385333248 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:12:26,037 GCInspector.java (line 122) GC for ParNew:
211 ms for 1 collections, 3606106264 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:13:12,024 GCInspector.java (line 122) GC for ConcurrentMarkSweep:
1043 ms for 2 collections, 4079166304 used; max is 8357150720
>  INFO [ScheduledTasks:1] 2013-01-09 12:13:17,025 MessagingService.java (line 658) 1 READ
messages dropped in last 5000ms
> 
> After just a few minutes:
> 
>  WARN [ScheduledTasks:1] 2013-01-09 12:14:34,134 GCInspector.java (line 145) Heap is
0.9913292333203247 full.  You may need to reduce memtable and/or cache sizes.  Cassandra will
now flush up to the two largest memtables to free up memory.  Adjust flush_largest_memtables_at
threshold in cassandra.yaml if you don't want Cassandra to do this automatically
>  WARN [ScheduledTasks:1] 2013-01-09 12:14:34,170 StorageService.java (line 2855) Flushing
CFS(Keyspace='Disco', ColumnFamily='NamespaceRevision') to relieve memory pressure
>  INFO [ScheduledTasks:1] 2013-01-09 12:14:34,241 ColumnFamilyStore.java (line 659) Enqueuing
flush of Memtable-NamespaceRevision@1129512169(3597280/17167436 serialized/live bytes, 11918
ops)
>  INFO [FlushWriter:6] 2013-01-09 12:14:34,243 Memtable.java (line 264) Writing Memtable-NamespaceRevision@1129512169(3597280/17167436
serialized/live bytes, 11918 ops)
> 
>> 
>>> Dumping the SSTables shows that there were a lot of tombstones between those
2 columns.
>> How many is a lot ?
> 
> Millions. I have dumped the SSTables to JSON, but have yet to figure out a way to parse
and obtain more information like an exact number since the files are so big.
> 
> 
>>>  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
?
> 
> Well, as a matter of fact I tried to increase the heap to 16GB, and then I dumped it
and these were the statistics:
> 
> Heap
>  par new generation   total 737280K, used 656554K [0x00000003fae00000, 0x000000042ce00000,
0x000000042ce00000)
>   eden space 655360K, 100% used [0x00000003fae00000, 0x0000000422e00000, 0x0000000422e00000)
>   from space 81920K,   1% used [0x0000000427e00000, 0x0000000427f2aa98, 0x000000042ce00000)
>   to   space 81920K,   0% used [0x0000000422e00000, 0x0000000422e00000, 0x0000000427e00000)
>  concurrent mark-sweep generation total 15958016K, used 15958015K [0x000000042ce00000,
0x00000007fae00000, 0x00000007fae00000)
>  concurrent-mark-sweep perm gen total 43068K, used 25708K [0x00000007fae00000, 0x00000007fd80f000,
0x0000000800000000)
> 
> 
> These are the relevant JVM settings:
> 
> -XX:+UseThreadPriorities 
> -XX:ThreadPriorityPolicy=42 
> -Xms8049M 
> -Xmx8049M 
> -Xmn800M 
> -XX:+HeapDumpOnOutOfMemoryError 
> -Xss196k 
> -XX:+UseParNewGC 
> -XX:+UseConcMarkSweepGC 
> -XX:+CMSParallelRemarkEnabled 
> -XX:SurvivorRatio=8 
> -XX:MaxTenuringThreshold=1 
> -XX:CMSInitiatingOccupancyFraction=75 
> -XX:+UseCMSInitiatingOccupancyOnly 
> -Djava.net.preferIPv4Stack=true 
> 
>> 
>>> 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
> 
> I'm assuming you mean that tombstones are purged quicker after gc_grace_period using
Levelled compaction, right? The problem remains before gc_grace_period occurs. I'll take a
look at that again, the first time I've looked I was worried that too many files would be
created per-CF. 10MB per stable seems too little, what real-world values are used for that
setting for large (20GB+) CFs?
> 
>> 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.
> 
> I still have not delved into different schemas, this one seems simple enough that Cassandra
should handle it. For example, using your suggestion, it would imply some sort of synchronisation
whenever an operation decided that a new row should be written otherwise I could lose some
updates. I guess I could use Zookeeper for that but it seems over-engineering.
> 
> Thanks,
> André


Mime
View raw message