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

Reduce pickle size for tasks #11687

Open
wants to merge 7 commits into
base: main
Choose a base branch
from
Open

Conversation

fjetter
Copy link
Member

@fjetter fjetter commented Jan 21, 2025

This makes the tasks a bit leaner by reducing pickle size. The performance gain is rather secondary but this should save a lot of memory client side when shipping things to the scheduler.

(I doubt that it will save anything on the scheduler since the pool for the pickle byte stream will be eaten up by order and the scheduler state but I haven't tried yet)

Two things left

  • I would like to avoid that every subclass of Task has to overload the getstate/setstate which is why I added the BaseTask. Child classes don't necessarily need this complexity. For example, the shuffle barrier task we're using as a subclass is created once per shuffle and a couple bytes overhead is not that bad. The option I chose here is very ugly. I'm still exploring options. Open for suggestions (This is why linting is still broken)
  • A part of me wonders if we can write a pseudo PickleBuffer object that serializes this stuff on demand. That would help a lot while we're still materializing the graph fully client side. I will explore this briefly... I suspect this is possible but I expect it to break memoization. If somebody already knows this is stupid, please stop me! Answer: Tricky. Couldn't find a way to do it. the picklebuffer interface is really strict (but I haven't spent much time ont his. We'll hopefully avoid client side materialization soon)

@fjetter
Copy link
Member Author

fjetter commented Jan 21, 2025

I also still have to test this on an actual graph. If everything that I'm cutting off here is memoized the entire PR is worthless and I will close it again

Comment on lines +501 to +502
def __reduce__(self):
return Alias, (self.key, self.target)
Copy link
Member Author

Choose a reason for hiding this comment

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

This is an example where I believe memoization is not helping since even in a memoized version, the picklestream would still save foo like frozenset, binget, ... which isn't much but considering how thin this class is, it adds up (unconfirmed)

Comment on lines 794 to 814
def __setstate__(self, state):
(
self.key,
self.func,
self.args,
self._token,
self._data_producer,
self._dependencies,
self.kwargs,
) = state

def __getstate__(self):
return (
self.key,
self.func,
self.args,
self._token,
self._data_producer,
self._dependencies,
self.kwargs,
)
Copy link
Member Author

Choose a reason for hiding this comment

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

this is saving a ton when looking at an individual task but I'm not sure if it does when looking at a graph. If it just impacts an individual task this PR may not be worth it (although it still helps a bit when sending stuff to the worker)

state = super().__getstate__()
# The constructor as a kwarg is redundant since this is encoded in the
# class itself. Serializing the builtin types is not trivial
# This saves about 15% of overhead
Copy link
Member Author

Choose a reason for hiding this comment

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

this is true for every level, i.e. these 15% cannot be memoized

Copy link
Contributor

github-actions bot commented Jan 21, 2025

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   4h 28m 2s ⏱️ +23s
 17 142 tests ±0   15 962 ✅ ±0   1 180 💤 ±0  0 ❌ ±0 
211 226 runs  ±0  194 192 ✅  - 1  17 034 💤 +1  0 ❌ ±0 

Results for commit 09b4649. ± Comparison against base commit 8886f04.

This pull request removes 1 and adds 1 tests. Note that renamed tests count towards both.
dask.tests.test_task_spec ‑ test_unpickled_repr
dask.tests.test_task_spec ‑ test_pickle_size

♻️ This comment has been updated with latest results.

@fjetter
Copy link
Member Author

fjetter commented Jan 28, 2025

Some preliminary tests on small-ish scale using the coiled geobenchmark climatology / highlevel API show that this change reduces the client side memory footprint by about 30% reduction of RSS memory.

The pickled graph size reduces from roughly 330MiB to 130MiB, i.e. that's a 60% reduction. That's still a bit large considering the size (small size in the benchmark suite) but most of this is driven by the graph being materialized fully. Dropping the client side materialization should shrink this dramatically.

dask/_task_spec.py Outdated Show resolved Hide resolved
Comment on lines +757 to +764
def __setstate__(self, state):
slots = self.__class__.get_all_slots()
for sl, val in zip(slots, state):
setattr(self, sl, val)

def __getstate__(self):
slots = self.__class__.get_all_slots()
return tuple(getattr(self, sl) for sl in slots)
Copy link
Member Author

Choose a reason for hiding this comment

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

This is an implementation that should cover all subclasses that do not require any custom logic upon deserialization and are implementing the slots protocol properly, i.e. subclasses typically don't have to implement anything else

@fjetter fjetter marked this pull request as ready for review January 29, 2025 10:14
@hendrikmakait hendrikmakait self-requested a review January 29, 2025 13:49
@fjetter
Copy link
Member Author

fjetter commented Jan 29, 2025

One test is failing dask/dataframe/dask_expr/tests/test_datasets.py::test_timeseries_gaph_size[None]

This is a test that compares graph sizes between legacy and dask-expr timeseries implementations. The dask-expr is typically strictly smaller. Apparently on this windows test that is not true but I suspect this is somehow due to the random seed not being set.

I suspect we'll kill the legacy version of anyhow. I'll retrigger the test job and will remove the legacy version

@fjetter
Copy link
Member Author

fjetter commented Jan 29, 2025

Interestingly, in my local testing, the dask-expr graph is about 15x slimmer than the legacy one. It looks like this is only the case when we're setting the random seed explicitly so I assume when doing this some state is duplicated or lambdas/local functions are contaminating the pickle stream.
without the seed, both graphs are very similar

@fjetter
Copy link
Member Author

fjetter commented Jan 29, 2025

Interestingly, the windows thing is persistent. I'll skip that test on this platform for now. I suspect we're going to rip out the legacy timeseries anyhow

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.

1 participant