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] New executor [7/n]--- bare bones implementation of StreamingExecutor #31579

Merged
merged 24 commits into from
Jan 17, 2023

Conversation

ericl
Copy link
Contributor

@ericl ericl commented Jan 11, 2023

Why are these changes needed?

Initial implementation of ray-project/enhancements#18, dependent on #30903

Streaming execution can be toggled with the following env var: RAY_DATASET_USE_STREAMING_EXECUTOR=0|1.

Initial PR TODOs:

  • Add basic tests and demo code

Future TODOs:

  • Implement stats
  • Implement resource limits
  • Handle autoscaling actor pools

@ericl ericl changed the title [WIP] Prototype the streaming executor API [data] New executor [7/n]--- bare bones implementation of StreamingExecutor Jan 12, 2023
Signed-off-by: Eric Liang <[email protected]>
Signed-off-by: Eric Liang <[email protected]>
Signed-off-by: Eric Liang <[email protected]>
Copy link
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

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

Might be missing something, but actually I wonder if we really need the inputs_done interface? I think this is needed for all-to-all operations and to flush at the end of the job, right?

For all-to-all, I think we could move the logic inside the bulk_fn that's passed to the operator. For example, we can call the bulk_fn on each input added, and bulk_fn would only run the shuffle once it's received the expected number of inputs. This also better matches what the operator would look like in the windowed shuffle case.

For the flush case, we can track the done inputs in the op state as we are right now, but then only call op.flush() once per op instead of once per op input.

Fine leaving this cleanup to later but it occurred to me that it may simplify the current code.

return selected


def dispatch_next_task(op_state: OpState) -> None:
Copy link
Contributor

Choose a reason for hiding this comment

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

Should this just be a method of OpState?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

@@ -78,6 +79,7 @@ def add_input(self, refs: RefBundle, input_index: int) -> None:

def inputs_done(self, input_index: int) -> None:
self._execution_state.inputs_done(input_index)
self._inputs_done = True
Copy link
Contributor

Choose a reason for hiding this comment

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

Is there an assumption here that there is only one input (add a note?)?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

self._completed = True

def completed(self) -> bool:
return self._completed
Copy link
Contributor

Choose a reason for hiding this comment

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

Why don't we need to check whether self.has_next() as we do in the map operator?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done (moved to physical op class).

@@ -94,6 +96,11 @@ def get_work_refs(self) -> List[ray.ObjectRef]:
def notify_work_completed(self, task: ray.ObjectRef) -> None:
self._execution_state.work_completed(task)

def completed(self) -> bool:
return (
self._inputs_done and len(self.get_work_refs()) == 0 and not self.has_next()
Copy link
Contributor

Choose a reason for hiding this comment

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

Seems like this definition could be shared across the different operators?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

python/ray/data/tests/test_operators.py Show resolved Hide resolved
op_state = OpState(o2)
o2.add_input = MagicMock()
op_state.inqueues[0].append("dummy1")
dispatch_next_task(op_state)
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggest running this multiple times so we can test indices other than 0.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Added a test for multiple inputs, and a TODO for multiple indices.

Copy link
Contributor Author

@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.

Updated. Regarding removing the completion method, I don't think that's possible if we want to support operators with unknown output size in general. Currently, we have a method that returns an estimate of the number of outputs for the progress bar, but this is allowed to return None for operators that don't know their number of outputs at planning time.

self._completed = True

def completed(self) -> bool:
return self._completed
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done (moved to physical op class).

@@ -78,6 +79,7 @@ def add_input(self, refs: RefBundle, input_index: int) -> None:

def inputs_done(self, input_index: int) -> None:
self._execution_state.inputs_done(input_index)
self._inputs_done = True
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed.

@@ -94,6 +96,11 @@ def get_work_refs(self) -> List[ray.ObjectRef]:
def notify_work_completed(self, task: ray.ObjectRef) -> None:
self._execution_state.work_completed(task)

def completed(self) -> bool:
return (
self._inputs_done and len(self.get_work_refs()) == 0 and not self.has_next()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

return selected


def dispatch_next_task(op_state: OpState) -> None:
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Done.

python/ray/data/tests/test_operators.py Show resolved Hide resolved
op_state = OpState(o2)
o2.add_input = MagicMock()
op_state.inqueues[0].append("dummy1")
dispatch_next_task(op_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.

Added a test for multiple inputs, and a TODO for multiple indices.

@stephanie-wang
Copy link
Contributor

Updated. Regarding removing the completion method, I don't think that's possible if we want to support operators with unknown output size in general. Currently, we have a method that returns an estimate of the number of outputs for the progress bar, but this is allowed to return None for operators that don't know their number of outputs at planning time.

Hmm I see, yeah I was thinking we would just flush once enough outputs have been accumulated, but I guess it ends up being the same thing.

It's more minor, but maybe we could at least change the signature from inputs_done(self, input_index: int) to inputs_done(self)? I don't think we actually need to track the input indices in the operators as long as we're tracking in the OpState.

@ericl
Copy link
Contributor Author

ericl commented Jan 12, 2023

Yup good point, I can't think of where an operator would need the index for the done signal. Removed.

Copy link
Contributor

@stephanie-wang stephanie-wang left a comment

Choose a reason for hiding this comment

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

LGTM!

Signed-off-by: Eric Liang <[email protected]>
Copy link
Contributor

@clarkzinzow clarkzinzow left a comment

Choose a reason for hiding this comment

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

LGTM overall, mostly questions and impl/comment/test nits.

Copy link
Contributor

@jianoaix jianoaix left a comment

Choose a reason for hiding this comment

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

LGTM

Copy link
Contributor Author

@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.

All comments addressed, ptal.

Signed-off-by: Eric Liang <[email protected]>
@ericl ericl merged commit d66d722 into ray-project:master Jan 17, 2023
andreapiso pushed a commit to andreapiso/ray that referenced this pull request Jan 22, 2023
…ecutor (ray-project#31579)

Initial implementation of ray-project/enhancements#18, dependent on ray-project#30903

Streaming execution can be toggled with the following env var: RAY_DATASET_USE_STREAMING_EXECUTOR=0|1.

Signed-off-by: Andrea Pisoni <[email protected]>
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.

5 participants