Re: Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?

2013-07-15 Thread sulong
Yes, that's what I am looking for. Thanks.


On Mon, Jul 15, 2013 at 10:08 PM, Jake Luciani  wrote:

> Take a look at https://issues.apache.org/jira/browse/CASSANDRA-5661
>
>
> On Mon, Jul 15, 2013 at 4:18 AM, sulong  wrote:
>
>> Thanks for your help. Yes, I will try to increase the sstable size. I
>> hope it can save me.
>>
>> 9000 SSTableReader x 10 RandomAccessReader x 64Kb = 5.6G memory. If there
>> is only one RandomAccessReader, the memory will be 9000 * 1 * 64Kb = 0.56G
>> . Looks great. But I think it must be reasonable to recycle the
>> RandomAccessReader.
>>
>>
>> On Mon, Jul 15, 2013 at 4:02 PM, Janne Jalkanen > > wrote:
>>
>>>
>>> I had exactly the same problem, so I increased the sstable size (from 5
>>> to 50 MB - the default 5MB is most certainly too low for serious usecases).
>>>  Now the number of SSTableReader objects is manageable, and my heap is
>>> happier.
>>>
>>> Note that for immediate effect I stopped the node, removed the *.json
>>> files and restarted - which put all SSTables to L0, which meant a weekend
>>> full of compactions… Would be really cool if there was a way to
>>> automatically drop all LCS SSTables one level down to make them compact
>>> earlier without avoiding the
>>> "OMG-must-compact-everything-aargh-my-L0-is-full" -effect of removing the
>>> JSON file.
>>>
>>> /Janne
>>>
>>> On 15 Jul 2013, at 10:48, sulong  wrote:
>>>
>>> > Why does cassandra PoolingSegmentedFile recycle the
>>> RandomAccessReader? The RandomAccessReader objects consums too much memory.
>>> >
>>> > I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap.
>>> The cassandra's memory is full after about one month, so I have to restart
>>> the 4 nodes every month.
>>> >
>>> > I have 100G data on every node, with LevedCompactionStrategy and 10M
>>> sstable size, so there are more than 1 sstable files. By looking
>>> through the heap dump file, I see there are more than 9000 SSTableReader
>>> objects in memory, which references lots of  RandomAccessReader objects.
>>> The memory is consumed by these RandomAccessReader objects.
>>> >
>>> > I see the PoolingSegementedFile has a recycle method, which puts the
>>> RandomAccessReader to a queue. Looks like the Queue always grow until the
>>> sstable is compacted.  Is there any way to stop the RandomAccessReader
>>> recycling? Or, set a limit to the recycled RandomAccessReader's number?
>>> >
>>> >
>>>
>>>
>>
>
>
> --
> http://twitter.com/tjake
>


Re: Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?

2013-07-15 Thread Jake Luciani
Take a look at https://issues.apache.org/jira/browse/CASSANDRA-5661


On Mon, Jul 15, 2013 at 4:18 AM, sulong  wrote:

> Thanks for your help. Yes, I will try to increase the sstable size. I hope
> it can save me.
>
> 9000 SSTableReader x 10 RandomAccessReader x 64Kb = 5.6G memory. If there
> is only one RandomAccessReader, the memory will be 9000 * 1 * 64Kb = 0.56G
> . Looks great. But I think it must be reasonable to recycle the
> RandomAccessReader.
>
>
> On Mon, Jul 15, 2013 at 4:02 PM, Janne Jalkanen 
> wrote:
>
>>
>> I had exactly the same problem, so I increased the sstable size (from 5
>> to 50 MB - the default 5MB is most certainly too low for serious usecases).
>>  Now the number of SSTableReader objects is manageable, and my heap is
>> happier.
>>
>> Note that for immediate effect I stopped the node, removed the *.json
>> files and restarted - which put all SSTables to L0, which meant a weekend
>> full of compactions… Would be really cool if there was a way to
>> automatically drop all LCS SSTables one level down to make them compact
>> earlier without avoiding the
>> "OMG-must-compact-everything-aargh-my-L0-is-full" -effect of removing the
>> JSON file.
>>
>> /Janne
>>
>> On 15 Jul 2013, at 10:48, sulong  wrote:
>>
>> > Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?
>> The RandomAccessReader objects consums too much memory.
>> >
>> > I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap.
>> The cassandra's memory is full after about one month, so I have to restart
>> the 4 nodes every month.
>> >
>> > I have 100G data on every node, with LevedCompactionStrategy and 10M
>> sstable size, so there are more than 1 sstable files. By looking
>> through the heap dump file, I see there are more than 9000 SSTableReader
>> objects in memory, which references lots of  RandomAccessReader objects.
>> The memory is consumed by these RandomAccessReader objects.
>> >
>> > I see the PoolingSegementedFile has a recycle method, which puts the
>> RandomAccessReader to a queue. Looks like the Queue always grow until the
>> sstable is compacted.  Is there any way to stop the RandomAccessReader
>> recycling? Or, set a limit to the recycled RandomAccessReader's number?
>> >
>> >
>>
>>
>


