[
https://issues.apache.org/jira/browse/STORM-677?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Michael Noll updated STORM-677:
-------------------------------
Description:
h3. Background
Storm currently supports the configuration setting
storm.messaging.netty.max_retries. This setting is supposed to limit the
number of reconnection attempts a Netty client will perform in case of a
connection loss.
Unfortunately users have run into situations where this behavior will result in
data loss:
{quote}
https://github.com/apache/storm/pull/429/files#r24681006
This could be a separate JIRA, but we ran into a situation where we hit the
maximum number of reconnection attempts, and the exception was eaten because it
was thrown from a background thread and it just killed the background thread.
This code appears to do the same thing.
{quote}
The problem can be summarized by the following example: Once a Netty client
hits the maximum number of connection retries, it will stop trying to reconnect
(as intended) but will also continue to run forever without being able to send
any messages to its designated remote targets. At this point data will be lost
because any messages that the Netty client is supposed to send will be dropped
(by design). And since the Netty client is still alive and thus considered
"functional", Storm is not able to do something about this data loss situation.
For a more detailed description please take a look at the discussion in
https://github.com/apache/storm/pull/429/files#r24742354.
h3. Possible solutions
(Most of this section is copy-pasted from an [earlier discussion on this
problem|https://github.com/apache/storm/pull/429/files#r24742354].)
There are at least three approaches we may consider:
# Let the Netty client die if max retries is reached, so that the Storm task
has the chance to re-create a client and thus break out of the client's
discard-messages-forever state.
# Let the "parent" Storm task die if (one of its possibly many) Netty clients
dies, so that by restarting the task we'll also get a new Netty client.
# Remove the max retries semantics as well as the corresponding setting from
Storm's configuration. Here, a Netty client will continue to reconnect to a
remote destination forever. The possible negative impact of these reconnects
(e.g. number of TCP connection attempts in a cluster) are kept in check by our
exponential backoff policy for such connection retries.
My personal opinion on these three approaches:
- I do not like (1) because I feel it introduces potentially confusing
semantics: We keep having a max retries setting, but it is not really a hard
limit anymore. It rather becomes a "max retries until we recreate a Netty
client", and would also reset any exponential backoff strategy of the
"previous" Netty client instance (cf. StormBoundedExponentialBackoffRetry). If
we do want such resets (but I don't think we do at this point), then a cleaner
approach would be to implement such resetting inside the retry policy (again,
cf. StormBoundedExponentialBackoffRetry).
- I do not like (2) because a single "bad" Netty client would be able to take
down a Storm task, which among other things would also impact any other,
working Netty clients of the Storm task.
- Option (3) seems a reasonable approach, although it breaks backwards
compatibility with regard to Storm's configuration (because we'd now ignore
storm.messaging.netty.max_retries).
Here's initial feedback from other developers:
{quote}
https://github.com/apache/storm/pull/429/files#r24824540
revans2: I personally prefer option 3, no maximum number of reconnection
attempts. Having the client decide that it is done, before nimbus does feels
like it is asking for trouble.
{quote}
{quote}
https://github.com/ptgoetz
ptgoetz: I'm in favor of option 3 as well. I'm not that concerned about
storm.messaging.netty.max_retries being ignored. We could probably just log a
warning that that configuration option is deprecated and will be ignored if the
value is set.
{quote}
If we decide to go with option 3, then the essence of the fix is the following
modification of Client.java:
{code}
private boolean reconnectingAllowed() {
// BEFORE:
// return !closing && connectionAttempts.get() <=
(maxReconnectionAttempts + 1);
return !closing;
}
{code}
was:
h3. Background
Storm currently supports the configuration setting
storm.messaging.netty.max_retries. This setting is supposed to limit the
number of reconnection attempts a Netty client will perform in case of a
connection loss.
Unfortunately users have run into situations where this behavior will result in
data loss:
{quote}
https://github.com/apache/storm/pull/429/files#r24681006
This could be a separate JIRA, but we ran into a situation where we hit the
maximum number of reconnection attempts, and the exception was eaten because it
was thrown from a background thread and it just killed the background thread.
This code appears to do the same thing.
{quote}
The problem can be summarized by the following example: Once a Netty client
hits the maximum number of connection retries, it will stop trying to reconnect
(as intended) but will also continue to run forever without being able to send
any messages to its designated remote targets. At this point data will be lost
because any messages that the Netty client is supposed to send will be dropped
(by design). And since the Netty client is still alive and thus considered
"functional", Storm is not able to do something about this data loss situation.
For a more detailed description please take a look at the discussion in
https://github.com/apache/storm/pull/429/files#r24742354.
h3. Possible solutions
(Most of this section is copy-pasted from an [earlier discussion on this
problem|https://github.com/apache/storm/pull/429/files#r24742354].)
There are at least three approaches we may consider:
# Let the Netty client die if max retries is reached, so that the Storm task
has the chance to re-create a client and thus break out of the client's
discard-messages-forever state.
# Let the "parent" Storm task die if (one of its possibly many) Netty clients
dies, so that by restarting the task we'll also get a new Netty client.
# Remove the max retries semantics as well as the corresponding setting from
Storm's configuration. Here, a Netty client will continue to reconnect to a
remote destination forever. The possible negative impact of these reconnects
(e.g. number of TCP connection attempts in a cluster) are kept in check by our
exponential backoff policy for such connection retries.
My personal opinion on these three approaches:
- I do not like (1) because I feel it introduces potentially confusing
semantics: We keep having a max retries setting, but it is not really a hard
limit anymore. It rather becomes a "max retries until we recreate a Netty
client", and would also reset any exponential backoff strategy of the
"previous" Netty client instance (cf. StormBoundedExponentialBackoffRetry). If
we do want such resets (but I don't think we do at this point), then a cleaner
approach would be to implement such resetting inside the retry policy (again,
cf. StormBoundedExponentialBackoffRetry).
- I do not like (2) because a single "bad" Netty client would be able to take
down a Storm task, which among other things would also impact any other,
working Netty clients of the Storm task.
- Option (3) seems a reasonable approach, although it breaks backwards
compatibility with regard to Storm's configuration (because we'd now ignore
storm.messaging.netty.max_retries).
Here's initial feedback from other developers:
{quote}
https://github.com/apache/storm/pull/429/files#r24824540
revans2: I personally prefer option 3, no maximum number of reconnection
attempts. Having the client decide that it is done, before nimbus does feels
like it is asking for trouble.
{quote}
{quote}
https://github.com/ptgoetz
ptgoetz: I'm in favor of option 3 as well. I'm not that concerned about
storm.messaging.netty.max_retries being ignored. We could probably just log a
warning that that configuration option is deprecated and will be ignored if the
value is set.
{quote}
> Maximum retries strategy may cause data loss
> --------------------------------------------
>
> Key: STORM-677
> URL: https://issues.apache.org/jira/browse/STORM-677
> Project: Apache Storm
> Issue Type: Bug
> Affects Versions: 0.9.3, 0.10.0
> Reporter: Michael Noll
> Priority: Minor
> Labels: Netty
>
> h3. Background
> Storm currently supports the configuration setting
> storm.messaging.netty.max_retries. This setting is supposed to limit the
> number of reconnection attempts a Netty client will perform in case of a
> connection loss.
> Unfortunately users have run into situations where this behavior will result
> in data loss:
> {quote}
> https://github.com/apache/storm/pull/429/files#r24681006
> This could be a separate JIRA, but we ran into a situation where we hit the
> maximum number of reconnection attempts, and the exception was eaten because
> it was thrown from a background thread and it just killed the background
> thread. This code appears to do the same thing.
> {quote}
> The problem can be summarized by the following example: Once a Netty client
> hits the maximum number of connection retries, it will stop trying to
> reconnect (as intended) but will also continue to run forever without being
> able to send any messages to its designated remote targets. At this point
> data will be lost because any messages that the Netty client is supposed to
> send will be dropped (by design). And since the Netty client is still alive
> and thus considered "functional", Storm is not able to do something about
> this data loss situation.
> For a more detailed description please take a look at the discussion in
> https://github.com/apache/storm/pull/429/files#r24742354.
> h3. Possible solutions
> (Most of this section is copy-pasted from an [earlier discussion on this
> problem|https://github.com/apache/storm/pull/429/files#r24742354].)
> There are at least three approaches we may consider:
> # Let the Netty client die if max retries is reached, so that the Storm task
> has the chance to re-create a client and thus break out of the client's
> discard-messages-forever state.
> # Let the "parent" Storm task die if (one of its possibly many) Netty clients
> dies, so that by restarting the task we'll also get a new Netty client.
> # Remove the max retries semantics as well as the corresponding setting from
> Storm's configuration. Here, a Netty client will continue to reconnect to a
> remote destination forever. The possible negative impact of these reconnects
> (e.g. number of TCP connection attempts in a cluster) are kept in check by
> our exponential backoff policy for such connection retries.
> My personal opinion on these three approaches:
> - I do not like (1) because I feel it introduces potentially confusing
> semantics: We keep having a max retries setting, but it is not really a hard
> limit anymore. It rather becomes a "max retries until we recreate a Netty
> client", and would also reset any exponential backoff strategy of the
> "previous" Netty client instance (cf. StormBoundedExponentialBackoffRetry).
> If we do want such resets (but I don't think we do at this point), then a
> cleaner approach would be to implement such resetting inside the retry policy
> (again, cf. StormBoundedExponentialBackoffRetry).
> - I do not like (2) because a single "bad" Netty client would be able to take
> down a Storm task, which among other things would also impact any other,
> working Netty clients of the Storm task.
> - Option (3) seems a reasonable approach, although it breaks backwards
> compatibility with regard to Storm's configuration (because we'd now ignore
> storm.messaging.netty.max_retries).
> Here's initial feedback from other developers:
> {quote}
> https://github.com/apache/storm/pull/429/files#r24824540
> revans2: I personally prefer option 3, no maximum number of reconnection
> attempts. Having the client decide that it is done, before nimbus does feels
> like it is asking for trouble.
> {quote}
> {quote}
> https://github.com/ptgoetz
> ptgoetz: I'm in favor of option 3 as well. I'm not that concerned about
> storm.messaging.netty.max_retries being ignored. We could probably just log a
> warning that that configuration option is deprecated and will be ignored if
> the value is set.
> {quote}
> If we decide to go with option 3, then the essence of the fix is the
> following modification of Client.java:
> {code}
> private boolean reconnectingAllowed() {
> // BEFORE:
> // return !closing && connectionAttempts.get() <=
> (maxReconnectionAttempts + 1);
> return !closing;
> }
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)