You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@storm.apache.org by Kevin Peek <kp...@salesforce.com> on 2016/12/01 14:22:19 UTC

Re: problem with shuffleGrouping

Ohad, I have been investigating further into this issue, and I think the
problem with shuffle grouping identified in STORM-2210 is not the same
problem you are seeing - it turned out not to be the root cause of our
issue. I think the actual problem has to do with the
LoadAwareShuffleGrouping, which storm uses by default in 1.x whenever you
add a grouping like:

setBolt(.....).shuffleGrouping("someStream");

You can disable the Load Aware behavior by setting the following key in
your config:

topology.disable.loadaware.messaging: true

Please let me know if you try this and if it helps.

*Note, you may see the key 'topology.disable.loadaware' in your topology
config already. This is mistake in the defaults.yaml file, and has already
been fixed for upcoming releases.

On Mon, Nov 21, 2016 at 4:28 PM, Stephen Powis <sp...@salesforce.com>
wrote:

> Sure.  I believe the JIRA ticket that Kevin linked above should resolve
> the issue, it seems like its a bug in the ShuffleGrouping implementation.
> We're about toss his fix into our production topologies and verify it helps
> even out the distribution.
>
> On Mon, Nov 21, 2016 at 4:18 PM, Ohad Edelstein <oh...@mintigo.com> wrote:
>
>> That exactly the behavior we are having, can I had the screen shot to the
>> stack overflow issue I opened?
>> Thanks!
>>
>> From: Stephen Powis <sp...@salesforce.com>
>> Reply-To: "user@storm.apache.org" <us...@storm.apache.org>
>> Date: Monday, 21 November 2016 at 22:11
>>
>> To: "user@storm.apache.org" <us...@storm.apache.org>
>> Subject: Re: problem with shuffleGrouping
>>
>> Hey Ohad,
>>
>> This isn't the best example as this compares two separate hosts that have
>> different uptimes... unfortunately I don't have an example on hand that
>> shows the same uptime.  We consistently see this behavior of this across
>> pretty much all of our bolts that use ShuffleGrouping, across all of our
>> topologies.
>>
>> In the screenshot below, the worker host which has been online longer has
>> Executed far fewer tuples.  These specific bolt instances are all feed from
>> a single upstream bolt using ShuffleGrouping.  For some reason the tasks on
>> this host just seem to be skipped over more often than not.  We're running
>> version 1.0.2.
>>
>> I'm not entirely sure if this is the same behavior that you're describing
>> or not?
>>
>> [image: Inline image 1]
>>
>>
>>
>> On Mon, Nov 21, 2016 at 3:00 PM, Ohad Edelstein <oh...@mintigo.com>
>> wrote:
>>
>>> Stephen, can you describe what the distribution that you are seeing?
>>>
>>> What we see is as follow:
>>> In storm version 0.9.3 we see that the machine with the spout works
>>> harder ( get almost twice the work then other machines )
>>>
>>> In storm version 1.0.1 the machine with the spout actually gets work to
>>> do ( with the shuffleGrouping we only see a few task submitted to the bolts
>>> ).
>>> Again according to the documentation noneGrouping should works the same
>>> as shuffleGrouping, But we see that noneGrouping distributes the result
>>> better.
>>>
>>> I didn’t find any complaints on the web to that, so I guess that this
>>> issue has something to do with what we do.
>>>
>>>
>>> From: Kevin Peek <kp...@salesforce.com>
>>> Reply-To: "user@storm.apache.org" <us...@storm.apache.org>
>>> Date: Monday, 21 November 2016 at 19:47
>>> To: "user@storm.apache.org" <us...@storm.apache.org>
>>> Subject: Re: problem with shuffleGrouping
>>>
>>> I played around a little bit with Stephen's test and it seems that the
>>> Collection.shuffle() call here is causing the problem (at least the problem
>>> Stephen is talking about). https://github.com/apa
>>> che/storm/blob/1.0.x-branch/storm-core/src/jvm/org/apache/st
>>> orm/grouping/ShuffleGrouping.java#L58
>>>
>>> I created a ticket to address this uneven task distribution:
>>> https://issues.apache.org/jira/browse/STORM-2210
>>>
>>> On Mon, Nov 21, 2016 at 11:20 AM, Stephen Powis <sp...@salesforce.com>
>>> wrote:
>>>
>>>> So we've seen some weird distributions using ShuffleGrouping as well.
>>>> I noticed there's no test case for ShuffleGrouping and got curious.  Also
>>>> the implementation seemed overly complicated (in my head anyhow, perhaps
>>>> there's a reason for it?) so I put together a much more simple version of
>>>> round robin shuffling.
>>>>
>>>> Gist here: https://gist.github.com/Crim/61537958df65a5e13b3844b2d5e28cd
>>>> e
>>>>
>>>> Its possible I've setup my test cases incorrectly, but it seems like
>>>> when using multiple threads in my test ShuffleGrouping provides wildly
>>>> un-even distribution?  In the Javadocs above each test case I've pasted the
>>>> output that I get locally.
>>>>
>>>> Thoughts?
>>>>
>>>> On Sat, Nov 19, 2016 at 2:49 AM, Ohad Edelstein <oh...@mintigo.com>
>>>> wrote:
>>>>
>>>>> It happened to you also?
>>>>> We are upgrading from 0.9.3 to 1.0.1,
>>>>> In 0.9.3 we didn’t have that problem.
>>>>>
>>>>> But Ones I use localOrShuffle the messages are send only to the same
>>>>> machine.
>>>>>
>>>>> From: Chien Le <Ch...@ds-iq.com>
>>>>> Reply-To: "user@storm.apache.org" <us...@storm.apache.org>
>>>>> Date: Saturday, 19 November 2016 at 6:05
>>>>> To: "user@storm.apache.org" <us...@storm.apache.org>
>>>>> Subject: Re: Testing serializers with multiple workers
>>>>>
>>>>> Ohad,
>>>>>
>>>>>
>>>>> We found that we had to use localOrShuffle grouping in order to see
>>>>> activity in the same worker as the spout.
>>>>>
>>>>>
>>>>> -Chien
>>>>>
>>>>>
>>>>> ------------------------------
>>>>> *From:* Ohad Edelstein <oh...@mintigo.com>
>>>>> *Sent:* Friday, November 18, 2016 8:38:35 AM
>>>>> *To:* user@storm.apache.org
>>>>> *Subject:* Re: Testing serializers with multiple workers
>>>>>
>>>>> Hello,
>>>>>
>>>>> We just finished setting up storm 1.0.1 with 3 supervisors and one
>>>>> nimbus machine.
>>>>> Total of 4 machines in aws.
>>>>>
>>>>> We see the following phanomenon:
>>>>> lets say spout on host2,
>>>>> host1 - using 100% cpu
>>>>> host3 - using 100% cpu
>>>>> host2 - idle (some message are being handled by it, not many)
>>>>> its not slots problem, we have even amount of bolts.
>>>>>
>>>>> We also tried to deploy only 2 host, and the same thing happened, the
>>>>> host with the spout is idle, the other host at 100% cpu.
>>>>>
>>>>> We switched from shuffleGrouping to noneGrouping, and its seems to
>>>>> work,
>>>>> The documentation says that:
>>>>> None grouping: This grouping specifies that you don't care how the
>>>>> stream is grouped. Currently, none groupings are equivalent to shuffle
>>>>> groupings. Eventually though, Storm will push down bolts with none
>>>>> groupings to execute in the same thread as the bolt or spout they subscribe
>>>>> from (when possible).
>>>>>
>>>>> We are still trying to understand what is wrong with shuffleGrouping
>>>>> in our system,
>>>>>
>>>>> Any ideas?
>>>>>
>>>>> Thanks!
>>>>>
>>>>> From: Aaron Niskodé-Dossett <do...@gmail.com>
>>>>> Reply-To: "user@storm.apache.org" <us...@storm.apache.org>
>>>>> Date: Friday, 18 November 2016 at 17:04
>>>>> To: "user@storm.apache.org" <us...@storm.apache.org>
>>>>> Subject: Re: Testing serializers with multiple workers
>>>>>
>>>>> Hit send too soon... that really is the option :-)
>>>>>
>>>>> On Fri, Nov 18, 2016 at 9:03 AM Aaron Niskodé-Dossett <
>>>>> dossett@gmail.com> wrote:
>>>>>
>>>>>> topology.testing.always.try.serialize = true
>>>>>>
>>>>>> On Fri, Nov 18, 2016 at 8:57 AM Kristopher Kane <kk...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>> Does anyone have any techniques for testing serializers that would
>>>>>> only surface when the serializer is uses in a multi-worker topology?
>>>>>>
>>>>>> Kris
>>>>>>
>>>>>>
>>>>
>>>
>>
>

