-
Notifications
You must be signed in to change notification settings - Fork 29
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
[REP] Execution Optimizer for Ray Datasets #19
Conversation
Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
reps/2022-12-15-optimizer-data.md
Outdated
|
||
Architecture after REP: | ||
|
||
<img width="945" alt="new-architecture" src="https://user-images.githubusercontent.com/4629931/207807703-bb65db63-41a0-41d9-8e7b-154e1a0ed565.png"> |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Since we are considering re-optimization outside the scope of this REP, can we also remove that from the diagram?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah, removed.
reps/2022-12-15-optimizer-data.md
Outdated
|
||
#### 3.2.1. Interfaces | ||
|
||
NOTE: `OneToOneOperator` used here is the same as `OneToOneOperator` in "Native pipelining support in Ray Datasets" REP. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This section needs to be updated, since the other REP now only proposes PhysicalOperator
.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@ericl - yeah updated. I need more thought to hook up BatchedOperator.process_batches
with PhysicalOperator.add_input/inputs_done/has_next/get_next
. But I think it should be implementation detail that we can figure it out later.
reps/2022-12-15-optimizer-data.md
Outdated
|
||
## Summary | ||
|
||
Build the breakthrough foundation to tackle a series of fundamental issues around Ray Data. The foundation is (1) **lazy execution**, (2) **optimizer**, and (3) **vectorized execution with data batch**. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The summary is a bit low-level right now and solution-heavy. It might be good to focus more on the problems (expensive and unnecessary materialization, current design lacks an optimizer which makes materialization impossible to elide).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Moved this under General Motivation
to make it more coherent, as the top-level summary seem not strictly needed (not see in other REPs).
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes: * Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed. * `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block. * Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing. TODO: - [x] Fix all unit tests - [x] #31459 - [x] #31460 - [ ] Remove the behavior to eagerly compute first block for read - [ ] #31417 - [ ] Update documentation
Signed-off-by: Cheng Su <scnju13@gmail.com>
Signed-off-by: Cheng Su <scnju13@gmail.com>
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes: * Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed. * `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block. * Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing. TODO: - [x] Fix all unit tests - [x] #31459 - [x] #31460 - [ ] Remove the behavior to eagerly compute first block for read - [ ] #31417 - [ ] Update documentation
This PR is to enable lazy execution by default. See ray-project/enhancements#19 for motivation. The change includes: * Change `Dataset` constructor: `Dataset.__init__(lazy: bool = True)`. Also remove `defer_execution` field, as it's no longer needed. * `read_api.py:read_datasource()` returns a lazy `Dataset` with computing the first input block. * Add `ds.fully_executed()` calls to required unit tests, to make sure they are passing. TODO: - [x] Fix all unit tests - [x] ray-project#31459 - [x] ray-project#31460 - [ ] Remove the behavior to eagerly compute first block for read - [ ] ray-project#31417 - [ ] Update documentation Signed-off-by: tmynn <hovhannes.tamoyan@gmail.com>
This REP introduces (1) lazy execution, (2) optimizer, and (3) vectorized execution with data batch, to improve user experience and performance for Ray Datasets.