-- 
http://twitter.com/tjake


Re: Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?

2013-07-15 Thread sulong
Thanks for your help. Yes, I will try to increase the sstable size. I hope
it can save me.

9000 SSTableReader x 10 RandomAccessReader x 64Kb = 5.6G memory. If there
is only one RandomAccessReader, the memory will be 9000 * 1 * 64Kb = 0.56G
. Looks great. But I think it must be reasonable to recycle the
RandomAccessReader.


On Mon, Jul 15, 2013 at 4:02 PM, Janne Jalkanen wrote:

>
> I had exactly the same problem, so I increased the sstable size (from 5 to
> 50 MB - the default 5MB is most certainly too low for serious usecases).
>  Now the number of SSTableReader objects is manageable, and my heap is
> happier.
>
> Note that for immediate effect I stopped the node, removed the *.json
> files and restarted - which put all SSTables to L0, which meant a weekend
> full of compactions… Would be really cool if there was a way to
> automatically drop all LCS SSTables one level down to make them compact
> earlier without avoiding the
> "OMG-must-compact-everything-aargh-my-L0-is-full" -effect of removing the
> JSON file.
>
> /Janne
>
> On 15 Jul 2013, at 10:48, sulong  wrote:
>
> > Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?
> The RandomAccessReader objects consums too much memory.
> >
> > I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap. The
> cassandra's memory is full after about one month, so I have to restart the
> 4 nodes every month.
> >
> > I have 100G data on every node, with LevedCompactionStrategy and 10M
> sstable size, so there are more than 1 sstable files. By looking
> through the heap dump file, I see there are more than 9000 SSTableReader
> objects in memory, which references lots of  RandomAccessReader objects.
> The memory is consumed by these RandomAccessReader objects.
> >
> > I see the PoolingSegementedFile has a recycle method, which puts the
> RandomAccessReader to a queue. Looks like the Queue always grow until the
> sstable is compacted.  Is there any way to stop the RandomAccessReader
> recycling? Or, set a limit to the recycled RandomAccessReader's number?
> >
> >
>
>


Re: Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?

2013-07-15 Thread Janne Jalkanen

I had exactly the same problem, so I increased the sstable size (from 5 to 50 
MB - the default 5MB is most certainly too low for serious usecases).  Now the 
number of SSTableReader objects is manageable, and my heap is happier.

Note that for immediate effect I stopped the node, removed the *.json files and 
restarted - which put all SSTables to L0, which meant a weekend full of 
compactions… Would be really cool if there was a way to automatically drop all 
LCS SSTables one level down to make them compact earlier without avoiding the 
"OMG-must-compact-everything-aargh-my-L0-is-full" -effect of removing the JSON 
file.

/Janne

On 15 Jul 2013, at 10:48, sulong  wrote:

> Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader? The 
> RandomAccessReader objects consums too much memory.
> 
> I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap. The 
> cassandra's memory is full after about one month, so I have to restart the 4 
> nodes every month. 
> 
> I have 100G data on every node, with LevedCompactionStrategy and 10M sstable 
> size, so there are more than 1 sstable files. By looking through the heap 
> dump file, I see there are more than 9000 SSTableReader objects in memory, 
> which references lots of  RandomAccessReader objects. The memory is consumed 
> by these RandomAccessReader objects. 
> 
> I see the PoolingSegementedFile has a recycle method, which puts the 
> RandomAccessReader to a queue. Looks like the Queue always grow until the 
> sstable is compacted.  Is there any way to stop the RandomAccessReader 
> recycling? Or, set a limit to the recycled RandomAccessReader's number?
> 
> 



Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader?

2013-07-15 Thread sulong
Why does cassandra PoolingSegmentedFile recycle the RandomAccessReader? The
RandomAccessReader objects consums too much memory.

I have a cluster of 4 nodes. Every node's cassandra jvm has 8G heap. The
cassandra's memory is full after about one month, so I have to restart the
4 nodes every month.

I have 100G data on every node, with LevedCompactionStrategy and 10M
sstable size, so there are more than 1 sstable files. By looking
through the heap dump file, I see there are more than 9000 SSTableReader
objects in memory, which references lots of  RandomAccessReader objects.
The memory is consumed by these RandomAccessReader objects.

I see the PoolingSegementedFile has a recycle method, which puts the
RandomAccessReader to a queue. Looks like the Queue always grow until the
sstable is compacted.  Is there any way to stop the RandomAccessReader
recycling? Or, set a limit to the recycled RandomAccessReader's number?