Re: problem with shuffleGrouping

Posted by Ohad Edelstein <oh...@mintigo.com>.
Hey Kevin,
Thanks for keep looking into it,
I will check and let you know.
Thanks

From: Kevin Peek <kp...@salesforce.com>>
Reply-To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Date: Thursday, 1 December 2016 at 16:22
To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Subject: Re: problem with shuffleGrouping

Ohad, I have been investigating further into this issue, and I think the problem with shuffle grouping identified in STORM-2210 is not the same problem you are seeing - it turned out not to be the root cause of our issue. I think the actual problem has to do with the LoadAwareShuffleGrouping, which storm uses by default in 1.x whenever you add a grouping like:

setBolt(.....).shuffleGrouping("someStream");

You can disable the Load Aware behavior by setting the following key in your config:

topology.disable.loadaware.messaging: true

Please let me know if you try this and if it helps.

*Note, you may see the key 'topology.disable.loadaware' in your topology config already. This is mistake in the defaults.yaml file, and has already been fixed for upcoming releases.

On Mon, Nov 21, 2016 at 4:28 PM, Stephen Powis <sp...@salesforce.com>> wrote:
Sure.  I believe the JIRA ticket that Kevin linked above should resolve the issue, it seems like its a bug in the ShuffleGrouping implementation.  We're about toss his fix into our production topologies and verify it helps even out the distribution.

