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

Implement Spark executor node affinity using custom resource #366

Merged
merged 3 commits into from
Jul 25, 2023

Conversation

pang-wu
Copy link
Contributor

@pang-wu pang-wu commented Jul 17, 2023

What Problem Does the PR Solve

This PR enable raydp pin Spark executors on a specific set of machines in a Ray cluster -- a feature we found useful in our production. It is useful when the Ray cluster contains heterogeneous workloads (i.e. workloads using both Spark and Native Ray) and the total resource is less than the max resource Spark could potentially request -- which is possible in following cases:

  • User specify large number of Spark executors, which is more than the total amount of CPU in the cluster. In this case it could cause other Ray workload starving because Spark occupied all resources.
  • Dynamic allocation requests more actors than total CPU -- this is possible when dynamic allocation starts

In both scenarios, we want to limit the scheduling of the Spark executor actors into a subset of machines to avoid the Spark job take all resources in the ray cluster and starve other Ray workloads.
Another scenario this feature is useful is the Spark cluster needs to be schedule on special nodes i.e. spot vs. on demand.

This feature could also benefit multi-tenant Ray clusters where different users want to run their job on different nodegroups.
With the new feature, we can define a set of machines that only for scheduling Spark executors in the Ray cluster, for example:

# Ray cluster config
  # ....
  spark_on_spot:  # Spark only nodes
    resources:
      spark_executor: 100 # custom resource indicates these node group is for Spark only
    min_workers: 2
    max_workers: 10  # changing this also need to change the global max_workers
    node_config:
      # ....
  general_spot:  # Nodes for general Ray workloads
    min_workers: 2
    max_workers: 10  # changing this also need to change the global max_workers
    node_config:
      # ...

Then when initialize Spark session:

spark = raydp.init_spark(app_name='RayDP Example',
                         num_executors=executor_count,
                         executor_cores=3,
                         executor_memory=1 * 1024 * 1024 * 1024,
                         configs = {
                             ...
                             'spark.ray.raydp_spark_executor.actor.resource.spark_executor': 1,  # Schedule executor on nodes with custom resource spark_executor
                         })

@pang-wu pang-wu force-pushed the pang/exec-affinity2 branch from d53f214 to b8c6e74 Compare July 17, 2023 08:12
@carsonwang
Copy link
Collaborator

Thanks for the contributing the PR. This will be very useful. Can you please also update this in the document? Previously you introduced node affinity for the Spark driver and introduced the configuration like spark.ray.raydp_spark_master.actor.resource.spark_master. For this new configuration which is for executor, should we add raydp_executor to the name to be consistent like spark.ray.raydp_executor.actor.resource.spark_executor?

@pang-wu pang-wu force-pushed the pang/exec-affinity2 branch 2 times, most recently from 90b43a8 to e6535aa Compare July 18, 2023 08:09
@pang-wu
Copy link
Contributor Author

pang-wu commented Jul 18, 2023

@carsonwang Thanks for reviewing. Good point, I changed the config to spark.ray.raydp_spark_executor.actor.resource.*
Also updated the doc

@kira-lin
Copy link
Collaborator

why delete the test test_spark_on_fractional_custom_resource?

@pang-wu
Copy link
Contributor Author

pang-wu commented Jul 19, 2023

@kira-lin When doing my local test, I found that test interferes with the new tests -- it could be the Spark context is partially created. I can bring it back.
Another issue is the CI is failing -- I probably need some help there..

@kira-lin
Copy link
Collaborator

@pang-wu Look at raydp.yml:108, that test is run explicitly. CI failure may be due to no tests are selected.

@pang-wu pang-wu force-pushed the pang/exec-affinity2 branch from 13aa306 to 4e1ac6b Compare July 20, 2023 05:27
@pang-wu pang-wu force-pushed the pang/exec-affinity2 branch from 4e1ac6b to e194fcd Compare July 20, 2023 17:27
@pang-wu
Copy link
Contributor Author

pang-wu commented Jul 20, 2023

@kira-lin fixed.

@kira-lin kira-lin merged commit 9a77b96 into oap-project:master Jul 25, 2023
@kira-lin
Copy link
Collaborator

@pang-wu thanks

kira-lin pushed a commit that referenced this pull request Jul 28, 2023
* Implement Spark executor node affinity using custom resource

* reduce test flakeness

* Update document
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.

3 participants