-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
[Datasets] Add metadata override and inference in Dataset.to_dask()
.
#28625
Merged
clarkzinzow
merged 2 commits into
ray-project:master
from
clarkzinzow:datasets/feat/to-dask-meta
Sep 23, 2022
Merged
Changes from 1 commit
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -2895,7 +2895,17 @@ def make_generator(): | |
|
||
return dataset | ||
|
||
def to_dask(self) -> "dask.DataFrame": | ||
def to_dask( | ||
self, | ||
meta: Union[ | ||
"pandas.DataFrame", | ||
"pandas.Series", | ||
Dict[str, Any], | ||
Iterable[Any], | ||
Tuple[Any], | ||
None, | ||
] = None, | ||
) -> "dask.DataFrame": | ||
"""Convert this dataset into a Dask DataFrame. | ||
|
||
This is only supported for datasets convertible to Arrow records. | ||
|
@@ -2905,12 +2915,25 @@ def to_dask(self) -> "dask.DataFrame": | |
|
||
Time complexity: O(dataset size / parallelism) | ||
|
||
Args: | ||
meta: An empty pandas DataFrame or Series that matches the dtypes and column | ||
names of the Dataset. By default, this will be inferred from the | ||
underlying Dataset schema, with this argument supplying an optional | ||
override. | ||
|
||
Returns: | ||
A Dask DataFrame created from this dataset. | ||
""" | ||
import dask | ||
import dask.dataframe as dd | ||
import pandas as pd | ||
|
||
try: | ||
import pyarrow as pa | ||
except Exception: | ||
pa = None | ||
|
||
from ray.data._internal.pandas_block import PandasBlockSchema | ||
from ray.util.client.common import ClientObjectRef | ||
from ray.util.dask import ray_dask_get | ||
|
||
|
@@ -2927,10 +2950,25 @@ def block_to_df(block: Block): | |
) | ||
return block.to_pandas() | ||
|
||
# TODO(Clark): Give Dask a Pandas-esque schema via the Pyarrow schema, | ||
# once that's implemented. | ||
if meta is None: | ||
# Infer Dask metadata from Datasets schema. | ||
schema = self.schema(fetch_if_missing=True) | ||
if isinstance(schema, PandasBlockSchema): | ||
meta = pd.DataFrame( | ||
{ | ||
col: pd.Series(dtype=dtype) | ||
for col, dtype in zip(schema.names, schema.types) | ||
} | ||
) | ||
elif pa is not None and isinstance(schema, pa.Schema): | ||
meta = schema.empty_table().to_pandas() | ||
else: | ||
# Simple dataset or schema not available. | ||
meta = None | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: this branch seems unnecessary, and it could be |
||
|
||
ddf = dd.from_delayed( | ||
[block_to_df(block) for block in self.get_internal_block_refs()] | ||
[block_to_df(block) for block in self.get_internal_block_refs()], | ||
meta=meta, | ||
) | ||
return ddf | ||
|
||
|
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
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.
Can the last 4 types from Union be dropped then?
Also if it's a Series, how does it correspond to multiple columns of the Dataset? Is it performing a column projection when meta is a subset columns?
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.
I think Clark was following Dask type here -
meta: pd.DataFrame, pd.Series, dict, iterable, tuple, optional
in https://docs.dask.org/en/stable/generated/dask.dataframe.from_delayed.html .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.
It's not quite match the type annotation so we should fix either.
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.
yeah agree, the comment should be updated.