You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by kishorvpatil <gi...@git.apache.org> on 2015/10/12 18:34:03 UTC

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

GitHub user kishorvpatil opened a pull request:

    https://github.com/apache/storm/pull/795

    [STORM-1106] Netty should not limit attempts to reconnect

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kishorvpatil/incubator-storm storm1106

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/795.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #795
    
----
commit 7b4f4d7b71e72c5e5be24de8e70e893ebb741ec2
Author: Kishor Patil <kp...@yahoo-inc.com>
Date:   2015-10-12T16:31:24Z

    Netty should not limit attempts to reconnect

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147460971
  
    +1 pending travis


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147801288
  
    +1 I would be good to address [STORM-1107](https://issues.apache.org/jira/browse/STORM-1107) before doing a subsequent release.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147459755
  
    - Updated code to mark the configuration as deprecated
    - Remove private variable.
    - Created follow-up jira https://issues.apache.org/jira/browse/STORM-1107



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148208528
  
    1. Nimbus would not reassign anything because - workers come-up and heartbeat correctly ( as it is independent thread.)
    2. If workerA runs out of attempts to WorkerB ( because workerB was delayed in launching..) The workerA simply hangs..)
    
    This patch simply helps resolve that bottleneck for trying to connect forever.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148209299
  
    Also, there is bigger problem with `HashedWheelTimer` has bigger problems when worker has large number of connections to make and the remote workers are not accepting connections. At least for version of netty we are using there have been reports of it cracking over high CPU usage - i.e. once I saw `HashedWheelTimer` cause worker hangups. I am in the process of creating a reproducible test and solution for that. But we don't have to hold back 0.10.x release for that.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148217847
  
    Seems like default value of storm.messaging.netty.max_wait_ms is very small.
    
    I didn't see the details about StormBoundedExponentialBackoffRetry, but if it is running as intended via class comment, we're sleeping between 100 ~ 1000 ms for each retry by default.
    But we're pinning max retry count to 29 from ExponentialBackoffRetry.validateMaxRetries() since we use 1 << retryCount to sleep, resulting that we only sleep about max 29 secs, which is too short.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148222144
  
    @kishorvpatil 
    Yeah, after this PR gets merged. :)
    Thinking it once more, since default nimbus.task.timeout.secs is 30, retrying for each 1 secs seems not be short.
    Just curious about adjusting max_wait_ms may affect HashedWheelTimer's CPU usage as you stated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/795#discussion_r41903879
  
    --- Diff: storm-core/src/jvm/backtype/storm/messaging/netty/Client.java ---
    @@ -134,7 +128,7 @@
             LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize);
             int messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144);
     
    -        maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
    +        int maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
    --- End diff --
    
    > It would be nice to deprecate this and file a follow on JIRA so that we can use a better named/documented config in the backoff calculation
    
    Disregard: I see from the above comment what we intend to do.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by d2r <gi...@git.apache.org>.
Github user d2r commented on a diff in the pull request:

    https://github.com/apache/storm/pull/795#discussion_r41902116
  
    --- Diff: storm-core/src/jvm/backtype/storm/messaging/netty/Client.java ---
    @@ -134,7 +128,7 @@
             LOG.info("creating Netty Client, connecting to {}:{}, bufferSize: {}", host, port, bufferSize);
             int messageBatchSize = Utils.getInt(stormConf.get(Config.STORM_NETTY_MESSAGE_BATCH_SIZE), 262144);
     
    -        maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
    +        int maxReconnectionAttempts = Utils.getInt(stormConf.get(Config.STORM_MESSAGING_NETTY_MAX_RETRIES));
    --- End diff --
    
    If this config does not make sense anymore, should we stop using it in the code?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147456715
  
    The change itself looks OK, but I would like us to remove the maxConnectionAttemtps member variable.  It is not needed anymore, and it scares me that we will regress again with it there.  Additionally we should look at updating the config.
    
    ```
        /**
         * Netty based messaging: The max # of retries that a peer will perform when a remote is not accessible
         */
        public static final String STORM_MESSAGING_NETTY_MAX_RETRIES = "storm.messaging.netty.max_retries";
        public static final Object STORM_MESSAGING_NETTY_MAX_RETRIES_SCHEMA = ConfigValidation.IntegerValidator;
    ```
    
    Describes what the config was doing, but that is fundamentally flawed.  It would be nice to deprecate this and file a follow on JIRA so that we can use a better named/documented config in the backoff calculation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148210871
  
    @kishorvpatil 
    Thanks for the detail. I didn't think about situation 2. :)
    +1


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147739297
  
    The Travis build failure on 1.8 seems unrelated.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by kishorvpatil <gi...@git.apache.org>.
Github user kishorvpatil commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-148219849
  
    @HeartSaVioR, your understanding about `StormBoundedExponentialBackoffRetry` is correct, the 29 seconds is decent time - in case where workerB restarts due to any reason - during regular set-up.  But initial startup ( requires supervisor to read assignments/download code/launch workers.) which could mean much more than that. 
    
    The `StormBoundedExponentialBackoffRetry` lets you go beyond max attempts to give you 1sec sleep until you succeed. So, the workerA should keep trying unless nimbus informs workerA that workerB ( is no longer working on that port.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/795


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1106] Netty should not limit attempts t...

Posted by HeartSaVioR <gi...@git.apache.org>.
Github user HeartSaVioR commented on the pull request:

    https://github.com/apache/storm/pull/795#issuecomment-147896620
  
    @kishorvpatil 
    For confirming that I understood this issue well, I'd like you to elaborate this issue some more.
    
    As far as I understand, when retry count exceeds, Connect.run() throws RuntimeException, but worker doesn't be killed since it is a TimerTask.
    So it just closes the connection and wait for reassign for such worker.
    
    If Nimbus reassigns dead worker to another after retry limit exceed, another connection is being made and it would be fine.
    But some reason if problematic worker is just not able to connect (for example, STW, and so on) to another workers for longer than connection retrying but not forever, and nimbus doesn't reassign problematic worker, another workers cannot connect to problematic worker forever.
    
    Is my assumption right? Or there's other reason?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---