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

[Datasets] Defer first block computation when reading a Datasource with schema information in metadata #34251

Merged
merged 9 commits into from
Apr 18, 2023

Conversation

scottjlee
Copy link
Contributor

Why are these changes needed?

See #33943

Related issue number

Closes #33943

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 added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • 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 :(

Scott Lee added 5 commits April 10, 2023 16:58
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@scottjlee scottjlee marked this pull request as ready for review April 11, 2023 05:01

# Forces a data read.
values = [[s["one"], s["two"]] for s in ds.take_all()]
check_num_computed(ds, 2, 2)
check_num_computed(ds, 2, 0)
Copy link
Contributor

Choose a reason for hiding this comment

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

why we have difference between bulk and streaming?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

the comment here explains:

# When streaming executor is on, the _num_computed() is affected only
# by the ds.schema() which will still partial read the blocks, but will
# not affected by operations like take() as it's executed via streaming
# executor.

@@ -475,11 +479,13 @@ def test_parquet_read_partitioned(ray_start_regular_shared, fs, data_path):
[3, "f"],
[3, "g"],
]
check_num_computed(ds, 2, 0)
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto


# Test column selection.
ds = ray.data.read_parquet(data_path, columns=["one"], filesystem=fs)
values = [s["one"] for s in ds.take()]
assert sorted(values) == [1, 1, 1, 3, 3, 3]
check_num_computed(ds, 2, 0)
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto


# Forces a data read.
values = [[s["one"], s["two"]] for s in ds.take()]
check_num_computed(ds, 2, 2)
check_num_computed(ds, 2, 0)
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto

Signed-off-by: Scott Lee <sjl@anyscale.com>
Scott Lee added 2 commits April 17, 2023 09:35
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@scottjlee scottjlee requested a review from c21 April 17, 2023 20:49
Signed-off-by: Scott Lee <sjl@anyscale.com>
@c21
Copy link
Contributor

c21 commented Apr 18, 2023

Merging to master.

@c21 c21 merged commit c538e69 into ray-project:master Apr 18, 2023
elliottower pushed a commit to elliottower/ray that referenced this pull request Apr 22, 2023
…th schema information in metadata (ray-project#34251)

In the current implementation of [ExecutionPlan._get_unified_blocks_schema](https://github.com/ray-project/ray/blob/master/python/ray/data/_internal/plan.py#L418), we force execution to compute the first block when given a `LazyBlockList`. However, when creating a Dataset from a datasource which have schema information available before reading (e.g. Parquet), this unnecessarily forces execution, since we already check for metadata in the subsequent [ensure_metadata_for_first_block](https://github.com/ray-project/ray/blob/master/python/ray/data/_internal/lazy_block_list.py#L379). Therefore, we can remove `blocks.compute_first_block()`.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: elliottower <elliot@elliottower.com>
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
…th schema information in metadata (ray-project#34251)

In the current implementation of [ExecutionPlan._get_unified_blocks_schema](https://github.com/ray-project/ray/blob/master/python/ray/data/_internal/plan.py#L418), we force execution to compute the first block when given a `LazyBlockList`. However, when creating a Dataset from a datasource which have schema information available before reading (e.g. Parquet), this unnecessarily forces execution, since we already check for metadata in the subsequent [ensure_metadata_for_first_block](https://github.com/ray-project/ray/blob/master/python/ray/data/_internal/lazy_block_list.py#L379). Therefore, we can remove `blocks.compute_first_block()`.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Jack He <jackhe2345@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Dataset] Remove pre-read computation of first block when getting schema
4 participants