-
-
Notifications
You must be signed in to change notification settings - Fork 719
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 1 commit
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 |
---|---|---|
|
@@ -58,6 +58,7 @@ def __init__(self, interval, loop=None, serializers=None): | |
|
||
def start(self, comm): | ||
self.comm = comm | ||
self.please_stop = False | ||
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. One downside of using Tornado here is that we have no handle on this coroutine. If we did
|
||
|
||
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 |
||
|
@@ -98,6 +99,7 @@ def _background_send(self): | |
else: | ||
self.recent_message_log.append("large-message") | ||
self.byte_count += nbytes | ||
payload = [] # lose ref | ||
except CommClosedError: | ||
logger.info("Batched Comm Closed %r", self.comm, exc_info=True) | ||
break | ||
|
@@ -111,7 +113,9 @@ 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() | ||
|
@@ -121,7 +125,6 @@ def _background_send(self): | |
# 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() | ||
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. Can't comment there, but this is a comment for if self.comm is not None and self.comm.closed():
raise CommClosedError(f"Comm {self.comm!r} already closed.") This check feels insufficient. A more thorough check would have caught #5481. Should we also check On the other hand, we may be relying on the fact that messages can be enqueued to a BatchedSend even when it's closed. If we expect the BatchedSend to be restarted and reused soon, perhaps there should be no restrictions on when I'd like to:
|
||
|
@@ -152,6 +155,7 @@ def close(self, timeout=None): | |
self.please_stop = True | ||
self.waker.set() | ||
yield self.stopped.wait(timeout=timeout) | ||
payload = [] | ||
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. could move this below the |
||
if not self.comm.closed(): | ||
try: | ||
if self.buffer: | ||
|
@@ -160,14 +164,15 @@ def close(self, timeout=None): | |
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() | ||
|
||
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 |
||
if self.comm is None: | ||
return | ||
self.please_stop = True | ||
self.buffer = [] | ||
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 |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2568,11 +2568,9 @@ def fast_on_a(lock): | |
|
||
assert "Unexpected worker completed task" in s_logs.getvalue() | ||
|
||
# Ensure that all in-memory tasks on A have been restored on the | ||
# scheduler after reconnect | ||
for ts in a.tasks.values(): | ||
if ts.state == "memory": | ||
assert a.address in {ws.address for ws in s.tasks[ts.key].who_has} | ||
sts = s.tasks[f3.key] | ||
assert sts.state == "memory" | ||
assert s.workers[a.address] in sts.who_has | ||
|
||
del f1, f2, f3 | ||
while any(w.tasks for w in [a, b]): | ||
|
@@ -3196,3 +3194,38 @@ async def test_deadlock_cancelled_after_inflight_before_gather_from_worker( | |
args, kwargs = mocked_gather.call_args | ||
await Worker.gather_dep(b, *args, **kwargs) | ||
await fut3 | ||
|
||
|
||
@gen_cluster(nthreads=[("", 1)]) | ||
async def test_dont_loose_payload_reconnect(s, w): | ||
fjetter marked this conversation as resolved.
Show resolved
Hide resolved
|
||
"""Ensure that payload of a BatchedSend is not lost if a worker reconnects""" | ||
s.count = 0 | ||
|
||
def receive(worker, msg): | ||
s.count += 1 | ||
|
||
s.stream_handlers["receive-msg"] = receive | ||
w.batched_stream.next_deadline = w.loop.time() + 10_000 | ||
|
||
for x in range(100): | ||
w.batched_stream.send({"op": "receive-msg", "msg": x}) | ||
|
||
await s.stream_comms[w.address].comm.close() | ||
while not w.batched_stream.comm.closed(): | ||
await asyncio.sleep(0.1) | ||
before = w.batched_stream.buffer.copy() | ||
w.batched_stream.next_deadline = w.loop.time() | ||
assert len(w.batched_stream.buffer) == 100 | ||
with captured_logger("distributed.batched") as caplog: | ||
await w.batched_stream._background_send() | ||
|
||
assert "Batched Comm Closed" in caplog.getvalue() | ||
after = w.batched_stream.buffer.copy() | ||
|
||
# Payload that couldn't be submitted is prepended | ||
assert len(after) >= len(before) | ||
assert after[: len(before)] == before | ||
Comment on lines
+3261
to
+3263
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 everything went through before the close, then you're not testing anything useful. Could you change to assert before)
assert len(after) > len(before) while possibly drastically increasing the number of messages in the pipeline to avoid flakiness? 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 the payload would've been submitted before closing, the buffer would be empty before I am trying to send. If we want to absolutely have 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. Can you explicitly call 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 been done as part of
To me, that's sensible behaviour |
||
|
||
await w.heartbeat() | ||
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 assume this is to make the worker reconnect to the scheduler. Is there a way we could wait for that to happen naturally, to test that it does so properly? Or do we have other good tests around worker reconnection that we're confident in? 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. Yes, the heartbeat triggers a reconnect. There are |
||
while not s.count == 100: | ||
await asyncio.sleep(0.1) |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1238,6 +1238,7 @@ async def handle_scheduler(self, comm): | |
comm, every_cycle=[self.ensure_communicating, self.ensure_computing] | ||
) | ||
except Exception as e: | ||
self.batched_stream.please_stop = True | ||
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. Could we not mutate internal state of BatchedSend? This feels brittle. I see this is going to do something slightly different from Though I'd prefer having just one 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 was actually not even necessary. The BatchedSend infers automatically that the Comm is closed and there is no need to close anything |
||
logger.exception(e) | ||
raise | ||
finally: | ||
|
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.
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.
What I mean by a "clear interface" is that I'd like a docstring for
start
saying the contract for what it can/can't do. For example, thatstart
on a closed BatchedSend will restart it with a new comm. And thatstart
on a currently-running BatchedSend raises an error, etc. What are the invariants for this internal state? How do all the methods ensure they are upheld?