You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@cassandra.apache.org by graham sanderson <gr...@vast.com> on 2014/06/15 17:52:49 UTC

Re: CMS GC / fragmentation / memtables etc

Hi Benedict,

So I had a look at the code, and as you say it looked pretty easy to recycle on heap slabs… there is already RACE_ALLOCATED which keeps a strongly referenced pool, however I was thinking in this case of just WeakReferences.

In terms of on heap slabs, it seemed to me that recycling the oldest slab you have is probably the best heuristic, since it is less likely to be in eden (of course re-using in eden is no worse than worst case today), however since the problem tends to be promotion failure of slabs due to fragmentation of old gen, recycling one that is already there is even better - better still if it has been compacted somewhere pretty stable. I think this heuristic would also work well for G1, though I believe the recommendation is still not to use that with cassandra.

So for implementation of that I was thinking of using a ConcurrentSkipListMap, from a Long representing the allocation order of the Region to a weak reference to the Region (just regular 1M sized ones)… allocators can pull oldest and discard cleared references (might need a scrubber if the map got too big and we were only checking the first entry). Beyond that I don’t think there is any need for a configurable-lengthed collection of strongly referenced reusable slabs.

Question 1:

This is easy enough to implement, and probably should just be turned on by an orthogonal setting… I guess on heap slab is the current default, so this feature will be useful

Question 2:

Something similar could be done for off heap slabs… this would seem more like it would want a size limit on the number of re-usable slabs… strong references with explicit clean() is probably better, than using weak-references and letting PhantomReference cleaner on DirectByteBuffer do the cleaning later.

Let me know any thoughts and I’ll open an issue (probably 2 - one for on heap one for off)… let me know whether you’d like me to assign the first to you or me (I couldn’t work on it before next week)

Thanks,

Graham.

On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <be...@datastax.com> wrote:

> Graham,
> 
> This is largely fixed in 2.1 with the introduction of partially off-heap
> memtables - the slabs reside off-heap, so do not cause any GC issues.
> 
> As it happens the changes would also permit us to recycle on-heap slabs
> reasonable easily as well, so feel free to file a ticket for that, although
> it won't be back ported to 2.0.
> 
> 
> On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
> 
>> So i’ve been tinkering a bit with CMS config because we are still seeing
>> fairly frequent full compacting GC due to framgentation/promotion failure
>> 
>> As mentioned below, we are usually too fragmented to promote new in-flight
>> memtables.
>> 
>> This is likely caused by sudden write spikes (which we do have), though
>> actually the problems don’t generally happen at that time of our largest
>> write spikes (though any write spikes likely cause spill of both new
>> memtables along with many other new objects of unknown size into the
>> tenured gen, so they cause fragmentation if not immediate GC issue). We
>> have lots of things going on in this multi-tenant cluster (GC pauses are of
>> course extra bad, since they cause spike in hinted-handoff on other nodes
>> which were already busy etc…)
>> 
>> Anyway, considering possibilities:
>> 
>> 0) Try and make our application behavior more steady state - this is
>> probably possible, but there are lots of other things (e.g. compaction,
>> opscenter, repair etc.) which are both tunable and generally throttle-able
>> to think about too.
>> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
>> (I’d be curious what the “crud” is in particular that is getting spilled -
>> presumably it is larger objects since it affects the binary tree of large
>> objects)
>> 2) Given the above, if we can guarantee even > 24 hours without full GC, I
>> don’t think we’d mind running a regular rolling re-start on the servers
>> during off hours (note usually the GCs don’t have a visible impact, but
>> when they hit multiple machines at once they can)
>> 3) Zing is seriously an option, if it would save us large amounts of
>> tuning, and constant worry about the “next” thing tweaking the allocation
>> patterns - does anyone have any experience with Zing & Cassandra
>> 4) Given that we expect periodic bursts of writes,
>> memtable_total_space_in_mb is bounded, we are not actually short of memory
>> (it just gets fragmented), I’m wondering if anyone has played with pinning
>> (up to or initially?) that many 1MB chunks of memory via SlabAllocator and
>> re-using… It will get promoted once, and then these 1M chunks won’t be part
>> of the subsequent promotion hassle… it will probably also allow more crud
>> to die in eden under write load since we aren’t allocating these large
>> chunks in eden at the same time. Anyway, I had a little look at the code,
>> and the life cycles of memtables is not trivial, but was considering
>> attempting a patch to play with… anyone have any thoughts?
>> 
>> Basically in summary, the Slab allocator helps by allocating and freeing
>> lots of objects at the same time, however any time slabs are allocated
>> under load, we end up promoting them with whatever other live stuff in eden
>> is still there. If we only do this once and reuse the slabs, we are likely
>> to minimize our promotion problem later (at least for these large objects)
>> 
>> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com> wrote:
>> 
>>> Excellent - thank you…
>>> 
>>> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <sa...@urssaf.fr>
>> wrote:
>>> 
>>>> Hi,
>>>> This is arena allocation of memtables. See here for more infos :
>>>> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
>>>> 
>>>> 
>>>> 
>>>> 
>>>> De :    graham sanderson <gr...@vast.com>
>>>> A :     dev@cassandra.apache.org,
>>>> Date :  16/05/2014 14:03
>>>> Objet : Things that are about 1M big
>>>> 
>>>> 
>>>> 
>>>> So just throwing this out there for those for whom this might ring a
>> bell.
>>>> 
>>>> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
>>>> interestingly enough most of the objects giving us promotion failures
>> are
>>>> of size 131074 (dwords) - GC logging obviously doesn?t say what those
>> are,
>>>> but I?d wager money they are either 1M big byte arrays, or less likely
>>>> 256k entry object arrays backing large maps
>>>> 
>>>> So not strictly critical to solving my problem, but I was wondering if
>>>> anyone can think of any heap allocated C* objects which are (with no
>>>> significant changes to standard cassandra config) allocated in 1M
>> chunks.
>>>> (It would save me scouring the code, or a 9 gig heap dump if I need to
>>>> figure it out!)
>>>> 
>>>> Thanks,
>>>> 
>>>> Graham
>>> 
>> 
>> 


Re: CMS GC / fragmentation / memtables etc

Posted by Benedict Elliott Smith <be...@datastax.com>.
The discussion about virtual methods was around a number of considerations,
as there was some work to refactor the hierarchy and introduce more virtual
methods for a single allocator's cell hierarchies. The allocator also does
not determine the total set of possible cell implementations, as in-flight
data always exists as an on-heap buffer for this version of Cassandra, so
your supposition is a bit optimistic I'm afraid. This may change in a
future version, but depends on the zero-copy patch that has, as I
mentioned, been shelved for the moment.

