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

[Dataset] Add FromXXX operators #32959

Merged
merged 61 commits into from
Apr 8, 2023
Merged

Conversation

scottjlee
Copy link
Contributor

@scottjlee scottjlee commented Mar 2, 2023

Why are these changes needed?

From the Ray Data Read API (ray.data.read_api), from_xxx methods do not have logical operators. This PR implements the corresponding operators for such from_xxx methods.

Related issue number

Closes #32604

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@scottjlee scottjlee marked this pull request as ready for review March 2, 2023 00:15
from ray.data._internal.logical.operators.map_operator import AbstractMap


class FromItems(AbstractMap):
Copy link
Contributor

Choose a reason for hiding this comment

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

It's not really a map right? Shouldn't this inherit from Read?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah you're correct, it's not a map here. Discussed with @c21 earlier this week, and I think the distinction we wanted between read_xxx and from_xxx was that read is done from an external source, while from is read from in-memory. Should we still inherit from Read? As an alternative, we could instead inherit from the general parent LogicalOperator class?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, if it's hard to represent as a Read, it seems ok to inherit from logical operator directly.

Copy link
Contributor

@ericl ericl left a comment

Choose a reason for hiding this comment

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

The fact that this is distinct from Read seems strange to me, especially since range() generates a Read operator. I don't have a strong objection however.

@scottjlee scottjlee changed the title [Dataset] Add FromItems operator [Dataset] Add FromItems, FromPandasRef, and FromPandas operators Mar 6, 2023
@scottjlee scottjlee changed the title [Dataset] Add FromItems, FromPandasRef, and FromPandas operators [Dataset] Add FromXXX operators Mar 7, 2023
@scottjlee scottjlee marked this pull request as draft March 7, 2023 22:35
@scottjlee scottjlee added tests-ok The tagger certifies test failures are unrelated and assumes personal liability. and removed @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. labels Apr 6, 2023
@scottjlee
Copy link
Contributor Author

Tests look unrelated to me -- sorry for the long hiatus on this one, should be ready for review now @ericl

self._parallelism = parallelism


class FromTF(FromItems):
Copy link
Contributor

Choose a reason for hiding this comment

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

Does this create Arrow blocks or simple blocks? (I assume arrow?) I am wondering if it makes sense to extend the Arrow logical operator if that's the case.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yeah, this creates Arrow blocks I believe:

>>> import ray; import tensorflow_datasets as tfds
>>> tf_dataset, _ = tfds.load('cifar10', split=["train", "test"])
>>> dataset = ray.data.from_tf(tf_dataset)
>>> blocks = dataset._plan.execute()
>>> ray.get(blocks._blocks[0])
pyarrow.Table
id: binary
image: extension<arrow.py_extension_type<ArrowTensorType>>
label: int64

Since the existing from_tf implementation relies on from_items on a numpy iterator converted to a list, I was thinking this most closely reflects FromItems. Thoughts?

Copy link
Contributor

Choose a reason for hiding this comment

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

I think it's fine to refine FromTF to extend FromNumpyRefs, so it's being more specific.


def __init__(
self,
ndarrays: Union[List[ObjectRef["np.ndarray"]], List["np.ndarray"]],
Copy link
Contributor

Choose a reason for hiding this comment

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

Or perhaps from tf / torch should extend this class instead?

Copy link
Contributor

Choose a reason for hiding this comment

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

yeah I think from_tf can extend this class.

from_torch seems not guarantee to return NumPy ndarray, so it looks to me okay to extend FromItems.

@c21 c21 assigned jianoaix and unassigned clarkzinzow Apr 6, 2023
Copy link
Contributor

@c21 c21 left a comment

Choose a reason for hiding this comment

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

Thanks @scottjlee, major logic looks pretty solid, having some comments.

python/ray/data/tests/test_raydp_dataset.py Outdated Show resolved Hide resolved
python/ray/data/_internal/util.py Outdated Show resolved Hide resolved
python/ray/data/_internal/util.py Outdated Show resolved Hide resolved
python/ray/data/_internal/util.py Outdated Show resolved Hide resolved
python/ray/data/_internal/planner/plan_from_numpy_op.py Outdated Show resolved Hide resolved
python/ray/data/_internal/planner/plan_from_numpy_op.py Outdated Show resolved Hide resolved
python/ray/data/_internal/planner/plan_from_pandas_op.py Outdated Show resolved Hide resolved
python/ray/data/_internal/planner/plan_from_pandas_op.py Outdated Show resolved Hide resolved
python/ray/data/tests/test_execution_optimizer.py Outdated Show resolved Hide resolved
python/ray/data/read_api.py Outdated Show resolved Hide resolved
)


