You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Terje Marthinussen <tm...@gmail.com> on 2011/04/27 18:01:21 UTC
memtablePostFlusher blocking writes?
0.8 trunk:
When playing back a fairly large chunk of hints, things basically locks up
under load.
The hints are never processed successfully. Lots of Mutations dropped.
One thing is that maybe the default 10k columns per send with 50ms delays is
a bit on the aggressive side (10k*20 =200.000 columns in a second?), the
other thing is that it seems like the whole memtable flushing locks up.
I tried to increase number of memtable flushers and queue a bit (8
concurrent flushers) to make things work, but no luck.
Pool Name Active Pending Completed
ReadStage 0 0 1
RequestResponseStage 0 0 2236304
MutationStage 100 17564 4011533
ReadRepairStage 0 0 0
ReplicateOnWriteStage 0 0 0
GossipStage 0 0 2281
AntiEntropyStage 0 0 0
MigrationStage 0 0 0
MemtablePostFlusher 1 13 50
StreamStage 0 0 0
FlushWriter 8 14 73
MiscStage 0 0 0
FlushSorter 0 0 0
InternalResponseStage 0 0 0
HintedHandoff 1 8 3
A quick source code scan makes me believe that the MemtablePostFlusher
should not normally use a lot of time, but it seem like it does so here.
What may cause this?
Terje
Re: memtablePostFlusher blocking writes?
Posted by Jonathan Ellis <jb...@gmail.com>.
On Wed, Apr 27, 2011 at 5:23 PM, Terje Marthinussen
<tm...@gmail.com> wrote:
> I have two issues here.
> - The massive amount of mutation caused by the hints playback
I'm not sure how one node playing back hints could cause this. The
intent of the code in HintedHandoffManager is to send a single
mutation, wait for it to be completed, then proceed to the next. This
is deliberately not parallel to avoid overwhelming the recipient:
rm.add(cf);
IWriteResponseHandler responseHandler =
WriteResponseHandler.create(endpoint);
MessagingService.instance().sendRR(rm, endpoint, responseHandler);
try
{
responseHandler.get();
}
catch (TimeoutException e)
{
return false;
}
> - The long periods where there are no increase in completed mutations.
When all your memtables and flush queues are full, mutations have
stop. Otherwise it will OOM trying to hold an increasing number of
full memtables.
--
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com
Re: memtablePostFlusher blocking writes?
Posted by Terje Marthinussen <tm...@gmail.com>.
It is a good question what is the problem here.
I dont think it is the pending mutations and flushes, the real problem is
what causes them, and it is not me!
There was maybe a misleading comment in my original mail.
It is not the hinted handoffs sent from this node that is the problem, but
the 1.6 million hints for this node that is being sent to it from another
(neighbour) node.
A more compact view what happens when the other node starts playing back the
hints. There is 2 seconds between the lines.
MutationStage 100 615 2008362
MutationStage 100 1202 2054437
MutationStage 100 1394 2104971
MutationStage 100 9318 2142964
MutationStage 100 46392 2142964
MutationStage 100 83188 2142964
MutationStage 100 113156 2142964
MutationStage 100 149063 2142964
MutationStage 100 187514 2142964
MutationStage 100 226238 2142964
MutationStage 100 146267 2264194
MutationStage 100 141232 2314345
MutationStage 100 129730 2366987
MutationStage 100 128580 2412014
MutationStage 100 124101 2460093
MutationStage 100 119032 2509960
MutationStage 100 126888 2538537
MutationStage 100 163049 2538537
MutationStage 100 197243 2538537
MutationStage 100 231564 2538537
MutationStage 100 95212 2675457
MutationStage 100 43066 2727606
MutationStage 26 127 2779756
MutationStage 100 1115 2822694
MutationStage 22 22 2873449
I have two issues here.
- The massive amount of mutation caused by the hints playback
- The long periods where there are no increase in completed mutations.
Yes, there are other things going here when this happens, the system is
actually fairly busy, but has no problem handling the other traffic. The
RoundRobin Scheduler is also activated with 100 as throttle limit on a 12
node cluster, so there is no way at all that you should see 30k mutations
increasing on the pending side from the feeding side. This seems completely
triggered by the hints coming in.
I tried to reduce PAGE_SIZE to 5000 and set the throttle limit all the way
up to 5000ms, still it does not really seem to help much. It just take
longer time and it seems to be memtable flushing that blocks things.
Terje
On Thu, Apr 28, 2011 at 3:09 AM, Jonathan Ellis <jb...@gmail.com> wrote:
> MPF is indeed pretty lightweight, but since its job is to mark the
> commitlog replay position after a flush -- which has to be done in
> flush order to preserve correctness in failure scenarios -- you'll see
> the pending op count go up when you have multiple flushes happening.
> This is expected.
>
> Your real problem is the 17000 pending mutations, the 22 active +
> pending flushes, and probably compaction activity as well.
>
> (Also, if each of those pending mutations is 10,000 columns, you may
> be causing yourself memory pressure as well.)
>
> On Wed, Apr 27, 2011 at 11:01 AM, Terje Marthinussen
> <tm...@gmail.com> wrote:
> > 0.8 trunk:
> >
> > When playing back a fairly large chunk of hints, things basically locks
> up
> > under load.
> > The hints are never processed successfully. Lots of Mutations dropped.
> >
> > One thing is that maybe the default 10k columns per send with 50ms delays
> is
> > a bit on the aggressive side (10k*20 =200.000 columns in a second?), the
> > other thing is that it seems like the whole memtable flushing locks up.
> >
> > I tried to increase number of memtable flushers and queue a bit (8
> > concurrent flushers) to make things work, but no luck.
> >
> > Pool Name Active Pending Completed
> > ReadStage 0 0 1
> > RequestResponseStage 0 0 2236304
> > MutationStage 100 17564 4011533
> > ReadRepairStage 0 0 0
> > ReplicateOnWriteStage 0 0 0
> > GossipStage 0 0 2281
> > AntiEntropyStage 0 0 0
> > MigrationStage 0 0 0
> > MemtablePostFlusher 1 13 50
> > StreamStage 0 0 0
> > FlushWriter 8 14 73
> > MiscStage 0 0 0
> > FlushSorter 0 0 0
> > InternalResponseStage 0 0 0
> > HintedHandoff 1 8 3
> >
> > A quick source code scan makes me believe that the MemtablePostFlusher
> > should not normally use a lot of time, but it seem like it does so here.
> > What may cause this?
> >
> > Terje
> >
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com
>
Re: memtablePostFlusher blocking writes?
Posted by Jonathan Ellis <jb...@gmail.com>.
MPF is indeed pretty lightweight, but since its job is to mark the
commitlog replay position after a flush -- which has to be done in
flush order to preserve correctness in failure scenarios -- you'll see
the pending op count go up when you have multiple flushes happening.
This is expected.
Your real problem is the 17000 pending mutations, the 22 active +
pending flushes, and probably compaction activity as well.
(Also, if each of those pending mutations is 10,000 columns, you may
be causing yourself memory pressure as well.)
On Wed, Apr 27, 2011 at 11:01 AM, Terje Marthinussen
<tm...@gmail.com> wrote:
> 0.8 trunk:
>
> When playing back a fairly large chunk of hints, things basically locks up
> under load.
> The hints are never processed successfully. Lots of Mutations dropped.
>
> One thing is that maybe the default 10k columns per send with 50ms delays is
> a bit on the aggressive side (10k*20 =200.000 columns in a second?), the
> other thing is that it seems like the whole memtable flushing locks up.
>
> I tried to increase number of memtable flushers and queue a bit (8
> concurrent flushers) to make things work, but no luck.
>
> Pool Name Active Pending Completed
> ReadStage 0 0 1
> RequestResponseStage 0 0 2236304
> MutationStage 100 17564 4011533
> ReadRepairStage 0 0 0
> ReplicateOnWriteStage 0 0 0
> GossipStage 0 0 2281
> AntiEntropyStage 0 0 0
> MigrationStage 0 0 0
> MemtablePostFlusher 1 13 50
> StreamStage 0 0 0
> FlushWriter 8 14 73
> MiscStage 0 0 0
> FlushSorter 0 0 0
> InternalResponseStage 0 0 0
> HintedHandoff 1 8 3
>
> A quick source code scan makes me believe that the MemtablePostFlusher
> should not normally use a lot of time, but it seem like it does so here.
> What may cause this?
>
> Terje
>
--
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com