1) offheap_objects refers to the fact that we serialize the whole object
into an offheap region of memory, except for a single object reference to
it for accessing this data; they are still (currently) allocated as regions
of offheap memory, so there should not be fragmentation of the native heap
2) Yes, although obviously they are slightly more experimental so will not
be the default memtable type when 2.1 first drops
3) I doubt there would be much difference in the current implementation for
offheap memtables (possibly for row cache) as we allocate large (1M)
regions of native memory to save space and fragmentation (native
allocations typically have ~16 byte overhead which is significant at the
cell level). Currently we support jemalloc as an alternative allocator, but
I would benchmark this allocator in your environment, as it has very high
deallocation costs in some environments.

Tuning these variables will, as with everything, be highly workload
dependent. A rule of thumb for offheap_objects: heap: 100 bytes per
partition, 20 bytes per cell; offheap: 8 (timestamp) +  8 (overhead) +
{clustering+name+data size} per cell

On a memtable flush you will see information printed about occupancy of
each limit, so you will be able to tune accordingly if your data is
consistent.


On Sun, Jun 15, 2014 at 11:52 PM, graham sanderson <gr...@vast.com> wrote:

> Hi Benedict thanks once again for your explanations (hopefully these are
> my last questions!)… I just read
> https://issues.apache.org/jira/browse/CASSANDRA-6694 really quickly, so
> didn’t follow all the discussions…. one thing to note that was argued in
> the middle was about the cost of virtual methods with multiple
> implementations. c2 certainly has fast path for 1, and N known
> implementations where 1<N<something small (with slow path, possible
> de/re-compile on optimistic assumption failure, or actually when new
> classes appear). AFAIK this is also tracked per call site, and since the
> choice of allocator is static on startup, these calls are actually
> monomorphic and inline-able (note I don’t work on the compilers, nor have I
> looked closely at the code, but this is my understanding). Anyway I don’t
> think this affected your final implementation, but should be how it works
> for hot code.
>
> The questions:
>
> 1) Is it fair to say offheap_objects is similar to offheap_buffers, but
> with the added benefit of being able to store (large numbers of) individual
> objects which weren’t kept in the Regions on heap either, because they
> needed to be tracked as actual objects?
>
> 2) That offheap_objects is probably the way to go.
>
> Is/there or will there be some suggestions on tuning
>
> memtable_heap_space_in_mb;
> memtable_offheap_space_in_mb;
>
> compared to memtable_total_space_in_mb today (note
> memtable_total_space_in_mb is actually what is still in cassandra.yaml)
>
> 3) I saw somewhere some talk of using a different malloc implementation
> for less native heap fragmentation, but can’t find it now - do you have a
> recommendation to use the non-standard (have to install it yourself) one.
>
> Thanks,
>
> Graham.
>
> On Jun 15, 2014, at 2:04 PM, Benedict Elliott Smith <
> belliottsmith@datastax.com> wrote:
>
> The current implementation is slower for a single memtable-only read
> (dependent on workload, but similar ball park to on heap), but can store
> substantially more per memtable under offheap_objects (again dependent on
> workload, but as much as 8x in extreme cases) which means more queries are
> answerable from memtable, and write amplification is reduced accordingly,
> improving write throughput.
> On 15 Jun 2014 13:32, "graham sanderson" <gr...@vast.com> wrote:
>
> Hi Benedict,
>
> Ah, my mistake, I had assumed that since the memory backing the off heap
> ByteBuffer Regions was freed on discard, that the Regions would be
> immediately recyclable in the on heap case.
>
> I guess I’ll just configure for one of the off-heap variants when 2.1
> comes out…
>
> Any idea of the performance of offheap_buffers and offheap_objects
> (ignoring GC) compared to heap_buffers? I assume offheap_objects must have
> some benefits but presumably at the cost of fragmentation of the native
> heap… obviously I’ll play with it when it comes out…  right now we are
> testing something else, so I don’t have a good environment to try 2.1 - and
> that is always kind of meaningless anyways, since we have many different
> apps using cassandra with different usage patterns, and it is hard to mimic
> production load on all of them at the same time in beta
>
> Thanks anyway for your detailed explanations,
>
> Graham
>
> On Jun 15, 2014, at 1:11 PM, Benedict Elliott Smith <
> belliottsmith@datastax.com> wrote:
>
> Hi Graham,
>
> Unfortunately the problem is more difficult than you might think.
> References to the buffers can persist in-flight to clients long after the
> memtable is discarded, so you would be introducing a subtle corruption
>
> risk
>
> for data returned to clients. Unfortunately the current implementation in
> 2.1 won't solve this problem for on-heap buffers without introducing a
> performance penalty (by copying data from the buffers on read, as we
> currently do for off-heap data), so I don't expect this change will be
> introduced until zero-copy offheap memtables are introduced, which have
> been shelved for the moment.
>
>
> On 15 Jun 2014 10:53, "graham sanderson" <gr...@vast.com> wrote:
>
> Hi Benedict,
>
> So I had a look at the code, and as you say it looked pretty easy to
> recycle on heap slabs… there is already RACE_ALLOCATED which keeps a
> strongly referenced pool, however I was thinking in this case of just
> WeakReferences.
>
> In terms of on heap slabs, it seemed to me that recycling the oldest
>
> slab
>
> you have is probably the best heuristic, since it is less likely to be
>
> in
>
> eden (of course re-using in eden is no worse than worst case today),
> however since the problem tends to be promotion failure of slabs due to
> fragmentation of old gen, recycling one that is already there is even
> better - better still if it has been compacted somewhere pretty stable.
>
> I
>
> think this heuristic would also work well for G1, though I believe the
> recommendation is still not to use that with cassandra.
>
> So for implementation of that I was thinking of using a
> ConcurrentSkipListMap, from a Long representing the allocation order of
>
> the
>
> Region to a weak reference to the Region (just regular 1M sized ones)…
> allocators can pull oldest and discard cleared references (might need a
> scrubber if the map got too big and we were only checking the first
>
> entry).
>
> Beyond that I don’t think there is any need for a configurable-lengthed
> collection of strongly referenced reusable slabs.
>
> Question 1:
>
> This is easy enough to implement, and probably should just be turned on
>
> by
>
> an orthogonal setting… I guess on heap slab is the current default, so
>
> this
>
> feature will be useful
>
> Question 2:
>
> Something similar could be done for off heap slabs… this would seem more
> like it would want a size limit on the number of re-usable slabs… strong
> references with explicit clean() is probably better, than using
> weak-references and letting PhantomReference cleaner on
>
> DirectByteBuffer do
>
> the cleaning later.
>
> Let me know any thoughts and I’ll open an issue (probably 2 - one for on
> heap one for off)… let me know whether you’d like me to assign the
>
> first to
>
> you or me (I couldn’t work on it before next week)
>
> Thanks,
>
> Graham.
>
> On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <
> belliottsmith@datastax.com> wrote:
>
> Graham,
>
> This is largely fixed in 2.1 with the introduction of partially
>
> off-heap
>
> memtables - the slabs reside off-heap, so do not cause any GC issues.
>
> As it happens the changes would also permit us to recycle on-heap slabs
> reasonable easily as well, so feel free to file a ticket for that,
>
> although
>
> it won't be back ported to 2.0.
>
>
> On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
>
> So i’ve been tinkering a bit with CMS config because we are still
>
> seeing
>
> fairly frequent full compacting GC due to framgentation/promotion
>
> failure
>
>
> As mentioned below, we are usually too fragmented to promote new
>
> in-flight
>
> memtables.
>
> This is likely caused by sudden write spikes (which we do have),
>
> though
>
> actually the problems don’t generally happen at that time of our
>
> largest
>
> write spikes (though any write spikes likely cause spill of both new
> memtables along with many other new objects of unknown size into the
> tenured gen, so they cause fragmentation if not immediate GC issue).
>
> We
>
> have lots of things going on in this multi-tenant cluster (GC pauses
>
> are of
>
> course extra bad, since they cause spike in hinted-handoff on other
>
> nodes
>
> which were already busy etc…)
>
> Anyway, considering possibilities:
>
> 0) Try and make our application behavior more steady state - this is
> probably possible, but there are lots of other things (e.g.
>
> compaction,
>
> opscenter, repair etc.) which are both tunable and generally
>
> throttle-able
>
> to think about too.
> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
> (I’d be curious what the “crud” is in particular that is getting
>
> spilled -
>
> presumably it is larger objects since it affects the binary tree of
>
> large
>
> objects)
> 2) Given the above, if we can guarantee even > 24 hours without full
>
> GC, I
>
> don’t think we’d mind running a regular rolling re-start on the
>
> servers
>
> during off hours (note usually the GCs don’t have a visible impact,
>
> but
>
> when they hit multiple machines at once they can)
> 3) Zing is seriously an option, if it would save us large amounts of
> tuning, and constant worry about the “next” thing tweaking the
>
> allocation
>
> patterns - does anyone have any experience with Zing & Cassandra
> 4) Given that we expect periodic bursts of writes,
> memtable_total_space_in_mb is bounded, we are not actually short of
>
> memory
>
> (it just gets fragmented), I’m wondering if anyone has played with
>
> pinning
>
> (up to or initially?) that many 1MB chunks of memory via SlabAllocator
>
> and
>
> re-using… It will get promoted once, and then these 1M chunks won’t be
>
> part
>
> of the subsequent promotion hassle… it will probably also allow more
>
> crud
>
> to die in eden under write load since we aren’t allocating these large
> chunks in eden at the same time. Anyway, I had a little look at the
>
> code,
>
> and the life cycles of memtables is not trivial, but was considering
> attempting a patch to play with… anyone have any thoughts?
>
> Basically in summary, the Slab allocator helps by allocating and
>
> freeing
>
> lots of objects at the same time, however any time slabs are allocated
> under load, we end up promoting them with whatever other live stuff in
>
> eden
>
> is still there. If we only do this once and reuse the slabs, we are
>
> likely
>
> to minimize our promotion problem later (at least for these large
>
> objects)
>
>
> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com>
>
> wrote:
>
>
> Excellent - thank you…
>
> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <
>
> samuel.carriere@urssaf.fr>
>
> wrote:
>
>
> Hi,
> This is arena allocation of memtables. See here for more infos :
>
> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
>
>
>
>
>
> De :    graham sanderson <gr...@vast.com>
> A :     dev@cassandra.apache.org,
> Date :  16/05/2014 14:03
> Objet : Things that are about 1M big
>
>
>
> So just throwing this out there for those for whom this might ring a
>
> bell.
>
>
> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
> interestingly enough most of the objects giving us promotion
>
> failures
>
> are
>
> of size 131074 (dwords) - GC logging obviously doesn?t say what
>
> those
>
> are,
>
> but I?d wager money they are either 1M big byte arrays, or less
>
> likely
>
> 256k entry object arrays backing large maps
>
> So not strictly critical to solving my problem, but I was wondering
>
> if
>
> anyone can think of any heap allocated C* objects which are (with no
> significant changes to standard cassandra config) allocated in 1M
>
> chunks.
>
> (It would save me scouring the code, or a 9 gig heap dump if I need
>
> to
>
> figure it out!)
>
> Thanks,
>
> Graham
>
>
>
>
>
>
>
>
>
>

