Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Retry operations on network issues #3294

Merged
merged 2 commits into from
Dec 5, 2019

Conversation

jochen-ott-by
Copy link
Contributor

We operate distributed in the cloud and see tcp connection aborts. Unfortunately, distributed often does not recover cleanly from such situations, although a simple re-try would have helped in most cases.
This PR proposes to add a more generic retry to some operations.

Notes:

  • only some operations are re-tried, as for some operations, triggering it twice may have undesired effects. There are probably more operations that can / should be re-tried, so this is just a start for operations where it's "obviously" safe to retry.
  • parameters for the re-tries (maximum number of retry attempts, delay between re-tries) is configurable. The default is to not re-try at all to not change the current behavior (some might rely on/prefer seeing all connection failures, fast)

@jochen-ott-by jochen-ott-by force-pushed the retry-operations branch 2 times, most recently from 5a0cc70 to e81b9bc Compare December 3, 2019 10:59
@mrocklin
Copy link
Member

mrocklin commented Dec 3, 2019

Thank you @byjott . In principle this seems fine to me. I'm glad to have you all making sure that Dask operates well in situations where network connections are unstable.

What do you think about adding a retry= keyword directly into the rpc or ConnectionPool API?

await self.scheduler.foo(..., retry=True)

(not a requirement, it's an honest question)

@jochen-ott-by
Copy link
Contributor Author

jochen-ott-by commented Dec 4, 2019

Thanks for looking into this.

What do you think about adding a retry= keyword directly into the rpc or ConnectionPool API?

await self.scheduler.foo(..., retry=True)

Actually, I did this as the first implementation. It would also work and be more concise at the calling site.
I only changed it into this one after realizing that:

  • IMO, this breaks the cleanliness and separation of the API: currently, all arguments to foo are forwarded to the scheduler process, as part of the rpc. However, any retry parameter would not be forwarded but rather take effect on a different level. This mixing of parameters that are forwarded and some that are interpreted at some intermediate level goes against the "principle of least surpris".
    As a side-effect, you would not be able to actually have a parameter called retry that you want forwarded to the scheduler. (This might not be a practical limitation, but still such a limitation feels arbitrary and surprising).
  • not all of the operations we want to re-try would go through the PooledRPCCall interface (see e.g. gather_deps_from_worker), so we need a more generic async wrapper that retries (or something similarly generic) anyway, so why not just use only that
  • As far as I understand the code, Client.scheduler can either be a rpc or a PooledRPCCall, and I didn't quite get whether there are more options, so such a change would need to touch this (implicit) interface that rpc and PooledRPCCall implement. I avoided the risk of missing some class here and opted for the more explicit way to wrap the calls instead of intrusively adding parameters.

@fjetter
Copy link
Member

fjetter commented Dec 4, 2019

I'm wondering if there are any scenarios where we would not like to retry IOError/EnvironmentErrors, i.e. what about adding this retry mechanism to the ConnectionPool/rpc without adding the explicit parameter?

@jochen-ott-by
Copy link
Contributor Author

I'm wondering if there are any scenarios where we would not like to retry IOError/EnvironmentErrors, i.e. what about adding this retry mechanism to the ConnectionPool/rpc without adding the explicit parameter?

I though about that, but as I wrote above: "only some operations are re-tried, as for some operations, triggering it twice may have undesired effects". To give an example: I'm not sure whether we really want to re-try a scheduler "rebalance" operation, as this could lead to two "rebalance" operations being active at the same time, and I'm not sure the code was written to make this safe. Similar thoughts hold for other operations.
I guess one could make it safe by generating a unique id on the client side for each semantically distinct request and let the server ignore the second request with the same id (or better yet: send the same answer if need be). I think this would be safe, but it would also require a major refactoring.

retry_operations: # some operations (such as gathering data) are subject to re-tries with the below parameters
max_retries: 0 # the maximum number to retry an operation in case of a connection problem
base_delay: 1s # the first non-zero delay between re-tries
max_delay: 20s # the maximum delay between re-tries
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I recommend the following names:

retry:
  count: 0
  delay:
    min: 1s
    max: 20s

Also, if you want to keep with multi-word names then I recommend the use of hyphens over underscores. Mostly my reasoning here is for consistency with the names that are currently here.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Alright, I just changed it to what you proposed. Thanks for the quick review!

@mrocklin
Copy link
Member

mrocklin commented Dec 5, 2019

Other than the small comment about the configuration names I'm happy to merge what's here. Thanks @byjott for your efforts here.

@mrocklin mrocklin merged commit 4e9eb46 into dask:master Dec 5, 2019
@mrocklin
Copy link
Member

mrocklin commented Dec 5, 2019

Thank you for your effort here @byjott . This is in.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants