Skip to content

Conversation

@crusaderky
Copy link
Collaborator

@crusaderky crusaderky commented Jun 24, 2022

Provide initial support for pickling the WorkerState.
This is not (yet) used for dumping; consensus should be reached before we do anything about it.
For the time being, this is necessary because of dask/dask-benchmarks#50.

@crusaderky crusaderky marked this pull request as ready for review June 24, 2022 01:00
@github-actions
Copy link
Contributor

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

       15 files  ±  0         15 suites  ±0   10h 6m 8s ⏱️ - 33m 37s
  2 892 tests +  2    2 807 ✔️ +  2    83 💤 ±0  2 ±0 
21 422 runs  +14  20 455 ✔️ +18  965 💤  - 4  2 ±0 

For more details on these failures, see this check.

Results for commit eb308ba. ± Comparison against base commit dc019ed.

return f"<{type(self).__name__}: {len(self)} items>"

def __reduce__(self) -> tuple[Callable, tuple]:
heap = [(k, i, v) for k, i, vref in self._heap if (v := vref()) in self._data]
Copy link
Member

Choose a reason for hiding this comment

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

is the worker state ever pickled from a different thread than where it's being mutated?

Suggested change
heap = [(k, i, v) for k, i, vref in self._heap if (v := vref()) in self._data]
heap = [(k, i, v) for k, i, vref in self._heap.copy() if (v := vref()) in self._data]

Copy link
Member

Choose a reason for hiding this comment

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

unlikely but not impossible.

Standard pattern (beyond the current benchmarkign stuff in dask/dask-benchmarks#50)

would be something like client.run or a newer version of client.dump_cluster_state which would all be executing on the main htead.

However, we cannot protect from improper usages, e.g.

def task():
    return pickle.dumps(get_worker().state)
client.submit(task)

is valid dask but would pickle in another thread. I don't think we should protect or guard against this if it is expensive/ _heap is a list here, isn't it? a shallow copy should be fine but I'm not convinced it is necessary.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

None of the class is thread-safe; I don't see why pickling should be an exception.
Also, there's no guarantee that list.copy() will hold the GIL - if it does, it's an implementation detail.

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

P.S. the multithreading issue would be prevented with either client.run or an async task.

Copy link
Member

@graingert graingert Jun 24, 2022

Choose a reason for hiding this comment

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

list and dict copying is atomic

python/cpython#89598 (comment)

Copy link
Collaborator Author

Choose a reason for hiding this comment

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

We perhaps should explicitly document what non-trivial operations are atomical (for example list and dict copying is atomic) and whether atomacity is the part of the language specification or CPython implementation detail.

For as long as it's not explicitly documented, we must treat it as an implementation detail we can't rely upon

)
)
ws.handle_stimulus(UpdateDataEvent(data={"y": 123}, report=False, stimulus_id="s"))
ws2 = pickle.loads(pickle.dumps(ws))
Copy link
Member

Choose a reason for hiding this comment

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

that's very exciting

@crusaderky crusaderky merged commit 88e1fe0 into dask:main Jun 25, 2022
@crusaderky crusaderky deleted the WSMR/pickle branch June 25, 2022 15:54
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.

Alternatives for current ensure_communicating

3 participants