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

Deadlocks and infinite loops connected to failed dependencies #4360

Closed

Conversation

fjetter
Copy link
Member

@fjetter fjetter commented Dec 14, 2020

Two fixes in here which are unfortunately a bit intertwined.

  1. if send_rcv fails for whatever reason (e.g. deserialization error) but doesn't close the comm, the other end might still wait for a response. in the meantime this comm is resused, causing some weird behaviour, see also WIP Proper handling of application errors during get_data #4316 I'll still need to write a test capturing the behaviour described in there

  2. If we are in the case of a bad_dep, the current implementation never detects this and we're stuck in an infinite loop since the condition dep.suspicious_count > 5 is never hit. It is never hit because there are multiple calls to release-keysissued causing the worker to loose all its memory, therefore the suspicious count never climbs up but the task instance is reinitialized

@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from 56b6ccb to 9e29ace Compare December 14, 2020 13:39
@mrocklin
Copy link
Member

As always, thanks for identifying these tricky issues and reporting upstream. I haven't yet had a time to go through things deeply here (and that time may not come), but if you're confident in things here I'm happy to defer to you.

However, currently tests are failing, so something is going wrong I think.

@fjetter
Copy link
Member Author

fjetter commented Dec 14, 2020

Indeed, I will need to look deeper into the tests. I opened the PR a bit prematurely but I figured this is an area where a lot is happening lately and somebody else might have an opinion

@mrocklin
Copy link
Member

+1 on opening PRs before they're polished

Comment on lines 1479 to 1484
ts.runspec = runspec
if ts.state == "erred":
ts.exception = None
ts.traceback = None
else:
ts.state = "waiting"
self.transition(ts, "waiting", runspec=runspec)
Copy link
Member Author

Choose a reason for hiding this comment

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

Not setting the runspec for these cases can actually lead to a deadlock. I've been hunting this deadlock for a while now and I am very confident this is the root cause for #4133

I also have a suspicion that this is causing #4173 but I cannot reproduce locally

Fixes are still incomplete and I will need more time fixing CI since I cannot reproduce everything. I'm a bit confused about what the "waiting" state is actually supposed to flag since this is slightly different for dependencies and runnable tasks (see in code comment). Any pointers and opinions appreciated

distributed/nanny.py Outdated Show resolved Hide resolved
distributed/worker.py Outdated Show resolved Hide resolved
@mrocklin
Copy link
Member

cc @gforsyth would you be comfortable reviewing this?

@fjetter
Copy link
Member Author

fjetter commented Dec 17, 2020

Seems like the infinite loop I initially wanted to fix is appearing on CI (but not locally). Will need to investigate a bit further. There are also some broken tests which are failing on master as well #4374

Copy link
Contributor

@gforsyth gforsyth left a comment

Choose a reason for hiding this comment

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

Thanks for diving in to this @fjetter ! The changes here make sense to me overall -- just a few comments.

And we can certainly add a transition_new_waiting function -- most of that functionality is currently wrapped up in add_task

distributed/worker.py Outdated Show resolved Hide resolved
distributed/worker.py Outdated Show resolved Hide resolved
@fjetter
Copy link
Member Author

fjetter commented Dec 18, 2020

Ok, I can reproduce the CI failure locally... just took me 589 tries :/

@fjetter
Copy link
Member Author

fjetter commented Dec 18, 2020

Ok the (hopefully) last race condition is pretty bad and I don't have a resolution for it, yet, but I believe I figured out what's happening.