Re: CMS GC / fragmentation / memtables etc

Posted by graham sanderson <gr...@vast.com>.
Hi Benedict thanks once again for your explanations (hopefully these are my last questions!)… I just read https://issues.apache.org/jira/browse/CASSANDRA-6694 really quickly, so didn’t follow all the discussions…. one thing to note that was argued in the middle was about the cost of virtual methods with multiple implementations. c2 certainly has fast path for 1, and N known implementations where 1<N<something small (with slow path, possible de/re-compile on optimistic assumption failure, or actually when new classes appear). AFAIK this is also tracked per call site, and since the choice of allocator is static on startup, these calls are actually monomorphic and inline-able (note I don’t work on the compilers, nor have I looked closely at the code, but this is my understanding). Anyway I don’t think this affected your final implementation, but should be how it works for hot code.

The questions:

1) Is it fair to say offheap_objects is similar to offheap_buffers, but with the added benefit of being able to store (large numbers of) individual objects which weren’t kept in the Regions on heap either, because they needed to be tracked as actual objects?

2) That offheap_objects is probably the way to go.

Is/there or will there be some suggestions on tuning 

memtable_heap_space_in_mb;
memtable_offheap_space_in_mb;

compared to memtable_total_space_in_mb today (note memtable_total_space_in_mb is actually what is still in cassandra.yaml)

3) I saw somewhere some talk of using a different malloc implementation for less native heap fragmentation, but can’t find it now - do you have a recommendation to use the non-standard (have to install it yourself) one.

Thanks,

Graham.

On Jun 15, 2014, at 2:04 PM, Benedict Elliott Smith <be...@datastax.com> wrote:

