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

[Data]Update Data progress bars to use row as the iteration unit #46699

Merged
merged 18 commits into from
Aug 2, 2024

Conversation

Bye-legumes
Copy link
Contributor

@Bye-legumes Bye-legumes commented Jul 18, 2024

Why are these changes needed?

close #46579

Related issue number

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

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
@Bye-legumes
Copy link
Contributor Author

image

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
@Bye-legumes Bye-legumes changed the title [WIP] [Data]Update Data progress bars to use row as the iteration unit [Data]Update Data progress bars to use row as the iteration unit Jul 19, 2024
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
@Bye-legumes
Copy link
Contributor Author

@scottjlee can you check if my implementation is correct? or other missing part? Thanks!

@scottjlee scottjlee self-assigned this Jul 19, 2024
@scottjlee
Copy link
Contributor

@scottjlee can you check if my implementation is correct? or other missing part? Thanks!

Thanks @Bye-legumes, I will review this in the next several days.

Copy link
Contributor

@scottjlee scottjlee left a comment

Choose a reason for hiding this comment

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

initial changes LGTM, thanks! a couple requests:

  • could you also add num_output_rows_total() implementations for other one-to-one operators? like Limit, OutputSplitter? as well as N-ary operators like Union and Zip. these should be relatively straightforward to implement num_output_rows_total().
  • Could you add a TODO comment here to replace with row as the unit, once we implement num_output_rows_total() for AllToAllOperators (or, you can also implement it in this PR, but may be more involved).

@@ -459,3 +459,8 @@ def implements_accurate_memory_accounting(self) -> bool:
def supports_fusion(self) -> bool:
"""Returns ```True``` if this operator can be fused with other operators."""
return False

@property
def estimated_output_num_rows(self) -> Optional[int]:
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: rename property to keep naming convention consistent with num_outputs_total

Suggested change
def estimated_output_num_rows(self) -> Optional[int]:
def num_output_rows_total(self) -> Optional[int]:

Copy link
Contributor

Choose a reason for hiding this comment

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

also, can we add _estimated_output_num_rows as a class attribute in PhysicalOperator.__init__() (set to None)? so that here, we can simply return self._estimated_output_num_rows. and in MapOperator, we won't need to define its own estimated_output_num_rows

Comment on lines 436 to 439
@property
def estimated_output_num_rows(self) -> Optional[int]:
return getattr(self, "_estimated_output_num_rows", 0)

Copy link
Contributor

Choose a reason for hiding this comment

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

shouldn't need this once we implement class attribute at the PhysicalOperator level.

Comment on lines 245 to 246
num_rows = sum(meta.num_rows for _, meta in ref.blocks)
self.progress_bar.update(num_rows, self.op.estimated_output_num_rows)
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
num_rows = sum(meta.num_rows for _, meta in ref.blocks)
self.progress_bar.update(num_rows, self.op.estimated_output_num_rows)
self.progress_bar.update(ref.num_rows(), self.op.estimated_output_num_rows)

@scottjlee
Copy link
Contributor

another thing my colleague just reminded me of: in the case where the row/s is large, the output can become difficult to read or parse. several options i can think of to address this:

  • (easiest) call tqdm.format_num to format the number displayed in the progress bar
  • (more involved) add a utility function (e.g. in ray/data/_internal/util.py) which will format numbers in a concise format? e.g. we can use scientific notation for numbers >= 10e5, round to 2 decimal places, etc.

Bye-legumes and others added 3 commits July 23, 2024 11:42
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
@anyscalesam anyscalesam added triage Needs triage (eg: priority, bug/not-bug, and owning component) data Ray Data-related issues labels Jul 25, 2024
@Bye-legumes
Copy link
Contributor Author

@scottjlee I just made some modification, can you check them? Thx!

@@ -51,6 +51,13 @@ def safe_print(*args, **kwargs):
instance().unhide_bars()


def format_num(n):
Copy link
Contributor

Choose a reason for hiding this comment

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

instead of duplicating tqdm source code logic here, could we directly use tqdm.tqdm.format_num() in other places instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have consider this but finnally I just copy the codes from format_num. The reason I just define it here is 1. tqdm is not import in tqdm_ray currently.. and in fact we just only want to use format_num function. 2 .In addition, we self defined a class tqdm, so I just doesn't want to import tqdm here..

Copy link
Contributor

Choose a reason for hiding this comment

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

ah got it, that makes sense. i see the real_tqdm import is importing tqdm.auto, which does not have format_num. let's keep the method you copied here, thanks

