-
Notifications
You must be signed in to change notification settings - Fork 6.1k
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] Delineate between ref and raw APIs for the Pandas/Arrow integrations. #18992
Merged
ericl
merged 6 commits into
ray-project:master
from
clarkzinzow:datasets/fix/pandas-arrow-integration-api
Oct 1, 2021
+251
−118
Merged
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
69d40fa
Delineate between ref and raw APIs for the Pandas/Arrow integrations.
clarkzinzow 36abeae
Docs updates.
clarkzinzow 7000e76
Usage updates.
clarkzinzow b33a0e4
Add back to_pandas() with limit.
clarkzinzow 1d48d95
Use limit()
clarkzinzow 3ff7931
Single-line for docstring.
clarkzinzow 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
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 |
---|---|---|
|
@@ -681,8 +681,8 @@ def zip(self, other: "Dataset[U]") -> "Dataset[(T, U)]": | |
comes from the first dataset and v comes from the second. | ||
""" | ||
|
||
blocks1 = self.get_blocks() | ||
blocks2 = other.get_blocks() | ||
blocks1 = self.get_internal_block_refs() | ||
blocks2 = other.get_internal_block_refs() | ||
|
||
if len(blocks1) != len(blocks2): | ||
# TODO(ekl) consider supporting if num_rows are equal. | ||
|
@@ -1321,14 +1321,15 @@ def to_modin(self) -> "modin.DataFrame": | |
"""Convert this dataset into a Modin dataframe. | ||
|
||
This works by first converting this dataset into a distributed set of | ||
Pandas dataframes (using ``.to_pandas()``). Please see caveats there. | ||
Then the individual dataframes are used to create the modin DataFrame | ||
using | ||
Pandas dataframes (using ``.to_pandas_refs()``). Please see caveats | ||
there. Then the individual dataframes are used to create the modin | ||
DataFrame using | ||
``modin.distributed.dataframe.pandas.partitions.from_partitions()``. | ||
|
||
This is only supported for datasets convertible to Arrow records. | ||
This function induces a copy of the data. For zero-copy access to the | ||
underlying data, consider using ``.to_arrow()`` or ``.get_blocks()``. | ||
underlying data, consider using ``.to_arrow()`` or | ||
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. Thx for updating internal comments. |
||
``.get_internal_block_refs()``. | ||
|
||
Time complexity: O(dataset size / parallelism) | ||
|
||
|
@@ -1338,7 +1339,7 @@ def to_modin(self) -> "modin.DataFrame": | |
|
||
from modin.distributed.dataframe.pandas.partitions import ( | ||
from_partitions) | ||
pd_objs = self.to_pandas() | ||
pd_objs = self.to_pandas_refs() | ||
return from_partitions(pd_objs, axis=0) | ||
|
||
def to_spark(self, | ||
clarkzinzow marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
@@ -1354,17 +1355,45 @@ def to_spark(self, | |
core_worker = ray.worker.global_worker.core_worker | ||
locations = [ | ||
core_worker.get_owner_address(block) | ||
for block in self.get_blocks() | ||
for block in self.get_internal_block_refs() | ||
] | ||
return raydp.spark.ray_dataset_to_spark_dataframe( | ||
spark, self.schema(), self.get_blocks(), locations) | ||
spark, self.schema(), self.get_internal_block_refs(), locations) | ||
|
||
def to_pandas(self) -> List[ObjectRef["pandas.DataFrame"]]: | ||
def to_pandas(self, limit: int = 1000) -> "pandas.DataFrame": | ||
clarkzinzow marked this conversation as resolved.
Show resolved
Hide resolved
|
||
"""Convert this dataset into a single Pandas DataFrame. | ||
|
||
This is only supported for datasets convertible to Arrow records. This | ||
limits the number of records returned to the provided limit. | ||
|
||
clarkzinzow marked this conversation as resolved.
Show resolved
Hide resolved
|
||
Time complexity: O(limit) | ||
|
||
Args: | ||
limit: The maximum number of records to return. | ||
|
||
Returns: | ||
A Pandas DataFrame created from this dataset, containing a limited | ||
number of records. | ||
""" | ||
|
||
if self.count() > limit: | ||
logger.warning(f"Only returning the first {limit} records from " | ||
"to_pandas()") | ||
limited_ds = self.limit(limit) | ||
blocks = limited_ds.get_internal_block_refs() | ||
output = DelegatingArrowBlockBuilder() | ||
for block in ray.get(blocks): | ||
output.add_block(block) | ||
return output.build().to_pandas() | ||
|
||
@DeveloperAPI | ||
def to_pandas_refs(self) -> List[ObjectRef["pandas.DataFrame"]]: | ||
"""Convert this dataset into a distributed set of Pandas dataframes. | ||
|
||
This is only supported for datasets convertible to Arrow records. | ||
This function induces a copy of the data. For zero-copy access to the | ||
underlying data, consider using ``.to_arrow()`` or ``.get_blocks()``. | ||
underlying data, consider using ``.to_arrow()`` or | ||
``.get_internal_block_refs()``. | ||
|
||
Time complexity: O(dataset size / parallelism) | ||
|
||
|
@@ -1381,7 +1410,8 @@ def to_numpy(self, *, | |
|
||
This is only supported for datasets convertible to NumPy ndarrays. | ||
This function induces a copy of the data. For zero-copy access to the | ||
underlying data, consider using ``.to_arrow()`` or ``.get_blocks()``. | ||
underlying data, consider using ``.to_arrow()`` or | ||
``.get_internal_block_refs()``. | ||
|
||
Time complexity: O(dataset size / parallelism) | ||
|
||
|
@@ -1399,7 +1429,23 @@ def to_numpy(self, *, | |
for block in self._blocks | ||
] | ||
|
||
def to_arrow(self) -> List[ObjectRef["pyarrow.Table"]]: | ||
def to_arrow(self) -> List["pyarrow.Table"]: | ||
"""Convert this dataset into a list of Arrow tables. | ||
|
||
This is only supported for datasets convertible to Arrow records. | ||
This function is zero-copy if the existing data is already in Arrow | ||
format. Otherwise, the data will be converted to Arrow format. | ||
|
||
Time complexity: O(1) unless conversion is required. | ||
|
||
Returns: | ||
A list of Arrow tables created from this dataset. | ||
""" | ||
|
||
return ray.get(self.to_arrow_refs()) | ||
|
||
@DeveloperAPI | ||
def to_arrow_refs(self) -> List[ObjectRef["pyarrow.Table"]]: | ||
"""Convert this dataset into a distributed set of Arrow tables. | ||
|
||
This is only supported for datasets convertible to Arrow records. | ||
|
@@ -1546,7 +1592,7 @@ def __iter__(self): | |
return DatasetPipeline(it, length=len(it._splits)) | ||
|
||
@DeveloperAPI | ||
def get_blocks(self) -> List[ObjectRef[Block]]: | ||
def get_internal_block_refs(self) -> List[ObjectRef[Block]]: | ||
"""Get a list of references to the underlying blocks of this dataset. | ||
|
||
This function can be used for zero-copy access to the data. | ||
|
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 | ||||
---|---|---|---|---|---|---|
|
@@ -14,7 +14,7 @@ | |||||
|
||||||
import ray | ||||||
from ray.types import ObjectRef | ||||||
from ray.util.annotations import PublicAPI | ||||||
from ray.util.annotations import PublicAPI, DeveloperAPI | ||||||
from ray.data.block import Block, BlockAccessor, BlockMetadata | ||||||
from ray.data.dataset import Dataset | ||||||
from ray.data.datasource import Datasource, RangeDatasource, \ | ||||||
|
@@ -509,12 +509,27 @@ def from_modin(df: "modin.DataFrame") -> Dataset[ArrowRow]: | |||||
from modin.distributed.dataframe.pandas.partitions import unwrap_partitions | ||||||
|
||||||
parts = unwrap_partitions(df, axis=0) | ||||||
return from_pandas(parts) | ||||||
return from_pandas_refs(parts) | ||||||
|
||||||
|
||||||
@PublicAPI(stability="beta") | ||||||
def from_pandas(dfs: List[ObjectRef["pandas.DataFrame"]]) -> Dataset[ArrowRow]: | ||||||
"""Create a dataset from a set of Pandas dataframes. | ||||||
def from_pandas(dfs: List["pandas.DataFrame"]) -> Dataset[ArrowRow]: | ||||||
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.
Suggested change
|
||||||
"""Create a dataset from a list of Pandas dataframes. | ||||||
|
||||||
Args: | ||||||
dfs: A list of Pandas dataframes. | ||||||
|
||||||
Returns: | ||||||
Dataset holding Arrow records read from the dataframes. | ||||||
""" | ||||||
return from_pandas_refs([ray.put(df) for df in dfs]) | ||||||
|
||||||
|
||||||
@DeveloperAPI | ||||||
def from_pandas_refs( | ||||||
dfs: List[ObjectRef["pandas.DataFrame"]]) -> Dataset[ArrowRow]: | ||||||
"""Create a dataset from a list of Ray object references to Pandas | ||||||
dataframes. | ||||||
|
||||||
Args: | ||||||
dfs: A list of Ray object references to pandas dataframes. | ||||||
|
@@ -546,8 +561,23 @@ def from_numpy(ndarrays: List[ObjectRef[np.ndarray]]) -> Dataset[ArrowRow]: | |||||
|
||||||
|
||||||
@PublicAPI(stability="beta") | ||||||
def from_arrow(tables: List[ObjectRef[Union["pyarrow.Table", bytes]]] | ||||||
) -> Dataset[ArrowRow]: | ||||||
def from_arrow( | ||||||
tables: List[Union["pyarrow.Table", bytes]]) -> Dataset[ArrowRow]: | ||||||
"""Create a dataset from a list of Arrow tables. | ||||||
|
||||||
Args: | ||||||
tables: A list of Ray object references to Arrow tables, | ||||||
or its streaming format in bytes. | ||||||
|
||||||
Returns: | ||||||
Dataset holding Arrow records from the tables. | ||||||
""" | ||||||
return from_arrow_refs([ray.put(t) for t in tables]) | ||||||
|
||||||
|
||||||
@DeveloperAPI | ||||||
def from_arrow_refs(tables: List[ObjectRef[Union["pyarrow.Table", bytes]]] | ||||||
) -> Dataset[ArrowRow]: | ||||||
"""Create a dataset from a set of Arrow tables. | ||||||
|
||||||
Args: | ||||||
|
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.
How about we take
*args
for from_pandas()? I feel the common case is passing just a single df if not passing distributed refs, so accepting a list is a bit weird.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.
@ericl Agreed, I'm primarily trying to not step on this PR's toes!