> The current implementation is slower for a single memtable-only read
> (dependent on workload, but similar ball park to on heap), but can store
> substantially more per memtable under offheap_objects (again dependent on
> workload, but as much as 8x in extreme cases) which means more queries are
> answerable from memtable, and write amplification is reduced accordingly,
> improving write throughput.
> On 15 Jun 2014 13:32, "graham sanderson" <gr...@vast.com> wrote:
> 
>> Hi Benedict,
>> 
>> Ah, my mistake, I had assumed that since the memory backing the off heap
>> ByteBuffer Regions was freed on discard, that the Regions would be
>> immediately recyclable in the on heap case.
>> 
>> I guess I’ll just configure for one of the off-heap variants when 2.1
>> comes out…
>> 
>> Any idea of the performance of offheap_buffers and offheap_objects
>> (ignoring GC) compared to heap_buffers? I assume offheap_objects must have
>> some benefits but presumably at the cost of fragmentation of the native
>> heap… obviously I’ll play with it when it comes out…  right now we are
>> testing something else, so I don’t have a good environment to try 2.1 - and
>> that is always kind of meaningless anyways, since we have many different
>> apps using cassandra with different usage patterns, and it is hard to mimic
>> production load on all of them at the same time in beta
>> 
>> Thanks anyway for your detailed explanations,
>> 
>> Graham
>> 
>> On Jun 15, 2014, at 1:11 PM, Benedict Elliott Smith <
>> belliottsmith@datastax.com> wrote:
>> 
>>> Hi Graham,
>>> 
>>> Unfortunately the problem is more difficult than you might think.
>>> References to the buffers can persist in-flight to clients long after the
>>> memtable is discarded, so you would be introducing a subtle corruption
>> risk
>>> for data returned to clients. Unfortunately the current implementation in
>>> 2.1 won't solve this problem for on-heap buffers without introducing a
>>> performance penalty (by copying data from the buffers on read, as we
>>> currently do for off-heap data), so I don't expect this change will be
>>> introduced until zero-copy offheap memtables are introduced, which have
>>> been shelved for the moment.
>>> 
>>> 
>>> On 15 Jun 2014 10:53, "graham sanderson" <gr...@vast.com> wrote:
>>> 
>>>> Hi Benedict,
>>>> 
>>>> So I had a look at the code, and as you say it looked pretty easy to
>>>> recycle on heap slabs… there is already RACE_ALLOCATED which keeps a
>>>> strongly referenced pool, however I was thinking in this case of just
>>>> WeakReferences.
>>>> 
>>>> In terms of on heap slabs, it seemed to me that recycling the oldest
>> slab
>>>> you have is probably the best heuristic, since it is less likely to be
>> in
>>>> eden (of course re-using in eden is no worse than worst case today),
>>>> however since the problem tends to be promotion failure of slabs due to
>>>> fragmentation of old gen, recycling one that is already there is even
>>>> better - better still if it has been compacted somewhere pretty stable.
>> I
>>>> think this heuristic would also work well for G1, though I believe the
>>>> recommendation is still not to use that with cassandra.
>>>> 
>>>> So for implementation of that I was thinking of using a
>>>> ConcurrentSkipListMap, from a Long representing the allocation order of
>> the
>>>> Region to a weak reference to the Region (just regular 1M sized ones)…
>>>> allocators can pull oldest and discard cleared references (might need a
>>>> scrubber if the map got too big and we were only checking the first
>> entry).
>>>> Beyond that I don’t think there is any need for a configurable-lengthed
>>>> collection of strongly referenced reusable slabs.
>>>> 
>>>> Question 1:
>>>> 
>>>> This is easy enough to implement, and probably should just be turned on
>> by
>>>> an orthogonal setting… I guess on heap slab is the current default, so
>> this
>>>> feature will be useful
>>>> 
>>>> Question 2:
>>>> 
>>>> Something similar could be done for off heap slabs… this would seem more
>>>> like it would want a size limit on the number of re-usable slabs… strong
>>>> references with explicit clean() is probably better, than using
>>>> weak-references and letting PhantomReference cleaner on
>> DirectByteBuffer do
>>>> the cleaning later.
>>>> 
>>>> Let me know any thoughts and I’ll open an issue (probably 2 - one for on
>>>> heap one for off)… let me know whether you’d like me to assign the
>> first to
>>>> you or me (I couldn’t work on it before next week)
>>>> 
>>>> Thanks,
>>>> 
>>>> Graham.
>>>> 
>>>> On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <
>>>> belliottsmith@datastax.com> wrote:
>>>> 
>>>>> Graham,
>>>>> 
>>>>> This is largely fixed in 2.1 with the introduction of partially
>> off-heap
>>>>> memtables - the slabs reside off-heap, so do not cause any GC issues.
>>>>> 
>>>>> As it happens the changes would also permit us to recycle on-heap slabs
>>>>> reasonable easily as well, so feel free to file a ticket for that,
>>>> although
>>>>> it won't be back ported to 2.0.
>>>>> 
>>>>> 
>>>>> On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
>>>>> 
>>>>>> So i’ve been tinkering a bit with CMS config because we are still
>> seeing
>>>>>> fairly frequent full compacting GC due to framgentation/promotion
>>>> failure
>>>>>> 
>>>>>> As mentioned below, we are usually too fragmented to promote new
>>>> in-flight
>>>>>> memtables.
>>>>>> 
>>>>>> This is likely caused by sudden write spikes (which we do have),
>> though
>>>>>> actually the problems don’t generally happen at that time of our
>> largest
>>>>>> write spikes (though any write spikes likely cause spill of both new
>>>>>> memtables along with many other new objects of unknown size into the
>>>>>> tenured gen, so they cause fragmentation if not immediate GC issue).
>> We
>>>>>> have lots of things going on in this multi-tenant cluster (GC pauses
>>>> are of
>>>>>> course extra bad, since they cause spike in hinted-handoff on other
>>>> nodes
>>>>>> which were already busy etc…)
>>>>>> 
>>>>>> Anyway, considering possibilities:
>>>>>> 
>>>>>> 0) Try and make our application behavior more steady state - this is
>>>>>> probably possible, but there are lots of other things (e.g.
>> compaction,
>>>>>> opscenter, repair etc.) which are both tunable and generally
>>>> throttle-able
>>>>>> to think about too.
>>>>>> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
>>>>>> (I’d be curious what the “crud” is in particular that is getting
>>>> spilled -
>>>>>> presumably it is larger objects since it affects the binary tree of
>>>> large
>>>>>> objects)
>>>>>> 2) Given the above, if we can guarantee even > 24 hours without full
>>>> GC, I
>>>>>> don’t think we’d mind running a regular rolling re-start on the
>> servers
>>>>>> during off hours (note usually the GCs don’t have a visible impact,
>> but
>>>>>> when they hit multiple machines at once they can)
>>>>>> 3) Zing is seriously an option, if it would save us large amounts of
>>>>>> tuning, and constant worry about the “next” thing tweaking the
>>>> allocation
>>>>>> patterns - does anyone have any experience with Zing & Cassandra
>>>>>> 4) Given that we expect periodic bursts of writes,
>>>>>> memtable_total_space_in_mb is bounded, we are not actually short of
>>>> memory
>>>>>> (it just gets fragmented), I’m wondering if anyone has played with
>>>> pinning
>>>>>> (up to or initially?) that many 1MB chunks of memory via SlabAllocator
>>>> and
>>>>>> re-using… It will get promoted once, and then these 1M chunks won’t be
>>>> part
>>>>>> of the subsequent promotion hassle… it will probably also allow more
>>>> crud
>>>>>> to die in eden under write load since we aren’t allocating these large
>>>>>> chunks in eden at the same time. Anyway, I had a little look at the
>>>> code,
>>>>>> and the life cycles of memtables is not trivial, but was considering
>>>>>> attempting a patch to play with… anyone have any thoughts?
>>>>>> 
>>>>>> Basically in summary, the Slab allocator helps by allocating and
>> freeing
>>>>>> lots of objects at the same time, however any time slabs are allocated
>>>>>> under load, we end up promoting them with whatever other live stuff in
>>>> eden
>>>>>> is still there. If we only do this once and reuse the slabs, we are
>>>> likely
>>>>>> to minimize our promotion problem later (at least for these large
>>>> objects)
>>>>>> 
>>>>>> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com>
>> wrote:
>>>>>> 
>>>>>>> Excellent - thank you…
>>>>>>> 
>>>>>>> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <
>>>> samuel.carriere@urssaf.fr>
>>>>>> wrote:
>>>>>>> 
>>>>>>>> Hi,
>>>>>>>> This is arena allocation of memtables. See here for more infos :
>>>>>>>> 
>>>> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> De :    graham sanderson <gr...@vast.com>
>>>>>>>> A :     dev@cassandra.apache.org,
>>>>>>>> Date :  16/05/2014 14:03
>>>>>>>> Objet : Things that are about 1M big
>>>>>>>> 
>>>>>>>> 
>>>>>>>> 
>>>>>>>> So just throwing this out there for those for whom this might ring a
>>>>>> bell.
>>>>>>>> 
>>>>>>>> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
>>>>>>>> interestingly enough most of the objects giving us promotion
>> failures
>>>>>> are
>>>>>>>> of size 131074 (dwords) - GC logging obviously doesn?t say what
>> those
>>>>>> are,
>>>>>>>> but I?d wager money they are either 1M big byte arrays, or less
>> likely
>>>>>>>> 256k entry object arrays backing large maps
>>>>>>>> 
>>>>>>>> So not strictly critical to solving my problem, but I was wondering
>> if
>>>>>>>> anyone can think of any heap allocated C* objects which are (with no
>>>>>>>> significant changes to standard cassandra config) allocated in 1M
>>>>>> chunks.
>>>>>>>> (It would save me scouring the code, or a 9 gig heap dump if I need
>> to
>>>>>>>> figure it out!)
>>>>>>>> 
>>>>>>>> Thanks,
>>>>>>>> 
>>>>>>>> Graham
>>>>>>> 
>>>>>> 
>>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: CMS GC / fragmentation / memtables etc