self._bar.set_description(self._desc)
formatted_progress = format_num(self._progress)
formatted_total = (
format_num(self._bar.total) if self._bar.total is not None else "??"
Copy link
Contributor

Choose a reason for hiding this comment

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

here and elsewhere, i don't think we need the special case for handling if self._bar_total is None. according to tqdm docs, this is already handled by not showing the denominator (only basic stats showing numerator and rows/s):

The expected total number of iterations. If meaningless (None), only basic progress statistics are displayed (no ETA).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Since I update the progress bar manually, I need to make sure the number or None pass to format_num handled correctly. So I add ?? here. As when None is passed to format_num, it will raise error.

Copy link
Contributor

Choose a reason for hiding this comment

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

what if we change to:

formatted_total = (
                format_num(self._bar.total) if self._bar.total is not None else self._bar.total

so we only apply format_num if self._bar_total is not None.

Comment on lines 123 to 125
formatted_progress = format_num(self._progress)
formatted_total = (
format_num(self._bar.total) if self._bar.total is not None else "??"
Copy link
Contributor

Choose a reason for hiding this comment

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

instead of updating the bar description itself and calling format_num() for the updated description, maybe we can pass bar_format parameter to tqdm.tqdm(). specifically in this r_bar section:

r_bar='| {format_num(n_fmt)}/{format_num(total_fmt)} [{elapsed}<{remaining}, ' '{rate_fmt}{postfix}]'

something like this. then, i think we wouldn't need to manually update the entire description here and in _get_state()

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I have tried, this is current what bar looks like
image
but if we use bar_format, it will lose some infomation here, so I think the simplest solution is just change the number.
image

@scottjlee
Copy link
Contributor

@scottjlee I just did some test and actually, I am still so satisfied about my current solution as I add add the format_num/total in between in fact. It's not as easy as I think. Maybe you can check my other changes this is PR first which is to solve the rows then merge it. And I will proposed another PRs for better manager of the commit (In case of problems we know which module has problem as I worried that the current tqdm has issues...). PRs I draft: 1.[Progress bar] To solve this issue. 2 [Logs] other format issues in ray, i.e. when we show bytes when it's large. image 3. The calculation for the AlltoAll operator calculation. Is that OK?

@Bye-legumes that sounds good, thanks for the PR! For the followups, feel free to assign the PR to me, or ask any related questions.

@Bye-legumes
Copy link
Contributor Author

@scottjlee I just did some test and actually, I am still so satisfied about my current solution as I add add the format_num/total in between in fact. It's not as easy as I think. Maybe you can check my other changes this is PR first which is to solve the rows then merge it. And I will proposed another PRs for better manager of the commit (In case of problems we know which module has problem as I worried that the current tqdm has issues...). PRs I draft: 1.[Progress bar] To solve this issue. 2 [Logs] other format issues in ray, i.e. when we show bytes when it's large. image 3. The calculation for the AlltoAll operator calculation. Is that OK?

@Bye-legumes that sounds good, thanks for the PR! For the followups, feel free to assign the PR to me, or ask any related questions.

Thanks! Sorry for confusion! I mean I am not so satisfied with my current solution to progress bar.. So I want to try to find better solution that works for both ray_tqdm and tqdm. I will ping you later when I propose it! Thanks!

@scottjlee
Copy link
Contributor

Once #46816 is applied, we can combine with this PR, to have rows/s be represented in SI units.

@Bye-legumes Bye-legumes mentioned this pull request Jul 30, 2024
5 tasks
@scottjlee
Copy link
Contributor

@Bye-legumes by the way, could you update this PR with a screenshot/output of the progress bar now that #46816 is merged? ideally the example has a large enough row count that we can see the SI unit abbreviation used. once it looks good, i can do another pass on the PR, thanks!

@Bye-legumes
Copy link
Contributor Author

@Bye-legumes by the way, could you update this PR with a screenshot/output of the progress bar now that #46816 is merged? ideally the example has a large enough row count that we can see the SI unit abbreviation used. once it looks good, i can do another pass on the PR, thanks!

image
Hi, this is how is looks like now! Thanks! @scottjlee

@bveeramani bveeramani removed their assignment Jul 31, 2024
Copy link
Contributor

@scottjlee scottjlee left a comment

Choose a reason for hiding this comment

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

Thanks for the contribution!

Copy link
Contributor

@omatthew98 omatthew98 left a comment

Choose a reason for hiding this comment

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

One small question for my understanding but LGTM otherwise. Thanks for the contribution.

@Bye-legumes
Copy link
Contributor Author

@scottjlee OK, so the final step is to test on the premerge and get it merged, right? Thanks!

@scottjlee scottjlee added the go add ONLY when ready to merge, run all tests label Aug 1, 2024
@scottjlee
Copy link
Contributor

@Bye-legumes yes, i will work with our team and get it ready to merge! I am running full tests on premerge now.

@scottjlee
Copy link
Contributor

@Bye-legumes one quick note. Here, could we add an assertion check like assert ref.num_rows() is not None, to ensure we are calling progress_bar.update() with a valid number?

just discussed with my colleague, and we believe this will be true for all cases (except InputDataBuffer, which is already excluded from creating progress bar here). Thanks!

Signed-off-by: zhilong <zhilong.chen@mail.mcgill.ca>
@Bye-legumes
Copy link
Contributor Author

@Bye-legumes one quick note. Here, could we add an assertion check like assert ref.num_rows() is not None, to ensure we are calling progress_bar.update() with a valid number?

just discussed with my colleague, and we believe this will be true for all cases (except InputDataBuffer, which is already excluded from creating progress bar here). Thanks!

I just added and it finished premerge now! I don' think the docs/readthedocs.com:anyscale-ray is related...

@scottjlee
Copy link
Contributor

Just merged in latest master again, to re-trigger the docs build.

@Bye-legumes
Copy link
Contributor Author

Just merged in latest master again, to re-trigger the docs build.

I think all the CIs are failed with that doc build now..

@scottjlee
Copy link
Contributor

Yeah, there is an internal issue where all the docs builds get queued up and timed out. We are working internally to resolve it. Feel free to ignore this PR unless we ping you, you can consider it done for now. We will merge it on our end once we resolve the docs build issue! Thanks for your eyes on it

@Bye-legumes
Copy link
Contributor Author

Yeah, there is an internal issue where all the docs builds get queued up and timed out. We are working internally to resolve it. Feel free to ignore this PR unless we ping you, you can consider it done for now. We will merge it on our end once we resolve the docs build issue! Thanks for your eyes on it

I think it's OK now, can you merge it then I can run my test to test #46924, Thanks!

@bveeramani bveeramani merged commit b3312c1 into ray-project:master Aug 2, 2024
5 checks passed
scottjlee added a commit that referenced this pull request Aug 13, 2024
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

<!-- Please give a short summary of the change and the problem this
solves. -->
Fix a wrong variable name for a feature introduced in
#46699, which caused progress
bars to not show % progress / render the bar itself.

After the changes in this PR, the progress bar shows % progress as
desired:
![Screenshot at Aug 13
14-48-08](https://github.com/user-attachments/assets/f5fc5188-f33e-468c-a460-d3f115293e36)


## Related issue number

<!-- For example: "Closes #1234" -->

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

Signed-off-by: Scott Lee <sjl@anyscale.com>
simonsays1980 pushed a commit to simonsays1980/ray that referenced this pull request Aug 14, 2024
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

<!-- Please give a short summary of the change and the problem this
solves. -->
Fix a wrong variable name for a feature introduced in
ray-project#46699, which caused progress
bars to not show % progress / render the bar itself.

After the changes in this PR, the progress bar shows % progress as
desired:
![Screenshot at Aug 13
14-48-08](https://github.com/user-attachments/assets/f5fc5188-f33e-468c-a460-d3f115293e36)


## Related issue number

<!-- For example: "Closes ray-project#1234" -->

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

Signed-off-by: Scott Lee <sjl@anyscale.com>
simonsays1980 pushed a commit to simonsays1980/ray that referenced this pull request Aug 15, 2024
<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## Why are these changes needed?

<!-- Please give a short summary of the change and the problem this
solves. -->
Fix a wrong variable name for a feature introduced in
ray-project#46699, which caused progress
bars to not show % progress / render the bar itself.

After the changes in this PR, the progress bar shows % progress as
desired:
![Screenshot at Aug 13
14-48-08](https://github.com/user-attachments/assets/f5fc5188-f33e-468c-a460-d3f115293e36)


## Related issue number

<!-- For example: "Closes ray-project#1234" -->

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

Signed-off-by: Scott Lee <sjl@anyscale.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
data Ray Data-related issues go add ONLY when ready to merge, run all tests triage Needs triage (eg: priority, bug/not-bug, and owning component)
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Data] Update Data progress bars to use row as the iteration unit
5 participants