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] Implement push-based shuffle #23758

Merged
merged 47 commits into from
Apr 27, 2022

Conversation

stephanie-wang
Copy link
Contributor

@stephanie-wang stephanie-wang commented Apr 6, 2022

Why are these changes needed?

The simple shuffle currently implemented in Datasets does not reliably scale past 1000+ partitions due to metadata and I/O overhead.

This PR adds an experimental shuffle implementation for a "push-based shuffle", as described in this paper draft. This algorithm should see better performance at larger data scales. The algorithm works by merging intermediate map outputs at the reducer side while other map tasks are executing. Then, a final reduce task merges these merged outputs.

Currently, the PR exposes this option through the DatasetContext. It can also be set through a hidden OS environment variable (RAY_DATASET_PUSH_BASED_SHUFFLE). Once we have more comprehensive benchmarks, we can better document this option and allow the algorithm to be chosen at run time.

Related issue number

Closes #23758.

@stephanie-wang stephanie-wang changed the title [WIP][Datasets] Implement push-based shuffle [Datasets] Implement push-based shuffle Apr 14, 2022
@stephanie-wang stephanie-wang marked this pull request as ready for review April 14, 2022 20:15
@stephanie-wang stephanie-wang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Apr 19, 2022
@stephanie-wang
Copy link
Contributor Author

An initial run of sort on 1TB, 1k partitions produces 1h on 20 nodes vs 1h20min with simple shuffle. Looking into the performance a bit, there is definitely room for improvement on this PR, specifically around improving pipelining during the map-merge stage. But I think we should merge this version in first just to get the skeleton in there.