Posted by Benedict Elliott Smith <be...@datastax.com>.
The current implementation is slower for a single memtable-only read
(dependent on workload, but similar ball park to on heap), but can store
substantially more per memtable under offheap_objects (again dependent on
workload, but as much as 8x in extreme cases) which means more queries are
answerable from memtable, and write amplification is reduced accordingly,
improving write throughput.
On 15 Jun 2014 13:32, "graham sanderson" <gr...@vast.com> wrote:

> Hi Benedict,
>
> Ah, my mistake, I had assumed that since the memory backing the off heap
> ByteBuffer Regions was freed on discard, that the Regions would be
> immediately recyclable in the on heap case.
>
> I guess I’ll just configure for one of the off-heap variants when 2.1
> comes out…
>
> Any idea of the performance of offheap_buffers and offheap_objects
> (ignoring GC) compared to heap_buffers? I assume offheap_objects must have
> some benefits but presumably at the cost of fragmentation of the native
> heap… obviously I’ll play with it when it comes out…  right now we are
> testing something else, so I don’t have a good environment to try 2.1 - and
> that is always kind of meaningless anyways, since we have many different
> apps using cassandra with different usage patterns, and it is hard to mimic
> production load on all of them at the same time in beta
>
> Thanks anyway for your detailed explanations,
>
> Graham
>
> On Jun 15, 2014, at 1:11 PM, Benedict Elliott Smith <
> belliottsmith@datastax.com> wrote:
>
> > Hi Graham,
> >
> > Unfortunately the problem is more difficult than you might think.
> > References to the buffers can persist in-flight to clients long after the
> > memtable is discarded, so you would be introducing a subtle corruption
> risk
> > for data returned to clients. Unfortunately the current implementation in
> > 2.1 won't solve this problem for on-heap buffers without introducing a
> > performance penalty (by copying data from the buffers on read, as we
> > currently do for off-heap data), so I don't expect this change will be
> > introduced until zero-copy offheap memtables are introduced, which have
> > been shelved for the moment.
> >
> >
> > On 15 Jun 2014 10:53, "graham sanderson" <gr...@vast.com> wrote:
> >
> >> Hi Benedict,
> >>
> >> So I had a look at the code, and as you say it looked pretty easy to
> >> recycle on heap slabs… there is already RACE_ALLOCATED which keeps a
> >> strongly referenced pool, however I was thinking in this case of just
> >> WeakReferences.
> >>
> >> In terms of on heap slabs, it seemed to me that recycling the oldest
> slab
> >> you have is probably the best heuristic, since it is less likely to be
> in
> >> eden (of course re-using in eden is no worse than worst case today),
> >> however since the problem tends to be promotion failure of slabs due to
> >> fragmentation of old gen, recycling one that is already there is even
> >> better - better still if it has been compacted somewhere pretty stable.
> I
> >> think this heuristic would also work well for G1, though I believe the
> >> recommendation is still not to use that with cassandra.
> >>
> >> So for implementation of that I was thinking of using a
> >> ConcurrentSkipListMap, from a Long representing the allocation order of
> the
> >> Region to a weak reference to the Region (just regular 1M sized ones)…
> >> allocators can pull oldest and discard cleared references (might need a
> >> scrubber if the map got too big and we were only checking the first
> entry).
> >> Beyond that I don’t think there is any need for a configurable-lengthed
> >> collection of strongly referenced reusable slabs.
> >>
> >> Question 1:
> >>
> >> This is easy enough to implement, and probably should just be turned on
> by
> >> an orthogonal setting… I guess on heap slab is the current default, so
> this
> >> feature will be useful
> >>
> >> Question 2:
> >>
> >> Something similar could be done for off heap slabs… this would seem more
> >> like it would want a size limit on the number of re-usable slabs… strong
> >> references with explicit clean() is probably better, than using
> >> weak-references and letting PhantomReference cleaner on
> DirectByteBuffer do
> >> the cleaning later.
> >>
> >> Let me know any thoughts and I’ll open an issue (probably 2 - one for on
> >> heap one for off)… let me know whether you’d like me to assign the
> first to
> >> you or me (I couldn’t work on it before next week)
> >>
> >> Thanks,
> >>
> >> Graham.
> >>
> >> On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <
> >> belliottsmith@datastax.com> wrote:
> >>
> >>> Graham,
> >>>
> >>> This is largely fixed in 2.1 with the introduction of partially
> off-heap
> >>> memtables - the slabs reside off-heap, so do not cause any GC issues.
> >>>
> >>> As it happens the changes would also permit us to recycle on-heap slabs
> >>> reasonable easily as well, so feel free to file a ticket for that,
> >> although
> >>> it won't be back ported to 2.0.
> >>>
> >>>
> >>> On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
> >>>
> >>>> So i’ve been tinkering a bit with CMS config because we are still
> seeing
> >>>> fairly frequent full compacting GC due to framgentation/promotion
> >> failure
> >>>>
> >>>> As mentioned below, we are usually too fragmented to promote new
> >> in-flight
> >>>> memtables.
> >>>>
> >>>> This is likely caused by sudden write spikes (which we do have),
> though
> >>>> actually the problems don’t generally happen at that time of our
> largest
> >>>> write spikes (though any write spikes likely cause spill of both new
> >>>> memtables along with many other new objects of unknown size into the
> >>>> tenured gen, so they cause fragmentation if not immediate GC issue).
> We
> >>>> have lots of things going on in this multi-tenant cluster (GC pauses
> >> are of
> >>>> course extra bad, since they cause spike in hinted-handoff on other
> >> nodes
> >>>> which were already busy etc…)
> >>>>
> >>>> Anyway, considering possibilities:
> >>>>
> >>>> 0) Try and make our application behavior more steady state - this is
> >>>> probably possible, but there are lots of other things (e.g.
> compaction,
> >>>> opscenter, repair etc.) which are both tunable and generally
> >> throttle-able
> >>>> to think about too.
> >>>> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
> >>>> (I’d be curious what the “crud” is in particular that is getting
> >> spilled -
> >>>> presumably it is larger objects since it affects the binary tree of
> >> large
> >>>> objects)
> >>>> 2) Given the above, if we can guarantee even > 24 hours without full
> >> GC, I
> >>>> don’t think we’d mind running a regular rolling re-start on the
> servers
> >>>> during off hours (note usually the GCs don’t have a visible impact,
> but
> >>>> when they hit multiple machines at once they can)
> >>>> 3) Zing is seriously an option, if it would save us large amounts of
> >>>> tuning, and constant worry about the “next” thing tweaking the
> >> allocation
> >>>> patterns - does anyone have any experience with Zing & Cassandra
> >>>> 4) Given that we expect periodic bursts of writes,
> >>>> memtable_total_space_in_mb is bounded, we are not actually short of
> >> memory
> >>>> (it just gets fragmented), I’m wondering if anyone has played with
> >> pinning
> >>>> (up to or initially?) that many 1MB chunks of memory via SlabAllocator
> >> and
> >>>> re-using… It will get promoted once, and then these 1M chunks won’t be
> >> part
> >>>> of the subsequent promotion hassle… it will probably also allow more
> >> crud
> >>>> to die in eden under write load since we aren’t allocating these large
> >>>> chunks in eden at the same time. Anyway, I had a little look at the
> >> code,
> >>>> and the life cycles of memtables is not trivial, but was considering
> >>>> attempting a patch to play with… anyone have any thoughts?
> >>>>
> >>>> Basically in summary, the Slab allocator helps by allocating and
> freeing
> >>>> lots of objects at the same time, however any time slabs are allocated
> >>>> under load, we end up promoting them with whatever other live stuff in
> >> eden
> >>>> is still there. If we only do this once and reuse the slabs, we are
> >> likely
> >>>> to minimize our promotion problem later (at least for these large
> >> objects)
> >>>>
> >>>> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com>
> wrote:
> >>>>
> >>>>> Excellent - thank you…
> >>>>>
> >>>>> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <
> >> samuel.carriere@urssaf.fr>
> >>>> wrote:
> >>>>>
> >>>>>> Hi,
> >>>>>> This is arena allocation of memtables. See here for more infos :
> >>>>>>
> >> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> De :    graham sanderson <gr...@vast.com>
> >>>>>> A :     dev@cassandra.apache.org,
> >>>>>> Date :  16/05/2014 14:03
> >>>>>> Objet : Things that are about 1M big
> >>>>>>
> >>>>>>
> >>>>>>
> >>>>>> So just throwing this out there for those for whom this might ring a
> >>>> bell.
> >>>>>>
> >>>>>> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
> >>>>>> interestingly enough most of the objects giving us promotion
> failures
> >>>> are
> >>>>>> of size 131074 (dwords) - GC logging obviously doesn?t say what
> those
> >>>> are,
> >>>>>> but I?d wager money they are either 1M big byte arrays, or less
> likely
> >>>>>> 256k entry object arrays backing large maps
> >>>>>>
> >>>>>> So not strictly critical to solving my problem, but I was wondering
> if
> >>>>>> anyone can think of any heap allocated C* objects which are (with no
> >>>>>> significant changes to standard cassandra config) allocated in 1M
> >>>> chunks.
> >>>>>> (It would save me scouring the code, or a 9 gig heap dump if I need
> to
> >>>>>> figure it out!)
> >>>>>>
> >>>>>> Thanks,
> >>>>>>
> >>>>>> Graham
> >>>>>
> >>>>
> >>>>
> >>
> >>
>
>

