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

[Datasets] Improved naming of Ray Data map tasks #32585

Merged
merged 28 commits into from
Mar 3, 2023

Conversation

scottjlee
Copy link
Contributor

@scottjlee scottjlee commented Feb 15, 2023

Why are these changes needed?

On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

For example, we can run the following script:

ray.init()
ds = ray.data.read_csv("input.csv")

def fn_tasks(x):
	time.sleep(30)
	return x

def fn_actors(x):
	time.sleep(30)
	return x

ds_tasks = ds.map_batches(fn_tasks, compute="tasks")
ds_actors = ds.map_batches(fn_actors, compute="actors")
assert ds_tasks.take_all() == ds_actors.take_all()

which will yield these task names on the dashboard:
Screenshot at Feb 17 12-22-18

Related issue number

Closes #32753

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

Scott Lee added 11 commits February 7, 2023 11:33
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@scottjlee scottjlee marked this pull request as ready for review February 17, 2023 20:29
@c21
Copy link
Contributor

c21 commented Feb 17, 2023

Thanks @scottjlee! cc @clarkzinzow to review as well.

python/ray/data/dataset.py Outdated Show resolved Hide resolved
@@ -117,7 +117,7 @@ def _dispatch_tasks(self):
bundle = self._bundle_queue.popleft()
input_blocks = [block for block, _ in bundle.blocks]
ctx = TaskContext(task_idx=self._next_task_idx)
ref = actor.submit.options(num_returns="dynamic").remote(
ref = actor.submit.options(num_returns="dynamic", name=self.name).remote(
Copy link
Contributor

Choose a reason for hiding this comment

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

@rkooo567 - I thought you mentioned there's problem that actor should not be set name here, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So if we look at the dash screenshot above, I believe this isn't actually setting the name of the actor -- as we can see _MapWorker still being used in the Active Actors by Name chart. I think this part instead sets the name of the submitted task instead (e.g. fn_actors in above example).

Copy link
Contributor

Choose a reason for hiding this comment

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

Also the name of actors should be unique across the cluster. So let's avoid setting the name for now.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yep this is just setting the name of the actor task, so this should be fine, right?

@@ -290,6 +293,9 @@ def submit(
) -> Iterator[Union[Block, List[BlockMetadata]]]:
yield from _map_task(fn, ctx, *blocks)

def __repr__(self):
return f"MapWorker({self.src_fn_name})"
Copy link
Contributor

Choose a reason for hiding this comment

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

I am actually feeling the MapWorker naming might be confusing for user, shall we rename it to be more specific - like MapBatchesActor? @clarkzinzow

Copy link
Contributor

Choose a reason for hiding this comment

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

Agreed, but I think that the current MapWorker(MapBatches(fn_name)) is pretty good!

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Once #32922 is implemented, this should allow us to send this actor name to the dash

Signed-off-by: Scott Lee <sjl@anyscale.com>
@rkooo567
Copy link
Contributor

I don't have specific comments to the PR itself, but I'd like to do some dogfooding together with a couple of data team members after merging this PR!

Scott Lee added 2 commits February 21, 2023 14:39
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@c21
Copy link
Contributor

c21 commented Feb 22, 2023

Assuming CI test will be fixed, the change itself looks good to me. WDYT? @clarkzinzow.

@scottjlee scottjlee changed the title [Datasets] Improved naming of Ray Data map tasks/actors [Datasets] Improved naming of Ray Data map tasks Feb 22, 2023
Signed-off-by: Scott Lee <sjl@anyscale.com>
Scott Lee added 4 commits February 24, 2023 15:53
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@clarkzinzow
Copy link
Contributor

@scottjlee Ah it looks like some AIR tests that check the stats string content need to be updated as well. https://buildkite.com/ray-project/oss-ci-build-pr/builds/13201#018694ad-6e8b-4256-aba0-5a70395be959

@clarkzinzow clarkzinzow added the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 27, 2023
Signed-off-by: Scott Lee <sjl@anyscale.com>
@scottjlee scottjlee removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Feb 28, 2023
Scott Lee added 2 commits February 28, 2023 10:55
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Scott Lee added 3 commits February 28, 2023 15:18
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Scott Lee <sjl@anyscale.com>
@c21 c21 mentioned this pull request Mar 1, 2023
7 tasks
Signed-off-by: Scott Lee <sjl@anyscale.com>
@c21
Copy link
Contributor

c21 commented Mar 3, 2023

The test failure looks irrelevant.

@clarkzinzow clarkzinzow merged commit 39ae7b6 into ray-project:master Mar 3, 2023
@rkooo567
Copy link
Contributor

awesome!

@rkooo567
Copy link
Contributor

@scottjlee let's try some dogfooding? I can schedule a meeting

ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request Mar 21, 2023
On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Jack He <jackhe2345@gmail.com>
edoakes pushed a commit to edoakes/ray that referenced this pull request Mar 22, 2023
On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Edward Oakes <ed.nmi.oakes@gmail.com>
peytondmurray pushed a commit to peytondmurray/ray that referenced this pull request Mar 22, 2023
On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

Signed-off-by: Scott Lee <sjl@anyscale.com>
elliottower pushed a commit to elliottower/ray that referenced this pull request Apr 22, 2023
On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: elliottower <elliot@elliottower.com>
ProjectsByJackHe pushed a commit to ProjectsByJackHe/ray that referenced this pull request May 4, 2023
On the Ray Dashboard, Data-related tasks/actors are currently summarized based on their name, which currently is generic and unhelpful in distinguishing between multiple Ray Data usages. We can include a more detailed name which contains the underlying function name when scheduling the task, so that they are easily summarized on the dashboard.

Signed-off-by: Scott Lee <sjl@anyscale.com>
Signed-off-by: Jack He <jackhe2345@gmail.com>
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.

[Dataset] Set informative Dataset task/actor name to propagate to Ray dashboard
4 participants