We have two competing mechanisms to deal with missing dependencies (the cluster doesn't distinguish between actually missing deps, e.g. a worker died, and deps which could not be fetched due to some sort of exceptions, comm related or otherwise; everything is a missing dep)

1 Worker internal

If the worker is unsuccessful in fetching the dependencies after 5 times, it flags the dependency as bad and gives up. This is what my test intended to capture

for dep in deps:
if dep.suspicious_count > 5:
deps.remove(dep)
self.bad_dep(dep)

2 Escalate to scheduler

The competing mechanism escalates immediately to the scheduler via

self.batched_stream.send(
{"op": "missing-data", "errant_worker": worker, "key": d}
)

def handle_missing_data(self, key=None, errant_worker=None, **kwargs):
logger.debug("handle missing data key=%s worker=%s", key, errant_worker)
self.log.append(("missing", key, errant_worker))
ts: TaskState = self.tasks.get(key)
if ts is None or not ts._who_has:
return
if errant_worker in self.workers:
ws: WorkerState = self.workers[errant_worker]
if ws in ts._who_has:
ts._who_has.remove(ws)
ws._has_what.remove(ts)
ws._nbytes -= ts.get_nbytes()
if not ts._who_has:
if ts._run_spec:
self.transitions({key: "released"})
else:
self.transitions({key: "forgotten"})

which transitions the faulty dependency to released. In the scenario where we have two sequential tasks A -> B and A is the faulty dependency, this means A is transitioned memory -> released which also triggers As depentent B to be transitioned, namely from processing to waiting. Since there is no processing -> waiting transition, it transitions B via processing -> released -> waiting where the first processing->released triggers a release-key which resets the suspicious counter of the worker internal process -> infinite loop

I am trying to find a solution which does not require me to rewrite this entire thing but it feels to me as if this mechanism requires a redesign and cleanup w/ appropriate tests (testing this is hard)

@fjetter fjetter changed the title Fix infinite loop for failed dependencies Deadlocks and infinite loops connected to failed dependencies Dec 22, 2020
Copy link
Member

@jrbourbeau jrbourbeau left a comment

Choose a reason for hiding this comment

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

Just merged the main branch to resolve some unrelated CI failures

@fjetter
Copy link
Member Author

fjetter commented Dec 22, 2020

Working on a few more changes and will update by tomorrow. I hope I have stabilised this by now but I'm not certain, yet.

@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from 121ce5e to 6c1028c Compare December 23, 2020 17:36
@fjetter
Copy link
Member Author

fjetter commented Dec 23, 2020

Changes got a bit bigger than I was hoping for. Here are the highlights

  • Instead of implementing multiple ways to deal with missing dependencies there is now one. How this works can be discussed but this proposes to query who_has from scheduler and make a decision based on the result. If the who_has of the scheduler is different to what the worker knows, that's a valid case for a retry and the worker updates it's state of who_has and tries again. If the who_has is in sync with the scheduler, I deduce something is broken with the world and escalate to the scheduler to fix this, i.e. scheduler will release and reschedule the broken key
  • If something happens during dependency collection which is not a connection error or a missing one, we will immediately flag the task as faulty, working under the assumption that this is unrecoverable and retrying won't solve anything. This is much stricter than before and we should discuss this. It helped me to avoid the last race condition I was talking about but this might also be avoided differently if this behaviour seems to drastic

In terms of changed behaviour that should be it. Most code changes where some kind of missing or improper transitions or places I consider dead code. Some of these dead code places I just deleted, others raise exceptions.

Apologies for the big PR but as you can imagine this is very hard to break apart


dep_ts = self.tasks[dependency]
self.log.append((dependency, "new-dep", dep_ts.state))
dep_ts.state = "waiting"
Copy link
Contributor

Choose a reason for hiding this comment

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

I'm certainly not sure that my approach was the correct one, but if we call release_key on a task that has any dependents, then the key will end up popped out of self.tasks without also clearing the corresponding item out of self.data -- so if we set the state to waiting here without checking for presence in self.data it can fail the check in validate_task_waiting

Copy link
Member Author

Choose a reason for hiding this comment

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

Sure, will check this section again. In general I'm not very happy with the approach of modifying the state attribute manually and maybe this should also by done via a proper transition step. tbh, I haven't fully understood your comment, yet, but if multiple tasks / dependencies must be adjusted this sounds like something which should be handled by a transition method anyhow I suppose

Copy link
Contributor

Choose a reason for hiding this comment

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

My comment was very unclear (having just re-read it).

It is possible at this point to have a task whose result is present in self.data even though there is no corresponding TaskState object in self.tasks. If that is the case, then the correct state is memory. If we blanket set it to waiting it will fail the assertions in validate_task_waiting.

Copy link
Member Author

Choose a reason for hiding this comment

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

How can data end up in memory without it being a task? (other than manual manipulation of the dict)

Copy link
Member Author

Choose a reason for hiding this comment

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

I indeed am seeing this error case you are describing but I'm wondering if this is an allowed state altogether. Even when effectively setting data manually (i.e. using the respective API) a TaskState object is created if it doesn't exist.

def update_data(self, comm=None, data=None, report=True, serializers=None):
for key, value in data.items():
ts = self.tasks.get(key)
if getattr(ts, "state", None) is not None:
self.transition(ts, "memory", value=value)
else:
self.tasks[key] = ts = TaskState(key)
self.put_key_in_memory(ts, value)
ts.priority = None
ts.duration = None
self.log.append((key, "receive-from-scatter"))

I was working under the assumption that data is always a subset of tasks

Copy link
Contributor

Choose a reason for hiding this comment

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

That confused me quite a bit. I don't necessarily agree with the implementation right now but I'll restore everything as it was to reduce the number of changes.

I'm definitely game to sync up at some point and sketch out a cleaner implementation for this.

Copy link
Member Author

Choose a reason for hiding this comment

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

I'm definitely game to sync up at some point and sketch out a cleaner implementation for this.

Sounds good. Once this PR is finally stable, I'll open an issue and summarise the points I've seen where I think we can improve and we can sync up on the topic.

Copy link
Member Author

Choose a reason for hiding this comment

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

I updated the PR with my current state. Unfortunately, reinstating the full dependency/release/refcounting as described above seemed to make things worse :(

Copy link
Contributor

Choose a reason for hiding this comment

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

Doesn't this introduce memory leakage?

Seeing the question about memory leak, I'm curious if this could be related to #3898.

Copy link
Member Author

Choose a reason for hiding this comment

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

If my argumentation about the memory leak was true, the leak would happen on the worker since the worker would not release the data anymore. From my understanding, #3898 is about a potential leak on the scheduler.
Also, the leak I was referring to does not exist and I was mistaken. The worker applies some refcounting by tracking the dependents of each task and releases the data properly once all dependents are gone, i.e. the data is no longer needed. I missed this mechanism since it's a bit spread out in this module but from my understanding it works as intended.

@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from 6c1028c to cdd303f Compare January 7, 2021 16:42
@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from 2b49c73 to 42690f0 Compare January 8, 2021 09:08
@gforsyth
Copy link
Contributor

gforsyth commented Jan 8, 2021

Looking over the changes now --

I've run the two test_worker failures from travis on my OSX laptop 50 times each and they've passed every time, so I'm fairly certain those are spurious.

The test_stress failure also fails on my laptop, but my work laptop has several networking issues due to security stuff, so it may not be relevant.

Copy link
Contributor

@gforsyth gforsyth left a comment

Choose a reason for hiding this comment

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

The changes overall look good to me. I'm seeing a few assertion errors when I run test_failed_workers.py::test_broken_worker_during_computation. Every 5th time or so, there's an assertion error when a worker tries to transition a task from ready -> waiting which results in a KeyError.
I'm fairly certain this only happens before the worker in question is killed, so it may just be some messiness as the worker is coming down before a new worker takes its place.

tornado.application - ERROR - Exception in callback functools.partial(<bound method IOLoop._discard_future_result of <tornad
o.platform.asyncio.AsyncIOLoop object at 0x7f873d8dc5b0>>, <Task finished name='Task-722' coro=<Worker.gather_dep() done, de
fined at /Users/gil/github.com/dask/distributed/distributed/worker.py:2005> exception=KeyError(('ready', 'waiting'))>)
Traceback (most recent call last):
  File "/Users/gil/miniforge3/envs/dask-distributed/lib/python3.8/site-packages/tornado/ioloop.py", line 743, in _run_cal
lback
    ret = callback()
  File "/Users/gil/miniforge3/envs/dask-distributed/lib/python3.8/site-packages/tornado/ioloop.py", line 767, in _discard
_future_result
    future.result()
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2143, in gather_dep
    self.transition(ts, "waiting", worker=worker)
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 1551, in transition
    func = self._transitions[start, finish]
KeyError: ('ready', 'waiting')
distributed.nanny - WARNING - Restarting worker

and occasional assertion errors after the new workers come up (these two problems never occur during the same test run, it's either one or the other)

distributed.worker - ERROR - 
Traceback (most recent call last):
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2965, in validate_state
    assert ts.key in self.data or ts.key in self.actors
AssertionError
distributed.worker - ERROR - 
Traceback (most recent call last):
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2274, in release_key
    self.validate_state()
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2965, in validate_state
    assert ts.key in self.data or ts.key in self.actors
AssertionError
distributed.core - ERROR - 
Traceback (most recent call last):
  File "/Users/gil/github.com/dask/distributed/distributed/core.py", line 563, in handle_stream
    handler(**merge(extra, msg))
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 1430, in delete_data
    self.release_key(key, reason="delete")
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2274, in release_key
    self.validate_state()
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2965, in validate_state
    assert ts.key in self.data or ts.key in self.actors
AssertionError
distributed.worker - ERROR - 
Traceback (most recent call last):
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 985, in handle_scheduler
    await self.handle_stream(
  File "/Users/gil/github.com/dask/distributed/distributed/core.py", line 563, in handle_stream
    handler(**merge(extra, msg))
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 1430, in delete_data
    self.release_key(key, reason="delete")
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2274, in release_key
    self.validate_state()
  File "/Users/gil/github.com/dask/distributed/distributed/worker.py", line 2965, in validate_state
    assert ts.key in self.data or ts.key in self.actors
AssertionError

I'm trying to figure out the root cause (of mostly the second error) but I don't think it's any worse than the existing behavior and I wouldn't consider it a blocker.

@fjetter
Copy link
Member Author

fjetter commented Jan 11, 2021

Rechecked the mentioned tests myself and I also can see a

Traceback (most recent call last):
  File "/Users/fjetter/miniconda3/envs/dask-dev/lib/python3.8/site-packages/tornado/ioloop.py", line 741, in _run_callback
    ret = callback()
  File "/Users/fjetter/miniconda3/envs/dask-dev/lib/python3.8/site-packages/tornado/ioloop.py", line 765, in _discard_future_result
    future.result()
  File "/Users/fjetter/workspace/distributed/distributed/worker.py", line 2153, in gather_dep
    assert cause in self.tasks

in test_broken_worker_during_computation

back to it...

@fjetter
Copy link
Member Author

fjetter commented Jan 11, 2021

I believe the first AssertionError mentioned above about the state transition ready->waiting is merely a wrong transition and should be easily fixed.

The second looks a bit more troublesome. While I agree that it's probably not worse than before, having a task in state "memory" w/out any data sounds quite serious, even if it is very rare. I'll have another look

@fjetter
Copy link
Member Author

fjetter commented Jan 11, 2021

Currently, I am observing a few failures on travis which seem unrelated. I'll retrigger the runs to see if they are persistent

Py3.6

  • distributed/tests/test_stess::test_cancel_stress with an RuntimeError: ('Unclosed Comms', []) error
  • distributed/tests/test_stess:: test_close_connections with a simple timeout
  • distributed/tests/test_worker::test_wait_for_outgoing with RuntimeError: cannot schedule new futures after shutdown test_wait_for_outgoing breaking on master CI #4417

Py3.7

  • test_stop_doing_unnecessary_work timeout
  • distributed/tests/test_stess:: test_close_connections with a simple timeout
  • distributed/tests/test_worker::test_wait_for_outgoing with RuntimeError: cannot schedule new futures after shutdown test_wait_for_outgoing breaking on master CI #4417

Py3.8 a few failures which look similar to above but eventually segfault #4387

@mrocklin
Copy link
Member

mrocklin commented Jan 11, 2021 via email

@fjetter
Copy link
Member Author

fjetter commented Jan 11, 2021

Wow, I'm really falling into a rabbit hole with this. I was investigating the test_stop_doing_unnecessary_work since its failure seemed out of place. Turns out this is connected. I'm not sure if it is connected to my actual change but it is connected to the same root cause.

Actually looking at the logs reveals that the executing_count of the workers become negative (see the running status of the repr of the workers) and the while loop of this test keeps on spinning for a negative value.

c = <Client: 'tcp://127.0.0.1:41064' processes=2 threads=3, memory=16.74 GB>
1839s = <Scheduler: "tcp://127.0.0.1:41064" processes: 0 cores: 0>
1840a = <Worker: 'tcp://127.0.0.1:40927', 0, Status.closed, stored: 1, running: -1/1, ready: 0, comm: 0, waiting: 0>
1841b = <Worker: 'tcp://127.0.0.1:45391', 1, Status.closed, stored: 2, running: -2/2, ready: 0, comm: 0, waiting: 0>
1842
1843    @gen_cluster(client=True)
1844    async def test_stop_doing_unnecessary_work(c, s, a, b):
1845        futures = c.map(slowinc, range(1000), delay=0.01)
1846        await asyncio.sleep(0.1)
1847    
1848        del futures
1849    
1850        start = time()
1851        while a.executing_count:
1852            await asyncio.sleep(0.01)
1853>           assert time() - start < 0.5
1854E           assert (1610380514.3005424 - 1610380510.597802) < 0.5
1855E            +  where 1610380514.3005424 = time()

This negative value comes from a kind of "doublefree". The counter is decreased once during release_key (which acts as a transition function when triggered by the scheduler) and once by the execution_done transition. I.e. there is again a race condition for cases where tasks are released while actually being executed.

This connects to

if ts.state not in ("executing", "long-running"):
return
being improperly changed in #4107 . Before the change in #4107 this was fetching the "current" state of the task instead of the "stale" state of the task instance in that scope. If I fix this, the test fails even harder...


I'm slowly wondering how we want to proceed with this since I continue to find issues and am trying to plug all the holes at once in this PR. That makes the entire thing very messy and almost impossible to review, I assume. Merging anything might make things worse in the meantime, though. Any suggestions on how to proceed here?

@mrocklin
Copy link
Member

That makes the entire thing very messy and almost impossible to review, I assume. Merging anything might make things worse in the meantime, though. Any suggestions on how to proceed here?

That is a fair concern. Ideally incremental PRs would be better, but we're also fixing bugs after a larger change (worker.py::TaskState) and so that may or may not be possible. FWIW, tests are sufficiently robust in distributed that if both you and tests agree that the fix is a good one then I'm reasonably confident merging it in, even without diligent review.

Of course, that sort of relaxed behavior is also what caused this problem in the first place.

@gforsyth
Copy link
Contributor

If I fix this, the test fails even harder...

@fjetter do you have this fix pushed up to a branch somewhere?

@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from e1b92e4 to d378c84 Compare January 12, 2021 14:17
@fjetter
Copy link
Member Author

fjetter commented Jan 12, 2021

Ok, the last one was connected to work stealing. I mentioned this as a potential issue over in #4413 already. My solution is, for now, to put a stolen task into "waiting for dependencies" which translates to the actual state waiting + runspec == None if there are no dependents, otherwise it is released.

That was the cause for the breakage of at least test_stop_doing_unnecessary_work and test_close_connections. While my fixes run through smoothly locally, CI still seems to have issues with it.

@fjetter fjetter force-pushed the exception_handling_failed_dependency branch from d378c84 to 0c59e63 Compare January 12, 2021 15:16
@fjetter
Copy link
Member Author

fjetter commented Jan 12, 2021

I think the CI failures are finally "expected". However, doing some real stress tests with this still reveal instabilities.

There are some failures which are probably invalid in-memory task states

Traceback (most recent call last): File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/distributed/worker.py", line 2572, in execute data[k] = self.data[k] File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/zict/buffer.py", line 80, in __getitem__ raise KeyError(key) KeyError: "('repartition-415-5a8ec5d8ce9f7fa507c212dd1c024d87', 83)"

During handling of the above exception, another exception occurred:

Traceback (most recent call last):
File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/tornado/ioloop.py", line 743, in _run_callback
ret = callback()
File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/tornado/ioloop.py", line 767, in _discard_future_result
future.result()
File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/distributed/worker.py", line 2576, in execute
data[k] = Actor(type(self.actors[k]), self.address, k, self)

and some invalid state transitions triggered by some scheduler <-> worker interaction/rescheduling, I suspect again work stealing of some kind

('executing', 'ready') Traceback (most recent call last): File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/distributed/worker.py", line 1538, in add_task self.transition(ts, "ready") File "/mnt/mesos/sandbox/venv/lib/python3.6/site-packages/distributed/worker.py", line 1560, in transition func = self._transitions[start, finish] KeyError: ('executing', 'ready')

Unfortunately, these are causing hard deadlocks and I don't trust this to be merged just yet

@mrocklin
Copy link
Member

Hi @fjetter is there something that I can run in order to expose these issues?

With regards to unexpected transitions, what I would probably do in this situation is call Worker.story(key) on any such situation in order to see what happened. That might help to pinpoint the issue.

@jrbourbeau
Copy link
Member

@mrocklin I ran into the first issue @fjetter listed running the example at #4430 (comment)

@mrocklin
Copy link
Member

Yeah, I was hoping to find something reproducible locally, but I'll try taking that and putting that into a stress test.

@gforsyth
Copy link
Contributor

Yeah, I was hoping to find something reproducible locally, but I'll try taking that and putting that into a stress test.

I think Florian fixed all of my local reproducers -- I've run the previous failing tests 50 times each and can't get them to fail, so it may require something at scale

@fjetter
Copy link
Member Author

fjetter commented Jan 15, 2021

@mrocklin Yes I am aware of the Worker.story but this is often not enough to really understand what's going on (at least I'm still often in the dark) Sometimes it's cross interaction of the scheduler when dealing with missing dependencies, sometimes it's workstealing, sometimes a wrong transition and it's very hard to distinguish.

I'm fairly certain that most, if not all, problems are connected to how Worker.add_task and Worker.release_key works and am currently considering starting from scratch.

What's most frustrating is that most issues cannot reliably reproduced, at least not by unit tests. Unit tests hit these cases but only every few 100 times. (bash script with an endless loop...)

@fjetter
Copy link
Member Author

fjetter commented Jan 15, 2021

What I noticed is that a lot of the internal state machine is effectively tested using the Worker.validate and respective TaskState methods. What's a bit unfortunate is that I occasionally observe these assertions to actually raise but tests run through anyhow. I believe we should figure out another mechanism such that the tests fail if one of these special validation errors are raised.
Don't have any good ideas how at the moment, but in the future this might give us an opportunity to detect these issues earlier

@mrocklin
Copy link
Member

Unit tests hit these cases but only every few 100 times. (bash script with an endless loop...)

You might also consider iterating from within an ipython session. For fast unit tests you can often run the test several times a second.

I have some time to look into this. I'm curious how best I can spend this time. Do you have any suggestions?

Also, I know that it's late on a Friday for you, but if you want to connect over video on this let me know. I would be curious to learn more about what is going on and your thoughts on how to fix it.

I'm free 6-7pm Central European time if that's interesting to you.

@mrocklin
Copy link
Member

I would expect the validation tests to stop a gen_cluster test, but maybe not a stress test that used Nannys

@fjetter
Copy link
Member Author

fjetter commented Jan 15, 2021

I opened another smaller PR #4432 with a minimal set of changes. With this I can still see assertion errors (but the tests are not failing ;) e.g.test_worker_who_has_clears_after_failed_connection no Nanny )

Where this PR escalated was when I tried reworking the missing keys mechanism, I omitted in #4432. I believe there is a lot of ambiguity in how tasks are managed and released, in particular in cases of missing keys, comm failures or missing workers (which also happens in stress situations a lot). This afternoon I was investigating scheduler handlers and the way the worker escalates missing or erroneous tasks where I noticed the handlers missing-data and release to be very similar but are used in very different circumstances. At least the missing-data handler is triggering the race condition I was sketching out in #4360 (comment) and I suspect cleaning this up / clarifying the intentions would help a bit

@mrocklin I would love to connect but my internet is dropping every few minutes today and its getting late already. Maybe we can find a spot next week, assuming nobody else fixed it by then :)

@mrocklin
Copy link
Member

Sounds good. In the mean time I'm curious, are there particular things that you're doing to replicate failures? Or is this mostly just the normal test suite? If so, which tests in particular?

@gforsyth
Copy link
Contributor

in #4432 you can get some errors (although the test passes) running pytest test_failed_workers.py::test_worker_who_has_clears_after_failed_connection -s

@mrocklin
Copy link
Member

Thanks @gforsyth . That's helpful. Reproduced.

@fjetter
Copy link
Member Author

fjetter commented Jan 19, 2021

I'll close this PR now since it escalated pretty badly. The deadlock issue was (at least partially) addressed in #4432 and I suggest to open new PRs for follow up issues. Thanks everyone for review and input!

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.

5 participants