@PublicAPI
def from_arrow(
tables: Union["pyarrow.Table", bytes, List[Union["pyarrow.Table", bytes]]]
tables: Union["pyarrow.Table", bytes, List[Union["pyarrow.Table", bytes]]],
logical_op: Optional[FromArrowRefs] = None,
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

python/ray/data/read_api.py Outdated Show resolved Hide resolved
Copy link
Contributor

@c21 c21 left a comment

Choose a reason for hiding this comment

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

LGTM

@c21
Copy link
Contributor

c21 commented Apr 8, 2023

cc @ericl for any comments before merging. thanks.

@ericl ericl merged commit 82434e2 into ray-project:master Apr 8, 2023
c21 pushed a commit that referenced this pull request Apr 11, 2023
- #32959 added a good number of tests without changing any timeouts, and as a result, some of the tests will time out occasionally, making the Data CI tests flakey. Therefore, we should increase the timeout for Bazel targets which recently received additional test cases. 
- In addition, one of the failing tests, `test_from_huggingface_e2e`, was found to have a failure which was not caught in the original PR. `test_stats.test_dataset__repr__` also is flakey sometimes, so I add a fix for these tests.
- I also added a blank file, `python/ray/data/tests/block_batching/__init__.py`, which is needed to resolve a pytest error (non-unique test filename) for an existing test.

Signed-off-by: Scott Lee <[email protected]>
elliottower pushed a commit to elliottower/ray that referenced this pull request Apr 22, 2023
elliottower pushed a commit to elliottower/ray that referenced this pull request Apr 22, 2023
- ray-project#32959 added a good number of tests without changing any timeouts, and as a result, some of the tests will time out occasionally, making the Data CI tests flakey. Therefore, we should increase the timeout for Bazel targets which recently received additional test cases.
- In addition, one of the failing tests, `test_from_huggingface_e2e`, was found to have a failure which was not caught in the original PR. `test_stats.test_dataset__repr__` also is flakey sometimes, so I add a fix for these tests.
- I also added a blank file, `python/ray/data/tests/block_batching/__init__.py`, which is needed to resolve a pytest error (non-unique test filename) for an existing test.

Signed-off-by: Scott Lee <[email protected]>
Signed-off-by: elliottower <[email protected]>
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
- ray-project#32959 added a good number of tests without changing any timeouts, and as a result, some of the tests will time out occasionally, making the Data CI tests flakey. Therefore, we should increase the timeout for Bazel targets which recently received additional test cases.
- In addition, one of the failing tests, `test_from_huggingface_e2e`, was found to have a failure which was not caught in the original PR. `test_stats.test_dataset__repr__` also is flakey sometimes, so I add a fix for these tests.
- I also added a blank file, `python/ray/data/tests/block_batching/__init__.py`, which is needed to resolve a pytest error (non-unique test filename) for an existing test.

Signed-off-by: Scott Lee <[email protected]>
Signed-off-by: Jack He <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
tests-ok The tagger certifies test failures are unrelated and assumes personal liability.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Dataset] Add logical operator for from_xxx API
5 participants