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

Adding chunk & type information to dask high level graphs #7309

Merged
merged 13 commits into from Apr 29, 2021
Merged
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
6 changes: 6 additions & 0 deletions dask/array/core.py
Expand Up @@ -1156,6 +1156,12 @@ def __new__(cls, dask, name, chunks, dtype=None, meta=None, shape=None):
if result is not None:
self = result

if hasattr(self, "dask") and name in self.dask.layers:
self.dask.layers[name].info["type"] = type(self)
self.dask.layers[name].info["chunk_type"] = type(meta)
self.dask.layers[name].info["chunks"] = chunks
self.dask.layers[name].info["dtype"] = dtype

GenevieveBuckley marked this conversation as resolved.
Show resolved Hide resolved
return self

def __reduce__(self):
Expand Down
7 changes: 7 additions & 0 deletions dask/dataframe/core.py
Expand Up @@ -3609,6 +3609,13 @@ class DataFrame(_Frame):
_token_prefix = "dataframe-"
_accessors = set()

def __init__(self, dsk, name, meta, divisions):
super().__init__(dsk, name, meta, divisions)
if hasattr(self, "dask") and name in self.dask.layers:
self.dask.layers[name].info["type"] = type(self)
self.dask.layers[name].info["divisions"] = divisions
self.dask.layers[name].info["chunk_type"] = type(meta)

Copy link
Collaborator

Choose a reason for hiding this comment

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

  1. could use the same refactoring as da.Array
  2. why nothing for Series?
  3. dd does not have anything equivalent to array_plugins

Copy link
Contributor Author

Choose a reason for hiding this comment

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

  1. could use the same refactoring as da.Array - ok, done
  2. why nothing for Series? - Good point, I've added "series_dtypes": {col: meta[col].dtype for col in meta.columns}
  3. dd does not have anything equivalent to array_plugins - This looks like a comment and not a request. Thanks for mentioning it!

def __array_wrap__(self, array, context=None):
if isinstance(context, tuple) and len(context) > 0:
if isinstance(context[1][0], np.ndarray) and context[1][0].shape == ():
Expand Down
1 change: 1 addition & 0 deletions dask/highlevelgraph.py
Expand Up @@ -59,6 +59,7 @@ class Layer(collections.abc.Mapping):
annotations: Optional[Mapping[str, Any]]

def __init__(self, annotations: Mapping[str, Any] = None):
self.info = {}
Copy link
Collaborator

Choose a reason for hiding this comment

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

  1. please add type annotations above
  2. please add documentation
  3. you'll lose everything on clone(), cull(), and possibly some other methods

Copy link
Contributor Author

Choose a reason for hiding this comment

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

  1. please add type annotations above - added
  2. please add documentation - added
  3. you'll lose everything on clone(), cull(), and possibly some other methods - I think this is a comment, not a request to fix just now? If I've misunderstood please let me know.

if annotations:
self.annotations = annotations
else:
Expand Down