-
-
Notifications
You must be signed in to change notification settings - Fork 757
Description
Real world network infrastructure is not as reliable as one would hope. Short network disconnects are a possibility. This is particularly true for mobile clients, home networks, etc. but is also not uncommon in busy professional or cloud networks.
Currently, our entire networking code interprets a disconnecting or broken Comm as a dead remote server and handles this accordingly. In case a Client <-> Scheduler connection is broken, both sides act radically by cancelling or releasing all futures and tasks, respectively.
If both the Client and Scheduler are still alive, allowing them to reestablish the connection and recover the system to its original, functional state without loosing progress can save time and money in ad-hoc execution scenarios and can significantly increase stability for (semi-)automated workflow.
The current drastic behaviour is necessary to allow for either sides to clean up their state in case the remote actually vanishes.
Allowing a graceful reconnect requires us to change the behaviour on Scheduler and Client side
Scheduler
- The scheduler should allow for a certain grace period during which a Client is allowed to reconnect.
- During this time period, the client is not operational. However, we do not differentiate states in ClientState and would likely need to introduce states, e.g. "running" and "lost" and verify all
ClientStateusages - We will need to buffer all messages sent to this client during that time period and resubmit them after connection. period? Examples,
Scheduler.send_all,Scheduler.report,Scheduler.cancel_key,Scheduler.restart
Note: A similar message problem appears in worker reconnect scenarios.
Client:
The Client needs to distinguish three different scenarios
- Reconnect attempt fails. Cancel all futures. Close.
- New scheduler appears. Again, we need to cancel and remove all local futures.
- Same scheduler reconnects. Restore state to before connection failure.
All three scenarios should log appropriate information to the user.
Expected behaviour
- Client can reestablish a connection to the same scheduler and continue its progress if that scheduler is still alive
- If the reconnect only happens to a new scheduler, the client needs to invalidate all stale futures
- If the Client is gone for good, the scheduler needs to release all tasks as it is doing right now
- During the outage period / grace period, no new futures are allowed to be created on client side
@gen_cluster(client=True)
async def test_reconnect_same_scheduler(c, s, a, b):
f1 = c.submit(inc, 1, key="f1")
c.scheduler_comm.abort() # E.g. external network blip
# TODO: Assert log messages about disconnect on level >=WARNING
assert await f1 == 2Related issues:
The current behaviour is not only not resilient but also leads to confusing behaviour due to the exception messages raised, see #5666