-
-
Notifications
You must be signed in to change notification settings - Fork 727
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
Do not drop BatchedSend payload if worker reconnects #5457
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||||
---|---|---|---|---|---|---|---|---|
@@ -1,12 +1,14 @@ | ||||||||
import logging | ||||||||
from collections import deque | ||||||||
from uuid import uuid4 | ||||||||
|
||||||||
from tornado import gen, locks | ||||||||
from tornado.ioloop import IOLoop | ||||||||
|
||||||||
import dask | ||||||||
from dask.utils import parse_timedelta | ||||||||
|
||||||||
from .comm import Comm | ||||||||
from .core import CommClosedError | ||||||||
|
||||||||
logger = logging.getLogger(__name__) | ||||||||
|
@@ -43,6 +45,7 @@ def __init__(self, interval, loop=None, serializers=None): | |||||||
self.interval = parse_timedelta(interval, default="ms") | ||||||||
self.waker = locks.Event() | ||||||||
self.stopped = locks.Event() | ||||||||
self.stopped.set() | ||||||||
self.please_stop = False | ||||||||
self.buffer = [] | ||||||||
self.comm = None | ||||||||
|
@@ -56,12 +59,33 @@ def __init__(self, interval, loop=None, serializers=None): | |||||||
self.serializers = serializers | ||||||||
self._consecutive_failures = 0 | ||||||||
|
||||||||
def start(self, comm): | ||||||||
self.comm = comm | ||||||||
self.loop.add_callback(self._background_send) | ||||||||
def start(self, comm: Comm): | ||||||||
""" | ||||||||
Start the BatchedSend by providing an open Comm object. | ||||||||
|
||||||||
Calling this again on an already started BatchedSend will raise a | ||||||||
`RuntimeError` if the provided Comm is different to the current one. If | ||||||||
the provided Comm is identical this is a noop. | ||||||||
|
||||||||
In case the BatchedSend was already closed, this will use the newly | ||||||||
provided Comm to submit any accumulated messages in the buffer. | ||||||||
""" | ||||||||
if self.closed(): | ||||||||
if comm.closed(): | ||||||||
raise RuntimeError("Comm already closed.") | ||||||||
self.comm = comm | ||||||||
self.please_stop = False | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
although I also find the use of |
||||||||
self.loop.add_callback(self._background_send) | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If
I'm not actually sure if, with the exact way the code is written now, two coroutines running at once can actually do something bad, but it still seems like a bad and brittle situation. "only one This is another way in which having an asyncio Task handle on the |
||||||||
elif self.comm is not comm: | ||||||||
raise RuntimeError("BatchedSend already started.") | ||||||||
|
||||||||
def closed(self): | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I still think this check is insufficient for telling whether we're closed or not. While we're in the process of closing ( From the way this is used though, in both |
||||||||
return self.comm and self.comm.closed() | ||||||||
"""True if the BatchedSend hasn't been started or has been closed | ||||||||
already.""" | ||||||||
if self.comm is None or self.comm.closed(): | ||||||||
return True | ||||||||
else: | ||||||||
return False | ||||||||
|
||||||||
def __repr__(self): | ||||||||
if self.closed(): | ||||||||
|
@@ -98,6 +122,8 @@ def _background_send(self): | |||||||
else: | ||||||||
self.recent_message_log.append("large-message") | ||||||||
self.byte_count += nbytes | ||||||||
|
||||||||
payload.clear() # lose ref | ||||||||
except CommClosedError: | ||||||||
logger.info("Batched Comm Closed %r", self.comm, exc_info=True) | ||||||||
break | ||||||||
|
@@ -111,28 +137,35 @@ def _background_send(self): | |||||||
logger.exception("Error in batched write") | ||||||||
break | ||||||||
finally: | ||||||||
payload = None # lose ref | ||||||||
# If anything failed we should not loose payload. If a new comm | ||||||||
fjetter marked this conversation as resolved.
Show resolved
Hide resolved
|
||||||||
# is provided we can still resubmit messages | ||||||||
self.buffer = payload + self.buffer | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are you sure we should retain Though maybe we'd rather duplicate messages than drop them. In that case, let's add a note saying so. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. At this point I am assuming a certain behaviour of the comm. Either the comm writes all or nothing. That's likely not always true but I believe we cannot do much about it on this level of abstraction. imho, that guarantee should be implemented by our protocol and/or Comm interface. Either way, I'm happy if you have any suggestions to improve this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. After reading through TCP comm code, I don't think there's anywhere for an exception to happen after all of the payload has been sent. An exception could still happen when part has been sent and part hasn't, but either way, since we have to assume something here, I think it's more useful to assume that the payload hasn't been sent. @jcrist and I were discussing this, and given the way the BatchedSend interface works, it actually needs to implement some sort of protocol with an ack from the receiver for each sent batch to guarantee messages can't be dropped. Since OTOH there are lots of protocols out there for doing things like this, more performantly, robustly, and with better testing than we'll ever have. As with other things (framing in serialization), maybe the better solution is to stop duplicating functionality at the application level that should be the transport layer's job. Could we get rid of BatchedSend entirely with some well-tuned TCP buffering settings + a serialization scheme that was more efficient for many small messages? |
||||||||
else: | ||||||||
# nobreak. We've been gracefully closed. | ||||||||
self.stopped.set() | ||||||||
return | ||||||||
|
||||||||
self.stopped.set() | ||||||||
# If we've reached here, it means `break` was hit above and | ||||||||
# there was an exception when using `comm`. | ||||||||
# We can't close gracefully via `.close()` since we can't send messages. | ||||||||
# So we just abort. | ||||||||
# This means that any messages in our buffer our lost. | ||||||||
# To propagate exceptions, we rely on subsequent `BatchedSend.send` | ||||||||
# calls to raise CommClosedErrors. | ||||||||
self.stopped.set() | ||||||||
self.abort() | ||||||||
|
||||||||
if self.comm: | ||||||||
self.comm.abort() | ||||||||
yield self.close() | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why
Not that I actually think we should move all shut-down/clean-up logic into the coroutine here, remove |
||||||||
|
||||||||
def send(self, *msgs): | ||||||||
"""Schedule a message for sending to the other side | ||||||||
|
||||||||
This completes quickly and synchronously | ||||||||
This completes quickly and synchronously. | ||||||||
|
||||||||
If the BatchedSend or Comm is already closed, this raises a | ||||||||
CommClosedError and does not accept any further messages to the buffer. | ||||||||
""" | ||||||||
if self.comm is not None and self.comm.closed(): | ||||||||
if self.closed(): | ||||||||
raise CommClosedError(f"Comm {self.comm!r} already closed.") | ||||||||
|
||||||||
self.message_count += len(msgs) | ||||||||
|
@@ -143,7 +176,7 @@ def send(self, *msgs): | |||||||
|
||||||||
@gen.coroutine | ||||||||
def close(self, timeout=None): | ||||||||
"""Flush existing messages and then close comm | ||||||||
"""Flush existing messages and then close Comm | ||||||||
|
||||||||
If set, raises `tornado.util.TimeoutError` after a timeout. | ||||||||
""" | ||||||||
|
@@ -153,21 +186,27 @@ def close(self, timeout=None): | |||||||
self.waker.set() | ||||||||
yield self.stopped.wait(timeout=timeout) | ||||||||
if not self.comm.closed(): | ||||||||
payload = [] | ||||||||
try: | ||||||||
if self.buffer: | ||||||||
self.buffer, payload = [], self.buffer | ||||||||
yield self.comm.write( | ||||||||
payload, serializers=self.serializers, on_error="raise" | ||||||||
) | ||||||||
except CommClosedError: | ||||||||
pass | ||||||||
# If we're closing and there is an error there is little we | ||||||||
# can do about this to recover. | ||||||||
logger.error("Lost %i payload messages.", len(payload)) | ||||||||
yield self.comm.close() | ||||||||
Comment on lines
188
to
200
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I find it odd that we try to write within Basically, I'd prefer it if self.next_deadline = None
self.waker.set()
await self.stopped.wait() could guarantee that the buffer flushed. The coroutine's whole job is to flush out the buffer when it's awakened and past its deadline; why duplicate that logic elsewhere? I believe that tightening up the logic for |
||||||||
|
||||||||
def abort(self): | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should |
||||||||
"""Close the BatchedSend immediately, without waiting for any pending | ||||||||
operations to complete. Buffered data will be lost.""" | ||||||||
if self.comm is None: | ||||||||
return | ||||||||
self.please_stop = True | ||||||||
self.buffer = [] | ||||||||
self.please_stop = True | ||||||||
self.waker.set() | ||||||||
if not self.comm.closed(): | ||||||||
self.comm.abort() | ||||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. comment for both here and |
||||||||
self.comm = None |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1,3 +1,5 @@ | ||
import asyncio | ||
|
||
import ctypes | ||
import errno | ||
import functools | ||
|
@@ -290,7 +292,15 @@ async def write(self, msg, serializers=None, on_error="message"): | |
stream._total_write_index += each_frame_nbytes | ||
|
||
# start writing frames | ||
stream.write(b"") | ||
await stream.write(b"") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm confused, why were we not awaiting this before?! I traversed through the blames and it looks like https://github.com/dask/distributed/pull/661/files#diff-581957c552b88dd04319efca1429c0c0827daff509a115542e70b661c4c04914R235 5 years ago is where we stopped awaiting this future. There was even some conversation about it: #661 (comment), xref #653. Seems like there were some issues around concurrent writes to the same stream from multiple coroutines producing orphaned futures. I wonder if that's still the case? |
||
# FIXME: How do I test this? Why is the stream closed _sometimes_? | ||
# Diving into tornado, so far, I can only confirm that once the | ||
# write future has been awaited, the entire buffer has been written | ||
# to the socket. Not sure if one loop iteration is sufficient in | ||
# general or just sufficient for the local tests I've been running | ||
await asyncio.sleep(0) | ||
if stream.closed(): | ||
raise StreamClosedError() | ||
Comment on lines
+295
to
+303
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixing this problem, reusing the buffer, etc. is relatively easy to do assuming we can rely on the FWIW, I think even without this There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @fjetter to rephrase what you're saying:
I agree. Like I mentioned in another comment, I think we need a better protocol (either application-level or more careful use of TCP) to guarantee all messages are delivered through reconnects. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's what I figured but I was confused why I never encountered this for ordinary comms. |
||
except StreamClosedError as e: | ||
self.stream = None | ||
self._closed = True | ||
|
@@ -333,6 +343,8 @@ def abort(self): | |
stream.close() | ||
|
||
def closed(self): | ||
if self.stream and self.stream.closed(): | ||
self.abort() | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems odd to me that checking There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Well, technically this sets |
||
return self._closed | ||
|
||
@property | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.