-
Notifications
You must be signed in to change notification settings - Fork 94
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
Individual CUDA object spilling #451
Merged
Merged
Changes from all commits
Commits
Show all changes
64 commits
Select commit
Hold shift + click to select a range
8dc916b
Initial work on highres spilling
madsbk 4e58657
Tracking aliasing device memory objects
madsbk 51c8213
clean up
madsbk 994a8fe
renamed dynamic spilling => object spilling
madsbk 64e02de
removed debugging assert
madsbk c27b238
Avoid internal comparisons of ProxyObjects
madsbk bd37c73
obj_pxy_concat(): only proxify when using object spilling
madsbk 64dfdab
Removed use of _obj_pxy_get_meta()
madsbk 40adb1c
sizeof.register_lazy("cupy"): Includes cupy.ndarray handle
madsbk 7addac2
Overwriting the Dask dispatch of Pandas objects
madsbk 7eea004
get_device_memory_objects(): add multi index support
madsbk c21f62d
Handling make_meta() and get_parallel_type dispatching
madsbk 03fe9b8
Added test of parquet read/write of a proxy object
madsbk 316d0a4
Implemented assignment to a proxied object
madsbk 55eeb79
Renamed __obj_pxy_cache => _obj_pxy_cache
madsbk 597ba34
Added optional arguments to __array__()
madsbk 3b674dc
Implement dispatch of array-concat
madsbk fb7800d
Implement dispatch of array-tensordot
madsbk 0f6b082
Implement dispatch of array-einsum
madsbk 2e4f627
Implemented unproxify_input_wrapper()
madsbk 3453113
test: _PxyObjTest to use _obj_pxy[]
madsbk 40b294d
Removed ProxyObject.__slots__
madsbk 0631652
proxify(): added subclass=None
madsbk b10b7fc
Added missing docs
madsbk 314b280
proxify: inherit from cudf._lib.table.Table
madsbk 95be618
Caching device_memory_objects
madsbk 5ad0605
Implemented _obj_pxy_cache_wrapper()
madsbk c95febe
caching __sizeof__
madsbk 0528028
replacing lists with tuples
madsbk 6f40853
Deserializing CUDA-serialized ProxyObjects in the hostfile
madsbk eecdfd6
proxify_device_object(ProxyObject): check proxied_id_to_proxy
madsbk c91b6ee
obj_pxy_cuda_serialize(): shallow copy of obj.
madsbk bbe7853
Caching __len__()
madsbk 22e71f3
Locking when accessing self.store
madsbk 940e53c
Implemented get_proxied_id_to_proxy()
madsbk e304d29
Simplified device access info
madsbk 9d422ca
_obj_pxy_serialize(): Avoid doing double work
madsbk 4577a6e
Avoid materializing RangeIndex
madsbk 9e1737c
fixed typo
madsbk 55e80d2
concat_dispatch(): don't set hostfile
madsbk bd1850c
isort
madsbk d468ab6
Implemented ProxiesTally
madsbk 44a24c6
Implemented class ProxiesTally.get_unspilled_proxies()
madsbk d6526e3
ProxiesTally: get its own lock
madsbk 6b37c27
Implemented UnspilledProxies
madsbk 0e7a63c
reformat using correct versions of black and isort
madsbk fedf8b3
docs
madsbk 055d16d
clean up
madsbk 2ae97f0
Rename: ObjectSpillingHostFile => ProxifyHostFile
madsbk b8da4dc
removing the object-spilling option
madsbk 17702e1
clean up and docs
madsbk d25af3a
Fixed typo
madsbk e21204f
proxify_device_objects(): clean up
madsbk 0609916
get_device_memory_objects(): clean up
madsbk bcf56e4
Added some typing
madsbk 5b73daf
typo
madsbk fde45bf
Added CLI help doc for --enable-jit-unspill
madsbk 018db71
test of tensordot and einsum now uses dask.array.
madsbk bdaeaa6
test_one_item_limit(): testing proxies_tally
madsbk 85d962f
Fixed removal of an "optional"
madsbk 4093550
Fixing typos
madsbk 544417a
rename: _obj_pxy_serialized() => _is_obj_pxy_serialized()
madsbk d1933d5
removed old TODO list
madsbk 81668b1
test_proxy_object_parquet(): use pyarrow engine
madsbk File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,119 @@ | ||
from typing import Any, Set | ||
|
||
from dask.sizeof import sizeof | ||
from dask.utils import Dispatch | ||
|
||
dispatch = Dispatch(name="get_device_memory_objects") | ||
|
||
|
||
def get_device_memory_objects(obj: Any) -> Set: | ||
""" Find all CUDA device objects in `obj` | ||
|
||
Search through `obj` and find all CUDA device objects, which are objects | ||
that either are known to `dispatch` or implement `__cuda_array_interface__`. | ||
|
||
Notice, the CUDA device objects must be hashable. | ||
|
||
Parameters | ||
---------- | ||
obj: Any | ||
Object to search through | ||
|
||
Returns | ||
------- | ||
ret: set | ||
Set of CUDA device memory objects | ||
""" | ||
return set(dispatch(obj)) | ||
|
||
|
||
@dispatch.register(object) | ||
def get_device_memory_objects_default(obj): | ||
if hasattr(obj, "_obj_pxy"): | ||
if obj._obj_pxy["serializers"] is None: | ||
return dispatch(obj._obj_pxy["obj"]) | ||
else: | ||
return [] | ||
if hasattr(obj, "data"): | ||
return dispatch(obj.data) | ||
if hasattr(obj, "_owner") and obj._owner is not None: | ||
return dispatch(obj._owner) | ||
if hasattr(obj, "__cuda_array_interface__"): | ||
return [obj] | ||
return [] | ||
|
||
|
||
@dispatch.register(list) | ||
@dispatch.register(tuple) | ||
@dispatch.register(set) | ||
@dispatch.register(frozenset) | ||
def get_device_memory_objects_python_sequence(seq): | ||
ret = [] | ||
for s in seq: | ||
ret.extend(dispatch(s)) | ||
return ret | ||
|
||
|
||
@dispatch.register(dict) | ||
def get_device_memory_objects_python_dict(seq): | ||
ret = [] | ||
for s in seq.values(): | ||
ret.extend(dispatch(s)) | ||
return ret | ||
|
||
|
||
@dispatch.register_lazy("cupy") | ||
def get_device_memory_objects_register_cupy(): | ||
from cupy.cuda.memory import MemoryPointer | ||
|
||
@dispatch.register(MemoryPointer) | ||
def get_device_memory_objects_cupy(obj): | ||
return [obj.mem] | ||
|
||
|
||
@dispatch.register_lazy("cudf") | ||
def get_device_memory_objects_register_cudf(): | ||
import cudf.core.dataframe | ||
import cudf.core.index | ||
import cudf.core.multiindex | ||
import cudf.core.series | ||
|
||
@dispatch.register(cudf.core.dataframe.DataFrame) | ||
def get_device_memory_objects_cudf_dataframe(obj): | ||
|
||
ret = dispatch(obj._index) | ||
for col in obj._data.columns: | ||
ret += dispatch(col) | ||
return ret | ||
|
||
@dispatch.register(cudf.core.series.Series) | ||
def get_device_memory_objects_cudf_series(obj): | ||
return dispatch(obj._index) + dispatch(obj._column) | ||
|
||
@dispatch.register(cudf.core.index.RangeIndex) | ||
def get_device_memory_objects_cudf_range_index(obj): | ||
# Avoid materializing RangeIndex. This introduce some inaccuracies | ||
# in total device memory usage but we accept the memory use of | ||
# RangeIndexes are limited. | ||
return [] | ||
|
||
@dispatch.register(cudf.core.index.Index) | ||
def get_device_memory_objects_cudf_index(obj): | ||
return dispatch(obj._values) | ||
|
||
@dispatch.register(cudf.core.multiindex.MultiIndex) | ||
def get_device_memory_objects_cudf_multiindex(obj): | ||
return dispatch(obj._columns) | ||
|
||
|
||
@sizeof.register_lazy("cupy") | ||
def register_cupy(): # NB: this overwrites dask.sizeof.register_cupy() | ||
import cupy.cuda.memory | ||
|
||
@sizeof.register(cupy.cuda.memory.BaseMemory) | ||
def sizeof_cupy_base_memory(x): | ||
return int(x.size) | ||
|
||
@sizeof.register(cupy.ndarray) | ||
def sizeof_cupy_ndarray(x): | ||
return int(x.nbytes) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
The fact that there's no
memory_limit
here seems to point that there's no capability for host<->disk spilling, is that intended? We also see to be missinglocal_directory
, which will prevent users from running storing things anywhere but the current directory, that seems problematic for certain use cases.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.
Ok, I now see you added a comment about
memory_limit
inLocalCUDACluster
docs. Can you do the same indask-cuda/dask_cuda/cli/dask_cuda_worker.py
Lines 196 to 199 in b170b29
The
local_directory
question is still valid though.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.
Yes, in this PR we will not support any spilling to disk. I have removed
memory_limit
andlocal_directory
to make this clear. I don't thinklocal_directory
is used for anything else than disk spilling?I will be up to a future PR to implement disk spilling, which shouldn't be too difficult.
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.
My understanding was that
local_directory
was used for more than just spilling, so I went on a hunt. To be fair, it's still unclear to me even where in the codelocal_directory
is really used, I found one other case where it's used though, when you upload files, see for example https://github.com/dask/distributed/blob/607cfd2ce00edd44c99da3273de0763a426dda7d/distributed/tests/test_worker.py#L176-L202 . That isn't to say this is the only other case, but I couldn't confirm nor deny whether there are other cases besides spilling and file uploading. Maybe @quasiben or @jakirkham would know.