Replication is 3 and read consistency level is one. One of the non-cordinator 
mode is crashing, so the OOM is happening before aggregation of the data to be 
returned.

Thanks for the info about the space allocated to young generation heap. That is 
helpful.

Mohammed

On Jun 25, 2013, at 1:28 PM, "sankalp kohli" 
<kohlisank...@gmail.com<mailto:kohlisank...@gmail.com>> wrote:

Your young gen is 1/4 of 1.8G which is 450MB. Also in slice queries, the 
co-ordinator will get the results from replicas as per consistency level used 
and merge the results before returning to the client.
What is the replication in your keyspace and what consistency you are reading 
with.
Also 55MB on disk will not mean 55MB in memory. The data is compressed on disk 
and also there are other overheads.



On Mon, Jun 24, 2013 at 8:38 PM, Mohammed Guller 
<moham...@glassbeam.com<mailto:moham...@glassbeam.com>> wrote:
No deletes. In my test, I am just writing and reading data.

There is a lot of GC, but only on the younger generation. Cassandra terminates 
before the GC for old generation kicks in.

I know that our queries are reading an unusual amount of data. However, I 
expected it to throw a timeout exception instead of crashing. Also, don't 
understand why 1.8 Gb heap is getting full when the total data stored in the 
entire Cassandra cluster is less than 55 MB.

Mohammed

On Jun 21, 2013, at 7:30 PM, "sankalp kohli" 
<kohlisank...@gmail.com<mailto:kohlisank...@gmail.com>> wrote:

Looks like you are putting lot of pressure on the heap by doing a slice query 
on a large row.
Do you have lot of deletes/tombstone on the rows? That might be causing a 
problem.
Also why are you returning so many columns as once, you can use auto paginate 
feature in Astyanax.

Also do you see lot of GC happening?


On Fri, Jun 21, 2013 at 1:13 PM, Jabbar Azam 
<aja...@gmail.com<mailto:aja...@gmail.com>> wrote:
Hello Mohammed,

You should increase the heap space. You should also tune the garbage collection 
so young generation objects are collected faster, relieving pressure on heap We 
have been using jdk 7 and it uses G1 as the default collector. It does a better 
job than me trying to optimise the JDK 6 GC collectors.

Bear in mind though that the OS will need memory, so will the row cache and the 
filing system. Although memory usage will depend on the workload of your system.

I'm sure you'll also get good advice from other members of the mailing list.

Thanks

Jabbar Azam


On 21 June 2013 18:49, Mohammed Guller 
<moham...@glassbeam.com<mailto:moham...@glassbeam.com>> wrote:
We have a 3-node cassandra cluster on AWS. These nodes are running cassandra 
1.2.2 and have 8GB memory. We didn't change any of the default heap or GC 
settings. So each node is allocating 1.8GB of heap space. The rows are wide; 
each row stores around 260,000 columns. We are reading the data using Astyanax. 
If our application tries to read 80,000 columns each from 10 or more rows at 
the same time, some of the nodes run out of heap space and terminate with OOM 
error. Here is the error message:

java.lang.OutOfMemoryError: Java heap space
        at java.nio.HeapByteBuffer.duplicate(HeapByteBuffer.java:107)
        at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:50)
        at 
org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:60)
        at 
org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:126)
        at 
org.apache.cassandra.db.filter.ColumnCounter$GroupByPrefix.count(ColumnCounter.java:96)
        at 
org.apache.cassandra.db.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:164)
        at 
org.apache.cassandra.db.filter.QueryFilter.collateColumns(QueryFilter.java:136)
        at 
org.apache.cassandra.db.filter.QueryFilter.collateOnDiskAtom(QueryFilter.java:84)
        at 
org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:294)
        at 
org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:65)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1363)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1220)
        at 
org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1132)
        at org.apache.cassandra.db.Table.getRow(Table.java:355)
        at 
org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:70)
       at 
org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1052)
        at 
org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1578)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:722)

ERROR 02:14:05,351 Exception in thread Thread[Thrift:6,5,main]
java.lang.OutOfMemoryError: Java heap space
        at java.lang.Long.toString(Long.java:269)
        at java.lang.Long.toString(Long.java:764)
        at 
org.apache.cassandra.dht.Murmur3Partitioner$1.toString(Murmur3Partitioner.java:171)
        at 
org.apache.cassandra.service.StorageService.describeRing(StorageService.java:1068)
        at 
org.apache.cassandra.thrift.CassandraServer.describe_ring(CassandraServer.java:1192)
        at 
org.apache.cassandra.thrift.Cassandra$Processor$describe_ring.getResult(Cassandra.java:3766)
        at 
org.apache.cassandra.thrift.Cassandra$Processor$describe_ring.getResult(Cassandra.java:3754)
        at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:32)
        at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:34)
        at 
org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:199)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
        at java.lang.Thread.run(Thread.java:722)

The data in each column is less than 50 bytes. After adding all the column 
overheads (column name + metadata), it should not be more than 100 bytes. So 
reading 80,000 columns from 10 rows each means that we are reading 80,000 * 10 
* 100 = 80 MB of data. It is large, but not large enough to fill up the 1.8 GB 
heap. So I wonder why the heap is getting full. If the data request is too big 
to fill in a reasonable amount of time, I would expect Cassandra to return a 
TimeOutException instead of terminating.

One easy solution is to increase the heapsize. However that means Cassandra can 
still crash if someone reads 100 rows.  I wonder if there some other Cassandra 
setting that I can tweak to prevent the OOM exception?

Thanks,
Mohammed



Reply via email to