On Mon, Nov 21, 2016 at 4:18 PM, Ohad Edelstein <oh...@mintigo.com>> wrote:
That exactly the behavior we are having, can I had the screen shot to the stack overflow issue I opened?
Thanks!

From: Stephen Powis <sp...@salesforce.com>>
Reply-To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Date: Monday, 21 November 2016 at 22:11

To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Subject: Re: problem with shuffleGrouping

Hey Ohad,

This isn't the best example as this compares two separate hosts that have different uptimes... unfortunately I don't have an example on hand that shows the same uptime.  We consistently see this behavior of this across pretty much all of our bolts that use ShuffleGrouping, across all of our topologies.

In the screenshot below, the worker host which has been online longer has Executed far fewer tuples.  These specific bolt instances are all feed from a single upstream bolt using ShuffleGrouping.  For some reason the tasks on this host just seem to be skipped over more often than not.  We're running version 1.0.2.

I'm not entirely sure if this is the same behavior that you're describing or not?

[Inline image 1]



On Mon, Nov 21, 2016 at 3:00 PM, Ohad Edelstein <oh...@mintigo.com>> wrote:
Stephen, can you describe what the distribution that you are seeing?

What we see is as follow:
In storm version 0.9.3 we see that the machine with the spout works harder ( get almost twice the work then other machines )

In storm version 1.0.1 the machine with the spout actually gets work to do ( with the shuffleGrouping we only see a few task submitted to the bolts ).
Again according to the documentation noneGrouping should works the same as shuffleGrouping, But we see that noneGrouping distributes the result better.

