storm-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Martin Illecker <martin.illec...@gmail.com>
Subject Re: Urgent - Some workers stop processing after a few seconds
Date Sat, 28 Feb 2015 12:11:57 GMT
This issue does not occur when using one worker.
It only occurs when I use multiple workers and an unreliable spout with a
parallelism value of one.
Then some workers stop processing after a few seconds because the don't get
any new tuples.

But I changed my spout to be reliable and figured out a suitable
*maxSpoutPending* value to prevent tuple timeouts.
Now the topology runs also on multiple workers and they don't stop
processing.


2015-02-27 18:35 GMT+01:00 John Reilly <jr@inconspicuous.org>:

> I'm not sure what is going on here, but I wonder if it is some kind of
> serialization issue.  If you run all the same components with the same
> parallelism but use one worker do you see the same issue?
>
>
> On Thu, Feb 26, 2015 at 12:43 PM Martin Illecker <millecker@apache.org>
> wrote:
>
>> I have updated my spout to be reliable by emitting a *messageId*.
>>
>> But now the *spout* shows a lot of failed tuples. (see attachment)
>>
>> How can I fix this?
>> I'm using zeromq-2.1.7 on EC2.
>>
>>
>> 2015-02-26 18:36 GMT+01:00 Martin Illecker <millecker@apache.org>:
>>
>>> By the way, I'm using an *unreliable* spout.
>>> Here is the source of my spout:
>>> https://github.com/millecker/storm-apps/blob/master/commons/src/at/
>>> illecker/storm/commons/spout/DatasetSpout.java
>>>
>>> Maybe this might be the problem?
>>>
>>> 2015-02-26 18:30 GMT+01:00 Martin Illecker <millecker@apache.org>:
>>>
>>>> Hi,
>>>>
>>>> I believe this issue belongs to Storm or EC2 because on a single node
>>>> (one worker) my topology is operating fine.
>>>>
>>>> I have tried different combinations of the following parameters:
>>>>  - *shuffleGrouping* and *allGrouping* between the spout and the first
>>>> bolt
>>>>  - spout parallelism from 1 to numberOfWorkers (each worker has its own
>>>> spout task)
>>>>  - maxSpoutPending from 5000 down to 50
>>>>  - 1ms sleep in spout
>>>>
>>>> The issue occurs when one spout with parallelism 1 should feed multiple
>>>> workers.
>>>> For example, 5 workers including one spout with parallelism 1 and a
>>>> bolt with parallelism 5.
>>>> After a few seconds, 4 of these 5 workers become idle and only one
>>>> worker keeps processing.
>>>> This might be probably the worker including the spout task.
>>>>
>>>> If I increase the parallelism of the spout, then the performance drops
>>>> dramatically, but all workers keep working.
>>>>
>>>> There are no error messages in the worker or supervisor log.
>>>>
>>>> You've maxSpout pending set to 2k tuples do you see any where in your
>>>>> bolt code can be hanging before acking the tuple?.
>>>>
>>>> I thought I would receive an exception or a timeout if the bolt is
>>>> hanging?
>>>>
>>>> Please have a look a the full source of my topology:
>>>>
>>>> https://github.com/millecker/storm-apps/blob/master/sentiment_analysis_svm/src/at/illecker/storm/sentimentanalysis/svm/SentimentAnalysisSVMTopology.java
>>>>
>>>> Thanks!
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> 2015-02-26 17:31 GMT+01:00 Harsha <storm@harsha.io>:
>>>>
>>>>>  Martin,
>>>>>          Can't find anything wrong in the logs or in your
>>>>> topologyBuilder code. In your bolts code how are you doing the acking
of
>>>>> the tuples. You've maxSpout pending set to 2k tuples do you see any where
>>>>> in your bolt code can be hanging before acking the tuple?.
>>>>>
>>>>> -Harsha
>>>>>
>>>>> On Wed, Feb 25, 2015, at 09:02 AM, Martin Illecker wrote:
>>>>>
>>>>> How can I find out why workers do not get any tuples?
>>>>> After they have successfully processed a few thousand.
>>>>>
>>>>> I have also tested the *allGrouping* to ensure that each Bolt must
>>>>> receive tuples.
>>>>> But two workers including two Bolts stop receiving tuples after a few
>>>>> seconds.
>>>>>
>>>>> I would appreciate any help!
>>>>>
>>>>>
>>>>>
>>>>> 2015-02-25 17:40 GMT+01:00 Harsha <storm@harsha.io>:
>>>>>
>>>>>
>>>>> My bad was looking at another supervisor.log.  There are no errors in
>>>>> supervisor and worker logs.
>>>>>
>>>>>
>>>>> -Harsha
>>>>>
>>>>>
>>>>> On Wed, Feb 25, 2015, at 08:29 AM, Martin Illecker wrote:
>>>>>
>>>>> Hi Harsha,
>>>>>
>>>>> I'm using three c3.4xlarge EC2 instances:
>>>>>  1) Nimbus, WebUI, Zookeeper, Supervisor
>>>>>  2) Zookeeper, Supervisor
>>>>>  3) Zookeeper, Supervisor
>>>>>
>>>>> I cannot find this error message in my attached supervisor log?
>>>>> By the way, I'm running on Ubuntu EC2 nodes and there is no path C:\.
>>>>>
>>>>> I have not made any changes in these timeout values. Should be the
>>>>> default values:
>>>>> storm.zookeeper.session.timeout: 20000
>>>>> storm.zookeeper.connection.timeout: 15000
>>>>> supervisor.worker.timeout.secs: 30
>>>>>
>>>>> Thanks!
>>>>> Best regards
>>>>> Martin
>>>>>
>>>>>
>>>>> 2015-02-25 17:03 GMT+01:00 Harsha <storm@harsha.io>:
>>>>>
>>>>>
>>>>> Hi Martin,
>>>>>             Can you share your storm.zookeeper.session.timeout and
>>>>> storm.zookeeper.connection.timeout and supervisor.worker.timeout.secs.
By
>>>>> looking at the supervisor logs I see
>>>>> Error when processing event
>>>>> java.io.FileNotFoundException: File
>>>>> 'c:\hdistorm\workers\f3e70029-c5c8-4f55-a4a1-396096b37509\heartbeats\1417082031858'
>>>>>
>>>>> you might be running into
>>>>> https://issues.apache.org/jira/browse/STORM-682
>>>>> Is your zookeeper cluster on a different set of  nodes and can you
>>>>> check you are able to connect to it without any issues
>>>>> -Harsha
>>>>>
>>>>>
>>>>>
>>>>> On Wed, Feb 25, 2015, at 03:49 AM, Martin Illecker wrote:
>>>>>
>>>>> Hi,
>>>>>
>>>>> I'm still observing this strange issue.
>>>>> Two of three workers stop processing after a few seconds. (each worker
>>>>> is running on one dedicated EC2 node)
>>>>>
>>>>> My guess would be that the output stream of one spout is not properly
>>>>> distributed over all three workers.
>>>>> Or somehow directed to one worker only? But *shuffleGrouping* should
>>>>> guarantee equal distribution among multiple bolts right?
>>>>>
>>>>> I'm using the following topology:
>>>>>
>>>>>
>>>>> TopologyBuilder builder = new TopologyBuilder();
>>>>>
>>>>> builder.setSpout("dataset-spout", spout);
>>>>>
>>>>> builder.setBolt("tokenizer-bolt", tokenizerBolt, 3).shuffleGrouping(
>>>>>
>>>>> "dataset-spout");
>>>>>
>>>>> builder.setBolt("preprocessor-bolt", preprocessorBolt,
>>>>> 3).shuffleGrouping(
>>>>>
>>>>> "tokenizer-bolt");
>>>>>
>>>>> conf.setMaxSpoutPending(2000);
>>>>>
>>>>> conf.setNumWorkers(3);
>>>>>
>>>>>     StormSubmitter
>>>>>
>>>>>         .submitTopology(TOPOLOGY_NAME, conf, builder
>>>>> .createTopology());
>>>>>
>>>>> I have attached the screenshots of the topology and the truncated
>>>>> worker and supervisor log of one idle worker.
>>>>>
>>>>> The supervisor log includes a few interesting lines, but I think they
>>>>> are normal?
>>>>>
>>>>> supervisor [INFO] e76bc338-2ba5-444b-9854-bca94f9587b7 still hasn't
>>>>> started
>>>>>
>>>>> I hope, someone can help me with this issue!
>>>>>
>>>>> Thanks
>>>>> Best regards
>>>>> Martin
>>>>>
>>>>>
>>>>> 2015-02-24 20:37 GMT+01:00 Martin Illecker <millecker@apache.org>:
>>>>>
>>>>> Hi,
>>>>>
>>>>> I'm trying to run a topology on EC2, but I'm observing the following
>>>>> strange issue:
>>>>>
>>>>> Some workers stop processing after a few seconds, without any error in
>>>>> the worker log.
>>>>>
>>>>> For example, my topology consists of 3 workers and each worker is
>>>>> running on its own EC2 node.
>>>>> Two of them stop processing after a few seconds. But they have already
>>>>> processed several tuples successfully.
>>>>>
>>>>> I'm using only one spout and shuffleGrouping at all bolts.
>>>>> If I add more spouts then all workers keep processing, but the
>>>>> performance is very bad.
>>>>>
>>>>> Does anyone have a guess why this happens?
>>>>>
>>>>> The topology is currently running at:
>>>>> http://54.155.156.203:8080
>>>>>
>>>>> Thanks!
>>>>>
>>>>> Martin
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> Email had 4 attachments:
>>>>>
>>>>>    - topology.jpeg
>>>>>      161k (image/jpeg)
>>>>>    - component.jpeg
>>>>>      183k (image/jpeg)
>>>>>    - supervisor.log
>>>>>      7k (application/octet-stream)
>>>>>    - worker.log
>>>>>      37k (application/octet-stream)
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>
>>

Mime
View raw message