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] Improve str/repr of Dataset to include execution plan #31604

Merged
merged 2 commits into from Jan 12, 2023

Conversation

c21
Copy link
Contributor

@c21 c21 commented Jan 11, 2023

Signed-off-by: Cheng Su scnju13@gmail.com

Why are these changes needed?

This is a followup of #31286, we want to improve the Dataset.__repr__() to provide more useful information to users, given lazy execution is default behavior.

The change is to include execution plan (stages as a tree) into Dataset.__repr__(). Currently each stage only has stage name printed out. We shall add more information per stage/operator in the future, which is orthogonal to this PR. This PR is just to print out the existing information we have.

Example:

>>> import ray
>>> ds = ray.data.range(10)
>>> ds = ds.map_batches(lambda x:x)
>>> ds = ds.filter(lambda x: x > 0)
>>> ds = ds.random_shuffle()
>>> ds
RandomShuffle
+- Filter
   +- MapBatches
      +- Dataset(num_blocks=10, num_rows=10, schema=<class 'int'>)
>>> ds.fully_executed()
>>> ds
Dataset(num_blocks=10, num_rows=9, schema=<class 'int'>)

The code change includes:

  • Introduce ExecutionPlan.get_plan_as_string() to get the string representation above for the plan.
  • Refactor two private methods inside ExecutionPlan - _get_unified_blocks_schema() and _get_num_rows_from_blocks_metadata()
  • Change Dataset.__repr__ to call ExecutionPlan.get_plan_as_string() directly.

Related issue number

Closes #31417

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 :(

Signed-off-by: Cheng Su <scnju13@gmail.com>
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.

Could we also update the rst docs?

@ericl ericl added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Jan 11, 2023
@c21
Copy link
Contributor Author

c21 commented Jan 11, 2023

Could we also update the rst docs?

@ericl - yeah plan to do that in the same PR here, if reviewers has no objections on the string representation.

@stephanie-wang
Copy link
Contributor

Just a suggestion, but I think it would be nicer to keep the plan history even after fully_executed is called, more like a "cache" call.

@c21
Copy link
Contributor Author

c21 commented Jan 11, 2023

Just a suggestion, but I think it would be nicer to keep the plan history even after fully_executed is called, more like a "cache" call.

@stephanie-wang - I thought it before, the only thing I am worried about, is the plan gets super long after multiple calls, assuming users only care about the latest Dataset. WDYT? @ericl, @clarkzinzow and @jianoaix.

Alternative is to add Dataset.plan() or Dataset.explain() prints out all plan history.

@jianoaix
Copy link
Contributor

Just a suggestion, but I think it would be nicer to keep the plan history even after fully_executed is called, more like a "cache" call.

@stephanie-wang - I thought it before, the only thing I am worried about, is the plan gets super long after multiple calls, assuming users only care about the latest Dataset. WDYT? @ericl, @clarkzinzow and @jianoaix.

Alternative is to add Dataset.plan() or Dataset.explain() prints out all plan history.

It sounds good to me to have a separate API to display the plan. The repr is used quite often and I think it's too much details for a simple print(ds). It seems not bad idea to just leave plan out of repr as well.

@c21
Copy link
Contributor Author

c21 commented Jan 12, 2023

Could we also update the rst docs?

@ericl - acutally given @jianoaix is doing change to make from_item() being lazy in parallel. Should we update all Ray Data rst docs in one pass after both PRs are merged? This should save us time to do only one pass for running all code snippets of documentation.

Signed-off-by: Cheng Su <scnju13@gmail.com>
@c21 c21 changed the title [Dataset] Improve str/repr of Dataset [Dataset] Improve str/repr of Dataset to include execution plan Jan 12, 2023
@ericl
Copy link
Contributor

ericl commented Jan 12, 2023

Hmm, for the caching thing I think we should hide the plan if the Dataset is fully independent of the previous stages. if it still has a hidden reference, we should show those previous stages. This might matter since the serialization behavior of the two cases could be different.

@ericl
Copy link
Contributor

ericl commented Jan 12, 2023

I'm going to just merge this, since I think it's a reasonable first step. We can discuss further refinements on a longer timescale.

@ericl ericl merged commit fb00672 into ray-project:master Jan 12, 2023
@c21 c21 deleted the repr branch January 12, 2023 07:01
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
@author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer.
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[data] Improve str/repr of lazy Datasets
5 participants