I didn’t find any complaints on the web to that, so I guess that this issue has something to do with what we do.


From: Kevin Peek <kp...@salesforce.com>>
Reply-To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Date: Monday, 21 November 2016 at 19:47
To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Subject: Re: problem with shuffleGrouping

I played around a little bit with Stephen's test and it seems that the Collection.shuffle() call here is causing the problem (at least the problem Stephen is talking about). https://github.com/apache/storm/blob/1.0.x-branch/storm-core/src/jvm/org/apache/storm/grouping/ShuffleGrouping.java#L58

I created a ticket to address this uneven task distribution: https://issues.apache.org/jira/browse/STORM-2210

On Mon, Nov 21, 2016 at 11:20 AM, Stephen Powis <sp...@salesforce.com>> wrote:
So we've seen some weird distributions using ShuffleGrouping as well.  I noticed there's no test case for ShuffleGrouping and got curious.  Also the implementation seemed overly complicated (in my head anyhow, perhaps there's a reason for it?) so I put together a much more simple version of round robin shuffling.

Gist here: https://gist.github.com/Crim/61537958df65a5e13b3844b2d5e28cde

Its possible I've setup my test cases incorrectly, but it seems like when using multiple threads in my test ShuffleGrouping provides wildly un-even distribution?  In the Javadocs above each test case I've pasted the output that I get locally.

Thoughts?

On Sat, Nov 19, 2016 at 2:49 AM, Ohad Edelstein <oh...@mintigo.com>> wrote:
It happened to you also?
We are upgrading from 0.9.3 to 1.0.1,
In 0.9.3 we didn’t have that problem.

But Ones I use localOrShuffle the messages are send only to the same machine.

From: Chien Le <Ch...@ds-iq.com>>
Reply-To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Date: Saturday, 19 November 2016 at 6:05
To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Subject: Re: Testing serializers with multiple workers


Ohad,


We found that we had to use localOrShuffle grouping in order to see activity in the same worker as the spout.


-Chien


________________________________
From: Ohad Edelstein <oh...@mintigo.com>>
Sent: Friday, November 18, 2016 8:38:35 AM
To: user@storm.apache.org<ma...@storm.apache.org>
Subject: Re: Testing serializers with multiple workers

Hello,

We just finished setting up storm 1.0.1 with 3 supervisors and one nimbus machine.
Total of 4 machines in aws.

We see the following phanomenon:
lets say spout on host2,
host1 - using 100% cpu
host3 - using 100% cpu
host2 - idle (some message are being handled by it, not many)
its not slots problem, we have even amount of bolts.

We also tried to deploy only 2 host, and the same thing happened, the host with the spout is idle, the other host at 100% cpu.

We switched from shuffleGrouping to noneGrouping, and its seems to work,
The documentation says that:
None grouping: This grouping specifies that you don't care how the stream is grouped. Currently, none groupings are equivalent to shuffle groupings. Eventually though, Storm will push down bolts with none groupings to execute in the same thread as the bolt or spout they subscribe from (when possible).

We are still trying to understand what is wrong with shuffleGrouping in our system,

Any ideas?

Thanks!

From: Aaron Niskodé-Dossett <do...@gmail.com>>
Reply-To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Date: Friday, 18 November 2016 at 17:04
To: "user@storm.apache.org<ma...@storm.apache.org>" <us...@storm.apache.org>>
Subject: Re: Testing serializers with multiple workers

Hit send too soon... that really is the option :-)

On Fri, Nov 18, 2016 at 9:03 AM Aaron Niskodé-Dossett <do...@gmail.com>> wrote:
topology.testing.always.try.se<http://topology.testing.always.try.se>rialize = true

On Fri, Nov 18, 2016 at 8:57 AM Kristopher Kane <kk...@gmail.com>> wrote:
Does anyone have any techniques for testing serializers that would only surface when the serializer is uses in a multi-worker topology?

Kris