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] Implement limit physical operator #34705

Merged
merged 12 commits into from
Apr 25, 2023

Conversation

raulchen
Copy link
Contributor

@raulchen raulchen commented Apr 24, 2023

Why are these changes needed?

  • Implemented the Limit physical operator for streaming execution.
  • Added the LimitStage for legacy compatibility.

Note, currently when the limit operator reaches the limit, the upstream operators still won't stop producing data. This will be optimized in a follow-up PR.

Related issue number

#34234

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: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
# If we don't know the number of rows in the input, try to
# split at the maximum number of rows we can consume
# (`self._limit - self._consumed_rows`).
blocks_splits, metadata_splits = _split_at_indices(
Copy link
Contributor

Choose a reason for hiding this comment

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

Hmm this is a blocking call right? I think we want to ensure all operators are streaming compatible. Maybe there should be an op that we insert before the Limit that ensures all bundles have num_rows set, fetching it if needed.

Copy link
Contributor

Choose a reason for hiding this comment

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

Actually, can we assume num_rows is always available? After the read, I think this should always be populated.

@raulchen raulchen changed the title [WIP][Data] Optimize limit operator [Data] Implement limit physical operator Apr 24, 2023
return self._buffer.popleft()

def get_stats(self) -> StatsDict:
return {self._name: self._output_metadata}
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not sure if this is the correct way to handle stats. I just follow map_operator. Is there any docs?

Copy link
Contributor

Choose a reason for hiding this comment

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

Seems reasonable, probably we should improve the docstring in interfaces.py

# Slice the last block.
num_rows_to_take = self._limit - self._consumed_rows
self._consumed_rows = self._limit
block = BlockAccessor.for_block(ray.get(block)).slice(
Copy link
Contributor

Choose a reason for hiding this comment

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

We should still run it in a remote task to avoid needing to fetch the data block locally. This avoids the ray.put() later on.

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 purpose of doing this is to avoid putting too much data on the driver side? Or is there any other reason why we shouldn't do ray.put?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yeah, avoiding fetching large blocks to the driver is important, and it also avoids an extra data copy. If the data is split in a remote task, then only one put() happens, instead of two.

@@ -115,3 +115,18 @@ def __init__(
)
self._key = key
self._aggs = aggs


class Limit(AbstractAllToAll):
Copy link
Contributor

@ericl ericl Apr 24, 2023

Choose a reason for hiding this comment

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

Should this extend AbstractOneToOne instead of AbstractAllToAll?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This is the logical operator. Not used right now, but I added this for completeness of the Datastream.limit API.
I think limit is also a kind of all-to-all logical operator. So this should make sense? Or do you prefer to have it directly extend to LogicalOperator.

Copy link
Contributor

Choose a reason for hiding this comment

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

Don't have a strong opinion, but it makes sense that it should extend LogicalOperator to me and not AllToAll, since there's technically not any all:all data dependency between rows.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Makes sense. Fixed.

if logical_plan is not None:
op = Limit(logical_plan.dag, limit=limit)
logical_plan = LogicalPlan(op)
return Datastream(plan, self._epoch, self._lazy, logical_plan)
Copy link
Contributor

Choose a reason for hiding this comment

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

Nice.

@ericl
Copy link
Contributor

ericl commented Apr 24, 2023

I tried this out locally and it seems we don't shutdown the first operator after the limit is reached. I guess that would be in the second PR only?

RAY_DATA_VERBOSE_PROGRESS=1 python
Screenshot from 2023-04-24 15-23-51

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.

LGTM assuming you want to do the early shutdown in the second PR. Existing tests will suffice for this refactor.

@raulchen
Copy link
Contributor Author

LGTM assuming you want to do the early shutdown in the second PR. Existing tests will suffice for this refactor.

Yep, I plan to do that in a second PR. There are multiple ways to do this. I'll comment on the original issue. Let's continue discussing there.

Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
Signed-off-by: Hao Chen <[email protected]>
@ericl
Copy link
Contributor

ericl commented Apr 25, 2023

Some tests seem to be failing with "ValueError: The size in bytes of the block must be known: (ObjectRef(00ffffffffffffffffffffffffffffffffffffff0100000056010000), BlockMetadata(num_rows=10, size_bytes=None, schema=None, input_files=[], exec_stats=None))", but I think this is a bug in the test.

The new streaming executor assumes that at least size_bytes is always known for all datasources. So I think we should fill in size_bytes for these tests and/or disable these tests.

Previous the tests passed since we bypassed operator execution entirely with the old limit() impl.

@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 Apr 25, 2023
@raulchen raulchen merged commit c4a29b2 into ray-project:master Apr 25, 2023
@raulchen raulchen deleted the optimize-limit branch April 25, 2023 23:35
@c21
Copy link
Contributor

c21 commented Apr 26, 2023

Tests are still failing, let's fix them in master.
Screen Shot 2023-04-25 at 11 01 13 PM

Screen Shot 2023-04-25 at 11 01 50 PM

@raulchen raulchen mentioned this pull request Apr 26, 2023
8 tasks
raulchen added a commit that referenced this pull request Apr 27, 2023
## Why are these changes needed?

Fixes a bug introduced by #34705 

## Related issue number

#34234
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
## Why are these changes needed?

- Implemented the Limit physical operator for streaming execution.
- Added the `LimitStage` for legacy compatibility.

Note, currently when the limit operator reaches the limit, the upstream operators still won't stop producing data. This will be optimized in a follow-up PR.

## Related issue number

ray-project#34234

Signed-off-by: Jack He <[email protected]>
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
## Why are these changes needed?

Fixes a bug introduced by ray-project#34705

## Related issue number

ray-project#34234

Signed-off-by: Jack He <[email protected]>
architkulkarni pushed a commit to architkulkarni/ray that referenced this pull request May 16, 2023
## Why are these changes needed?

- Implemented the Limit physical operator for streaming execution. 
- Added the `LimitStage` for legacy compatibility.

Note, currently when the limit operator reaches the limit, the upstream operators still won't stop producing data. This will be optimized in a follow-up PR.

## Related issue number

ray-project#34234
architkulkarni pushed a commit to architkulkarni/ray that referenced this pull request May 16, 2023
## Why are these changes needed?

Fixes a bug introduced by ray-project#34705 

## Related issue number

ray-project#34234
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.

4 participants