Re: CMS GC / fragmentation / memtables etc

Posted by graham sanderson <gr...@vast.com>.
Hi Benedict,

Ah, my mistake, I had assumed that since the memory backing the off heap ByteBuffer Regions was freed on discard, that the Regions would be immediately recyclable in the on heap case.

I guess I’ll just configure for one of the off-heap variants when 2.1 comes out…

Any idea of the performance of offheap_buffers and offheap_objects (ignoring GC) compared to heap_buffers? I assume offheap_objects must have some benefits but presumably at the cost of fragmentation of the native heap… obviously I’ll play with it when it comes out…  right now we are testing something else, so I don’t have a good environment to try 2.1 - and that is always kind of meaningless anyways, since we have many different apps using cassandra with different usage patterns, and it is hard to mimic production load on all of them at the same time in beta

Thanks anyway for your detailed explanations,

Graham

On Jun 15, 2014, at 1:11 PM, Benedict Elliott Smith <be...@datastax.com> wrote:

> Hi Graham,
> 
> Unfortunately the problem is more difficult than you might think.
> References to the buffers can persist in-flight to clients long after the
> memtable is discarded, so you would be introducing a subtle corruption risk
> for data returned to clients. Unfortunately the current implementation in
> 2.1 won't solve this problem for on-heap buffers without introducing a
> performance penalty (by copying data from the buffers on read, as we
> currently do for off-heap data), so I don't expect this change will be
> introduced until zero-copy offheap memtables are introduced, which have
> been shelved for the moment.
> 
> 
> On 15 Jun 2014 10:53, "graham sanderson" <gr...@vast.com> wrote:
> 
>> Hi Benedict,
>> 
>> So I had a look at the code, and as you say it looked pretty easy to
>> recycle on heap slabs… there is already RACE_ALLOCATED which keeps a
>> strongly referenced pool, however I was thinking in this case of just
>> WeakReferences.
>> 
>> In terms of on heap slabs, it seemed to me that recycling the oldest slab
>> you have is probably the best heuristic, since it is less likely to be in
>> eden (of course re-using in eden is no worse than worst case today),
>> however since the problem tends to be promotion failure of slabs due to
>> fragmentation of old gen, recycling one that is already there is even
>> better - better still if it has been compacted somewhere pretty stable. I
>> think this heuristic would also work well for G1, though I believe the
>> recommendation is still not to use that with cassandra.
>> 
>> So for implementation of that I was thinking of using a
>> ConcurrentSkipListMap, from a Long representing the allocation order of the
>> Region to a weak reference to the Region (just regular 1M sized ones)…
>> allocators can pull oldest and discard cleared references (might need a
>> scrubber if the map got too big and we were only checking the first entry).
>> Beyond that I don’t think there is any need for a configurable-lengthed
>> collection of strongly referenced reusable slabs.
>> 
>> Question 1:
>> 
>> This is easy enough to implement, and probably should just be turned on by
>> an orthogonal setting… I guess on heap slab is the current default, so this
>> feature will be useful
>> 
>> Question 2:
>> 
>> Something similar could be done for off heap slabs… this would seem more
>> like it would want a size limit on the number of re-usable slabs… strong
>> references with explicit clean() is probably better, than using
>> weak-references and letting PhantomReference cleaner on DirectByteBuffer do
>> the cleaning later.
>> 
>> Let me know any thoughts and I’ll open an issue (probably 2 - one for on
>> heap one for off)… let me know whether you’d like me to assign the first to
>> you or me (I couldn’t work on it before next week)
>> 
>> Thanks,
>> 
>> Graham.
>> 
>> On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <
>> belliottsmith@datastax.com> wrote:
>> 
>>> Graham,
>>> 
>>> This is largely fixed in 2.1 with the introduction of partially off-heap
>>> memtables - the slabs reside off-heap, so do not cause any GC issues.
>>> 
>>> As it happens the changes would also permit us to recycle on-heap slabs
>>> reasonable easily as well, so feel free to file a ticket for that,
>> although
>>> it won't be back ported to 2.0.
>>> 
>>> 
>>> On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
>>> 
>>>> So i’ve been tinkering a bit with CMS config because we are still seeing
>>>> fairly frequent full compacting GC due to framgentation/promotion
>> failure
>>>> 
>>>> As mentioned below, we are usually too fragmented to promote new
>> in-flight
>>>> memtables.
>>>> 
>>>> This is likely caused by sudden write spikes (which we do have), though
>>>> actually the problems don’t generally happen at that time of our largest
>>>> write spikes (though any write spikes likely cause spill of both new
>>>> memtables along with many other new objects of unknown size into the
>>>> tenured gen, so they cause fragmentation if not immediate GC issue). We
>>>> have lots of things going on in this multi-tenant cluster (GC pauses
>> are of
>>>> course extra bad, since they cause spike in hinted-handoff on other
>> nodes
>>>> which were already busy etc…)
>>>> 
>>>> Anyway, considering possibilities:
>>>> 
>>>> 0) Try and make our application behavior more steady state - this is
>>>> probably possible, but there are lots of other things (e.g. compaction,
>>>> opscenter, repair etc.) which are both tunable and generally
>> throttle-able
>>>> to think about too.
>>>> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
>>>> (I’d be curious what the “crud” is in particular that is getting
>> spilled -
>>>> presumably it is larger objects since it affects the binary tree of
>> large
>>>> objects)
>>>> 2) Given the above, if we can guarantee even > 24 hours without full
>> GC, I
>>>> don’t think we’d mind running a regular rolling re-start on the servers
>>>> during off hours (note usually the GCs don’t have a visible impact, but
>>>> when they hit multiple machines at once they can)
>>>> 3) Zing is seriously an option, if it would save us large amounts of
>>>> tuning, and constant worry about the “next” thing tweaking the
>> allocation
>>>> patterns - does anyone have any experience with Zing & Cassandra
>>>> 4) Given that we expect periodic bursts of writes,
>>>> memtable_total_space_in_mb is bounded, we are not actually short of
>> memory
>>>> (it just gets fragmented), I’m wondering if anyone has played with
>> pinning
>>>> (up to or initially?) that many 1MB chunks of memory via SlabAllocator
>> and
>>>> re-using… It will get promoted once, and then these 1M chunks won’t be
>> part
>>>> of the subsequent promotion hassle… it will probably also allow more
>> crud
>>>> to die in eden under write load since we aren’t allocating these large
>>>> chunks in eden at the same time. Anyway, I had a little look at the
>> code,
>>>> and the life cycles of memtables is not trivial, but was considering
>>>> attempting a patch to play with… anyone have any thoughts?
>>>> 
>>>> Basically in summary, the Slab allocator helps by allocating and freeing
>>>> lots of objects at the same time, however any time slabs are allocated
>>>> under load, we end up promoting them with whatever other live stuff in
>> eden
>>>> is still there. If we only do this once and reuse the slabs, we are
>> likely
>>>> to minimize our promotion problem later (at least for these large
>> objects)
>>>> 
>>>> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com> wrote:
>>>> 
>>>>> Excellent - thank you…
>>>>> 
>>>>> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <
>> samuel.carriere@urssaf.fr>
>>>> wrote:
>>>>> 
>>>>>> Hi,
>>>>>> This is arena allocation of memtables. See here for more infos :
>>>>>> 
>> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> De :    graham sanderson <gr...@vast.com>
>>>>>> A :     dev@cassandra.apache.org,
>>>>>> Date :  16/05/2014 14:03
>>>>>> Objet : Things that are about 1M big
>>>>>> 
>>>>>> 
>>>>>> 
>>>>>> So just throwing this out there for those for whom this might ring a
>>>> bell.
>>>>>> 
>>>>>> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
>>>>>> interestingly enough most of the objects giving us promotion failures
>>>> are
>>>>>> of size 131074 (dwords) - GC logging obviously doesn?t say what those
>>>> are,
>>>>>> but I?d wager money they are either 1M big byte arrays, or less likely
>>>>>> 256k entry object arrays backing large maps
>>>>>> 
>>>>>> So not strictly critical to solving my problem, but I was wondering if
>>>>>> anyone can think of any heap allocated C* objects which are (with no
>>>>>> significant changes to standard cassandra config) allocated in 1M
>>>> chunks.
>>>>>> (It would save me scouring the code, or a 9 gig heap dump if I need to
>>>>>> figure it out!)
>>>>>> 
>>>>>> Thanks,
>>>>>> 
>>>>>> Graham
>>>>> 
>>>> 
>>>> 
>> 
>> 


