From a7a1ad1b94af2b3af4e4ba63bd1c4fc6b21ea0f5 Mon Sep 17 00:00:00 2001 From: Weiwei Yang Date: Mon, 22 Aug 2022 22:06:20 -0700 Subject: [PATCH 1/4] [SPARK-40187] Add doc for using Apache YuniKorn as a customized scheduler --- docs/running-on-kubernetes.md | 44 +++++++++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 5dba9ee2be510..1e81633f40da7 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1811,6 +1811,50 @@ spec: queue: default ``` +#### Using Apache YuniKorn as Customized Scheduler for Spark on Kubernetes + +[Apache YuniKorn](https://yunikorn.apache.org/) is a resource scheduler for Kubernetes that provides advanced batch scheduling +capabilities, such as job queuing, resource fairness, min/max queue capacity and flexible job ordering policies. +For available Apache YuniKorn features, please refer to [this doc](https://yunikorn.apache.org/docs/next/get_started/core_features). + +##### Prerequisites + +Install Apache YuniKorn: + +```bash +helm repo add yunikorn https://apache.github.io/yunikorn-release +helm repo update +kubectl create namespace yunikorn +helm install yunikorn yunikorn/yunikorn --namespace yunikorn +``` + +the above steps will install the latest version of YuniKorn on an existing Kubernetes cluster. + +##### Get started + +Submit Spark jobs with the following extra options: + +```bash +--conf spark.kubernetes.scheduler.name=yunikorn +--conf spark.kubernetes.driver.annotation.yunikorn.apache.org/app-id={{APP_ID}} +--conf spark.kubernetes.executor.annotation.yunikorn.apache.org/app-id={{APP_ID}} +``` + +Note, `{{APP_ID}}` is the builtin variable that will be substituted with Spark job ID automatically. +With the above configuration, the job will be scheduled by YuniKorn scheduler instead of the default Kubernetes scheduler. + +##### Work with YuniKorn queues + +Apache YuniKorn supports 2 types of resource queues: + +- Static +- Dynamic + +The static queues are predefined in YuniKorn configmap, and the dynamic queues are automatically created by the scheduler +based on [placement rules](https://yunikorn.apache.org/docs/next/user_guide/placement_rules). Spark supports to run with +both queue setup. Refer to this [doc](https://yunikorn.apache.org/docs/next/user_guide/resource_quota_management) for more +information how to run Spark with different queue setup. + ### Stage Level Scheduling Overview Stage level scheduling is supported on Kubernetes when dynamic allocation is enabled. This also requires spark.dynamicAllocation.shuffleTracking.enabled to be enabled since Kubernetes doesn't support an external shuffle service at this time. The order in which containers for different profiles is requested from Kubernetes is not guaranteed. Note that since dynamic allocation on Kubernetes requires the shuffle tracking feature, this means that executors from previous stages that used a different ResourceProfile may not idle timeout due to having shuffle data on them. This could result in using more cluster resources and in the worst case if there are no remaining resources on the Kubernetes cluster then Spark could potentially hang. You may consider looking at config spark.dynamicAllocation.shuffleTracking.timeout to set a timeout, but that could result in data having to be recomputed if the shuffle data is really needed. From fb50eabe837df625f60cf4eed37cc3c0cca33efe Mon Sep 17 00:00:00 2001 From: Weiwei Yang Date: Mon, 22 Aug 2022 22:11:38 -0700 Subject: [PATCH 2/4] minor updates --- docs/running-on-kubernetes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 1e81633f40da7..df37d9220feb9 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1853,7 +1853,7 @@ Apache YuniKorn supports 2 types of resource queues: The static queues are predefined in YuniKorn configmap, and the dynamic queues are automatically created by the scheduler based on [placement rules](https://yunikorn.apache.org/docs/next/user_guide/placement_rules). Spark supports to run with both queue setup. Refer to this [doc](https://yunikorn.apache.org/docs/next/user_guide/resource_quota_management) for more -information how to run Spark with different queue setup. +information about how to run Spark with different queue setup. ### Stage Level Scheduling Overview From 9dd97fbbab171c5eafbdb98e443117859b90deef Mon Sep 17 00:00:00 2001 From: Weiwei Yang Date: Tue, 23 Aug 2022 11:14:07 -0700 Subject: [PATCH 3/4] address review comments --- docs/running-on-kubernetes.md | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index df37d9220feb9..51b06f62015f6 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1815,7 +1815,7 @@ spec: [Apache YuniKorn](https://yunikorn.apache.org/) is a resource scheduler for Kubernetes that provides advanced batch scheduling capabilities, such as job queuing, resource fairness, min/max queue capacity and flexible job ordering policies. -For available Apache YuniKorn features, please refer to [this doc](https://yunikorn.apache.org/docs/next/get_started/core_features). +For available Apache YuniKorn features, please refer to [core features](https://yunikorn.apache.org/docs/get_started/core_features). ##### Prerequisites @@ -1828,7 +1828,7 @@ kubectl create namespace yunikorn helm install yunikorn yunikorn/yunikorn --namespace yunikorn ``` -the above steps will install the latest version of YuniKorn on an existing Kubernetes cluster. +The above steps will install the latest version of YuniKorn on an existing Kubernetes cluster. ##### Get started @@ -1840,7 +1840,7 @@ Submit Spark jobs with the following extra options: --conf spark.kubernetes.executor.annotation.yunikorn.apache.org/app-id={{APP_ID}} ``` -Note, `{{APP_ID}}` is the builtin variable that will be substituted with Spark job ID automatically. +Note that `{{APP_ID}}` is the built-in variable that will be substituted with Spark job ID automatically. With the above configuration, the job will be scheduled by YuniKorn scheduler instead of the default Kubernetes scheduler. ##### Work with YuniKorn queues @@ -1851,10 +1851,14 @@ Apache YuniKorn supports 2 types of resource queues: - Dynamic The static queues are predefined in YuniKorn configmap, and the dynamic queues are automatically created by the scheduler -based on [placement rules](https://yunikorn.apache.org/docs/next/user_guide/placement_rules). Spark supports to run with -both queue setup. Refer to this [doc](https://yunikorn.apache.org/docs/next/user_guide/resource_quota_management) for more +based on [placement rules](https://yunikorn.apache.org/docs/user_guide/placement_rules). Spark supports to run with +both queue setup. Refer to this [resource quota management](https://yunikorn.apache.org/docs/user_guide/resource_quota_management) for more information about how to run Spark with different queue setup. +##### Limitations + +- Apache YuniKorn currently only supports x86 Linux, running Spark on ARM64 with Apache YuniKorn is not supported at present. + ### Stage Level Scheduling Overview Stage level scheduling is supported on Kubernetes when dynamic allocation is enabled. This also requires spark.dynamicAllocation.shuffleTracking.enabled to be enabled since Kubernetes doesn't support an external shuffle service at this time. The order in which containers for different profiles is requested from Kubernetes is not guaranteed. Note that since dynamic allocation on Kubernetes requires the shuffle tracking feature, this means that executors from previous stages that used a different ResourceProfile may not idle timeout due to having shuffle data on them. This could result in using more cluster resources and in the worst case if there are no remaining resources on the Kubernetes cluster then Spark could potentially hang. You may consider looking at config spark.dynamicAllocation.shuffleTracking.timeout to set a timeout, but that could result in data having to be recomputed if the shuffle data is really needed. From 4cf1a1d354b8dfbee643f5e906165fd1f398dd32 Mon Sep 17 00:00:00 2001 From: Weiwei Yang Date: Wed, 24 Aug 2022 23:04:15 -0700 Subject: [PATCH 4/4] address review comments --- docs/running-on-kubernetes.md | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 51b06f62015f6..ed8f085b8bf95 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1825,10 +1825,10 @@ Install Apache YuniKorn: helm repo add yunikorn https://apache.github.io/yunikorn-release helm repo update kubectl create namespace yunikorn -helm install yunikorn yunikorn/yunikorn --namespace yunikorn +helm install yunikorn yunikorn/yunikorn --namespace yunikorn --version 1.0.0 ``` -The above steps will install the latest version of YuniKorn on an existing Kubernetes cluster. +The above steps will install YuniKorn v1.0.0 on an existing Kubernetes cluster. ##### Get started @@ -1843,21 +1843,9 @@ Submit Spark jobs with the following extra options: Note that `{{APP_ID}}` is the built-in variable that will be substituted with Spark job ID automatically. With the above configuration, the job will be scheduled by YuniKorn scheduler instead of the default Kubernetes scheduler. -##### Work with YuniKorn queues - -Apache YuniKorn supports 2 types of resource queues: - -- Static -- Dynamic - -The static queues are predefined in YuniKorn configmap, and the dynamic queues are automatically created by the scheduler -based on [placement rules](https://yunikorn.apache.org/docs/user_guide/placement_rules). Spark supports to run with -both queue setup. Refer to this [resource quota management](https://yunikorn.apache.org/docs/user_guide/resource_quota_management) for more -information about how to run Spark with different queue setup. - ##### Limitations -- Apache YuniKorn currently only supports x86 Linux, running Spark on ARM64 with Apache YuniKorn is not supported at present. +- Apache YuniKorn currently only supports x86 Linux, running Spark on ARM64 (or other platform) with Apache YuniKorn is not supported at present. ### Stage Level Scheduling Overview