merge_tasks_assigned += 1
leftover_cpu_map[node_id] -= 1
merge_task_args = [
{"scheduling_strategy": NodeAffinitySchedulingStrategy(node_id, soft=True)}
Copy link
Contributor

Choose a reason for hiding this comment

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

Should we also use NodeAffinitySchedulingStrategy to colocate merge_factor number of map tasks with the merge task?

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 think it is actually better to let Ray schedule the map tasks in case there are other factors like data locality being used to schedule those. I'll make a note of this in the code!

python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
# reducer.
merge_task_args = self._compute_merge_task_options(
num_merge_tasks_per_round, merge_factor, cpu_map
)
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we unit test the above topology configuration logic? Concretely, we could split it out into helper method that returns a struct/dict, and we can unit test the topology configured for various cluster scenarios. This would also help with splitting the code up for readability.

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
"map": shuffle_map_metadata,
"merge": shuffle_merge_metadata,
"reduce": new_metadata,
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Add some test to test_stats.py?

@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 21, 2022
Copy link
Contributor

@scv119 scv119 left a comment

Choose a reason for hiding this comment

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

This looks great! Some high level code structure suggestions.

@@ -28,6 +29,11 @@
# Whether to furthermore fuse prior map tasks with shuffle stages.
DEFAULT_OPTIMIZE_FUSE_SHUFFLE_STAGES = True

# Whether to use push-based shuffle by default.
DEFAULT_USE_PUSH_BASED_SHUFFLE = bool(
Copy link
Contributor

Choose a reason for hiding this comment

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

do we need to set this env for all nodes?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ah right, good point... It only needs to be set on whoever calls the Dataset shuffle call; after that I think the context will get propagated through the usual mechanism. Do you think that's an issue?

Copy link
Contributor

Choose a reason for hiding this comment

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

cc @clarkzinzow for best practice/convention here

Copy link
Contributor

Choose a reason for hiding this comment

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

@stephanie-wang @scv119 I think that this is fine, since we already rely on the user being able to mutate the global context in the driver before doing Dataset operations and expect those default overrides to propagate to all tasks, so this is no different IMO.

python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
# reducer.
merge_task_args = self._compute_merge_task_options(
num_merge_tasks_per_round, merge_factor, cpu_map
)
Copy link
Contributor

Choose a reason for hiding this comment

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

+1

python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
python/ray/data/impl/push_based_shuffle.py Outdated Show resolved Hide resolved
@stephanie-wang stephanie-wang removed the @author-action-required The PR author is responsible for the next step. Remove tag to send back to the reviewer. label Apr 22, 2022
@ericl
Copy link
Contributor

ericl commented Apr 22, 2022

I think some tests are failing here. I reviewed this at a high level and am ok with merging this to unblock further testing, though we might want to revisit the code structure at a later point (could be after we're happy with the performance).

@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 22, 2022
@@ -28,6 +29,11 @@
# Whether to furthermore fuse prior map tasks with shuffle stages.
DEFAULT_OPTIMIZE_FUSE_SHUFFLE_STAGES = True

# Whether to use push-based shuffle by default.
DEFAULT_USE_PUSH_BASED_SHUFFLE = bool(
Copy link
Contributor

Choose a reason for hiding this comment

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

cc @clarkzinzow for best practice/convention here

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, same thoughts as @ericl: I have some ideas of how to improve the code structure here, but this looks good enough to merge, so let's unblock next steps and revisit after perf testing.

@@ -28,6 +29,11 @@
# Whether to furthermore fuse prior map tasks with shuffle stages.
DEFAULT_OPTIMIZE_FUSE_SHUFFLE_STAGES = True

# Whether to use push-based shuffle by default.
DEFAULT_USE_PUSH_BASED_SHUFFLE = bool(
Copy link
Contributor

Choose a reason for hiding this comment

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

@stephanie-wang @scv119 I think that this is fine, since we already rely on the user being able to mutate the global context in the driver before doing Dataset operations and expect those default overrides to propagate to all tasks, so this is no different IMO.

Copy link
Contributor

@franklsf95 franklsf95 left a comment

Choose a reason for hiding this comment

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

Awesome seeing this work going into production! I left a few questions inline.

One thing we should keep in mind is data skew. The current pipelining schedule will not work very well if the input partitions are too imbalanced.

) -> Tuple[BlockList, Dict[str, List[BlockMetadata]]]:
logger.info("Using experimental push-based shuffle.")
# TODO(swang): For jobs whose reduce work is heavier than the map work,
# we should support fractional merge factors.
Copy link
Contributor

Choose a reason for hiding this comment

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

I don't understand what fractional merge factors mean. Like, how would a merge task merge 1.5x map outputs? And why is this related with reduce/map workload ratio?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It just means you have more merge tasks than map tasks. So if you have map:merge = 1/2 and 10 map tasks for every round, each merge task will output 1/2 of a partition's worth of data. Presumably you'd also want to have many more reduce tasks than map tasks in this scenario.

But yeah, there's no evidence yet that we'd want to support this case. I haven't seen a case yet where the reduce-side computation is heavier than the map.

@stephanie-wang stephanie-wang merged commit c1054a0 into ray-project:master Apr 27, 2022
@stephanie-wang stephanie-wang deleted the shuffle-op branch April 27, 2022 18:59
jjyao added a commit to jjyao/ray that referenced this pull request Apr 28, 2022
jjyao added a commit that referenced this pull request Apr 28, 2022
stephanie-wang added a commit that referenced this pull request Apr 28, 2022
The simple shuffle currently implemented in Datasets does not reliably scale past 1000+ partitions due to metadata and I/O overhead.

This PR adds an experimental shuffle implementation for a "push-based shuffle", as described in this paper draft. This algorithm should see better performance at larger data scales. The algorithm works by merging intermediate map outputs at the reducer side while other map tasks are executing. Then, a final reduce task merges these merged outputs.

Currently, the PR exposes this option through the DatasetContext. It can also be set through a hidden OS environment variable (RAY_DATASET_PUSH_BASED_SHUFFLE). Once we have more comprehensive benchmarks, we can better document this option and allow the algorithm to be chosen at run time.

Redo for #23758 to fix CI.
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.

None yet

6 participants