Re: CMS GC / fragmentation / memtables etc

Posted by Benedict Elliott Smith <be...@datastax.com>.
Hi Graham,

Unfortunately the problem is more difficult than you might think.
References to the buffers can persist in-flight to clients long after the
memtable is discarded, so you would be introducing a subtle corruption risk
for data returned to clients. Unfortunately the current implementation in
2.1 won't solve this problem for on-heap buffers without introducing a
performance penalty (by copying data from the buffers on read, as we
currently do for off-heap data), so I don't expect this change will be
introduced until zero-copy offheap memtables are introduced, which have
been shelved for the moment.


On 15 Jun 2014 10:53, "graham sanderson" <gr...@vast.com> wrote:

> Hi Benedict,
>
> So I had a look at the code, and as you say it looked pretty easy to
> recycle on heap slabs… there is already RACE_ALLOCATED which keeps a
> strongly referenced pool, however I was thinking in this case of just
> WeakReferences.
>
> In terms of on heap slabs, it seemed to me that recycling the oldest slab
> you have is probably the best heuristic, since it is less likely to be in
> eden (of course re-using in eden is no worse than worst case today),
> however since the problem tends to be promotion failure of slabs due to
> fragmentation of old gen, recycling one that is already there is even
> better - better still if it has been compacted somewhere pretty stable. I
> think this heuristic would also work well for G1, though I believe the
> recommendation is still not to use that with cassandra.
>
> So for implementation of that I was thinking of using a
> ConcurrentSkipListMap, from a Long representing the allocation order of the
> Region to a weak reference to the Region (just regular 1M sized ones)…
> allocators can pull oldest and discard cleared references (might need a
> scrubber if the map got too big and we were only checking the first entry).
> Beyond that I don’t think there is any need for a configurable-lengthed
> collection of strongly referenced reusable slabs.
>
> Question 1:
>
> This is easy enough to implement, and probably should just be turned on by
> an orthogonal setting… I guess on heap slab is the current default, so this
> feature will be useful
>
> Question 2:
>
> Something similar could be done for off heap slabs… this would seem more
> like it would want a size limit on the number of re-usable slabs… strong
> references with explicit clean() is probably better, than using
> weak-references and letting PhantomReference cleaner on DirectByteBuffer do
> the cleaning later.
>
> Let me know any thoughts and I’ll open an issue (probably 2 - one for on
> heap one for off)… let me know whether you’d like me to assign the first to
> you or me (I couldn’t work on it before next week)
>
> Thanks,
>
> Graham.
>
> On May 21, 2014, at 2:20 AM, Benedict Elliott Smith <
> belliottsmith@datastax.com> wrote:
>
> > Graham,
> >
> > This is largely fixed in 2.1 with the introduction of partially off-heap
> > memtables - the slabs reside off-heap, so do not cause any GC issues.
> >
> > As it happens the changes would also permit us to recycle on-heap slabs
> > reasonable easily as well, so feel free to file a ticket for that,
> although
> > it won't be back ported to 2.0.
> >
> >
> > On 21 May 2014 00:57, graham sanderson <gr...@vast.com> wrote:
> >
> >> So i’ve been tinkering a bit with CMS config because we are still seeing
> >> fairly frequent full compacting GC due to framgentation/promotion
> failure
> >>
> >> As mentioned below, we are usually too fragmented to promote new
> in-flight
> >> memtables.
> >>
> >> This is likely caused by sudden write spikes (which we do have), though
> >> actually the problems don’t generally happen at that time of our largest
> >> write spikes (though any write spikes likely cause spill of both new
> >> memtables along with many other new objects of unknown size into the
> >> tenured gen, so they cause fragmentation if not immediate GC issue). We
> >> have lots of things going on in this multi-tenant cluster (GC pauses
> are of
> >> course extra bad, since they cause spike in hinted-handoff on other
> nodes
> >> which were already busy etc…)
> >>
> >> Anyway, considering possibilities:
> >>
> >> 0) Try and make our application behavior more steady state - this is
> >> probably possible, but there are lots of other things (e.g. compaction,
> >> opscenter, repair etc.) which are both tunable and generally
> throttle-able
> >> to think about too.
> >> 1) Play with tweaking PLAB configs to see if we can ease fragmentation
> >> (I’d be curious what the “crud” is in particular that is getting
> spilled -
> >> presumably it is larger objects since it affects the binary tree of
> large
> >> objects)
> >> 2) Given the above, if we can guarantee even > 24 hours without full
> GC, I
> >> don’t think we’d mind running a regular rolling re-start on the servers
> >> during off hours (note usually the GCs don’t have a visible impact, but
> >> when they hit multiple machines at once they can)
> >> 3) Zing is seriously an option, if it would save us large amounts of
> >> tuning, and constant worry about the “next” thing tweaking the
> allocation
> >> patterns - does anyone have any experience with Zing & Cassandra
> >> 4) Given that we expect periodic bursts of writes,
> >> memtable_total_space_in_mb is bounded, we are not actually short of
> memory
> >> (it just gets fragmented), I’m wondering if anyone has played with
> pinning
> >> (up to or initially?) that many 1MB chunks of memory via SlabAllocator
> and
> >> re-using… It will get promoted once, and then these 1M chunks won’t be
> part
> >> of the subsequent promotion hassle… it will probably also allow more
> crud
> >> to die in eden under write load since we aren’t allocating these large
> >> chunks in eden at the same time. Anyway, I had a little look at the
> code,
> >> and the life cycles of memtables is not trivial, but was considering
> >> attempting a patch to play with… anyone have any thoughts?
> >>
> >> Basically in summary, the Slab allocator helps by allocating and freeing
> >> lots of objects at the same time, however any time slabs are allocated
> >> under load, we end up promoting them with whatever other live stuff in
> eden
> >> is still there. If we only do this once and reuse the slabs, we are
> likely
> >> to minimize our promotion problem later (at least for these large
> objects)
> >>
> >> On May 16, 2014, at 9:37 PM, graham sanderson <gr...@vast.com> wrote:
> >>
> >>> Excellent - thank you…
> >>>
> >>> On May 16, 2014, at 7:08 AM, Samuel CARRIERE <
> samuel.carriere@urssaf.fr>
> >> wrote:
> >>>
> >>>> Hi,
> >>>> This is arena allocation of memtables. See here for more infos :
> >>>>
> http://www.datastax.com/dev/blog/whats-new-in-cassandra-1-0-performance
> >>>>
> >>>>
> >>>>
> >>>>
> >>>> De :    graham sanderson <gr...@vast.com>
> >>>> A :     dev@cassandra.apache.org,
> >>>> Date :  16/05/2014 14:03
> >>>> Objet : Things that are about 1M big
> >>>>
> >>>>
> >>>>
> >>>> So just throwing this out there for those for whom this might ring a
> >> bell.
> >>>>
> >>>> I?m debugging some CMS memory fragmentation issues on 2.0.5 - and
> >>>> interestingly enough most of the objects giving us promotion failures
> >> are
> >>>> of size 131074 (dwords) - GC logging obviously doesn?t say what those
> >> are,
> >>>> but I?d wager money they are either 1M big byte arrays, or less likely
> >>>> 256k entry object arrays backing large maps
> >>>>
> >>>> So not strictly critical to solving my problem, but I was wondering if
> >>>> anyone can think of any heap allocated C* objects which are (with no
> >>>> significant changes to standard cassandra config) allocated in 1M
> >> chunks.
> >>>> (It would save me scouring the code, or a 9 gig heap dump if I need to
> >>>> figure it out!)
> >>>>
> >>>> Thanks,
> >>>>
> >>>> Graham
> >>>
> >>
> >>
>
>