You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Yikun Jiang <yi...@gmail.com> on 2022/02/24 03:35:05 UTC

Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

First, much thanks for all your help (Spark/Volcano/Yunikorn community) to
make this SPIP happen!

Especially,@dongjoon-hyun @holdenk @william-wang @attilapiros @HyukjinKwon
@martin-g @yangwwei @tgravescs

The SPIP is near the end of the stage. It can be said that it is beta
available at the basic level.

I also draft a simple slide to show how to use and help you understand what
we have done:
https://docs.google.com/presentation/d/1XDsTWPcsBe4PQ-1MlBwd9pRl8mySdziE_dJE6iATNw8

Below are also some recap to help you understand current implementation and
next step on SPIP:

*# Existing work*
*## Basic part:*
- SPARK-36059 <https://issues.apache.org/jira/browse/SPARK-36059> *New
configuration:* ability to specify "schedulerName" in driver/executor for
Spark on K8S
- SPARK-37331 <https://issues.apache.org/jira/browse/SPARK-37331> *New
workflow:*ability to create pre-populated resources before driver pod  for
Spark on K8S
- SPARK-37145 <https://issues.apache.org/jira/browse/SPARK-37145> *New
developer API:* support user feature step with configuration for Spark on
K8S
- *(reviewing)* *New Job Configurations* for Spark on K8S:
  - SPARK-38188 <https://issues.apache.org/jira/browse/SPARK-38188>:
spark.kubernetes.job.queue
  - SPARK-38187 <https://issues.apache.org/jira/browse/SPARK-38187>:
spark.kubernetes.job.[minCPU|minMemory]
  - SPARK-38189 <https://issues.apache.org/jira/browse/SPARK-38189>:
spark.kubernetes.job.priorityClassName

*## Volcano Part:*
- SPARK-37258 <https://issues.apache.org/jira/browse/SPARK-37258> *New
volcano extension* in kubernetes-client fabric8io/kubernetes-client#3579
- SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
profile: *-Pvolcano
- SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
Feature Step:* VolcanoFeatureStep
- SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
integration test:*
 *- Passed on x86 and Arm64 (Linux on Huawei Kunpeng 920 and MacOS on Apple
Silicon M1).*
 - Test basic volcano workflow
 - Test all existing tests based on the volcano.

*## Yunikorn Part:*
@yangwwei  will also make the efforts for Yunikorn module feature step
since this week.
I will help to complete the yunikorn integration based on previous
experience.

*# Next Plan*
There are also 3 main tasks to be completed before v3.3 code freeze:
1. (reviewing) SPARK-38188
<https://issues.apache.org/jira/browse/SPARK-38188>: Support queue
scheduling configuration
https://github.com/apache/spark/pull/35553
2. (reviewing) SPARK-38187
<https://issues.apache.org/jira/browse/SPARK-38187>: Support resource
reservation (minCPU/minMemory configuration)
https://github.com/apache/spark/pull/35640
3. (reviewing) SPARK-38187
<https://issues.apache.org/jira/browse/SPARK-38187>: Support priority
scheduling (priorityClass configuration):
https://issues.apache.org/jira/browse/SPARK-38189
https://github.com/apache/spark/pull/35639
4. (WIP) SPARK-37809 <https://issues.apache.org/jira/browse/SPARK-37809>:
Yunikorn integration

Also several misc work is gonna be completed before 3.3:
1. Integrated volcano deploy into integration test (x86 and arm)
- Add it to spark kubernetes integration test once cross compile support:
https://github.com/volcano-sh/volcano/pull/1571
2. Complete doc and test guideline.

Please feel free to contact me if you have any other concerns! Thanks!

[1] https://issues.apache.org/jira/browse/SPARK-36057

Fwd: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
I found some of the notes on Volcano and my tests back in Feb 2022. I did
my volcano tests on Spark 3.1.1. The results were not very great then.
Hence I asked in thread from @santosh, if any updated comparisons are
available. I will try the test with Spark 3.4.1 at some point. Maybe some
users have done some tests on Volcano with newer versions of Spark that
they care to share?


Thanks



Forwarded Conversation
Subject: Recap on current status of "SPIP: Support Customized Kubernetes
Schedulers",
------------------------


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Thu, 24 Feb 2022 at 09:16
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>


Hi,

what do expect the performance gain to be by using volcano versus standard
scheduler.

Just to be sure there are two aspects here.


   1. Procuring the Kubernetes cluster
   2. Running the job through spark-submit


Item 1 is left untouched and we should see improvements in item 2 with
Volcano

Thanks



   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.





----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Thu, 24 Feb 2022 at 23:35
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>



I did some preliminary tests without volcana and with volcano addition to
spark-submit.


*setup*


The K8s cluster used was a Google Kubernetes standard cluster with three
nodes with autoscale up to 6 nodes. It runs *spark 3.1.1* with spark-py
dockers also using *spark 3.1.1 with Java 8*.  In every run, it creates a
million rows of random data and inserts them from Spark DF into Google
BigQuery database. The choice of Spark 3.1.1 and Java 8 was for
compatibility for Spark API and the BigQuery.


To keep the systematics the same I used the same cluster with the only
difference being the spark-submit additional lines as below for volcano



         NEXEC=2

         MEMORY="8192m"

         VCORES=3


        FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”

        gcloud config set compute/zone $ZONE

        export PROJECT=$(gcloud info --format='value(config.project)')

        gcloud container clusters get-credentials ${CLUSTER_NAME} --zone
$ZONE

        export KUBERNETES_MASTER_IP=$(gcloud container clusters list
--filter name=${CLUSTER_NAME} --format='value(MASTER_IP)')

        spark-submit --verbose \

           --properties-file ${property_file} \

           --master k8s://https://$KUBERNETES_MASTER_IP:443 \

           --deploy-mode cluster \

           --name sparkBQ \

   *         --conf spark.kubernetes.scheduler=volcano \*

*           --conf spark.kubernetes.driver.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.executor.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.job.queue=queue1 \*

           --py-files $CODE_DIRECTORY_CLOUD/spark_on_gke.zip \

           --conf spark.kubernetes.namespace=$NAMESPACE \

           --conf spark.executor.instances=$NEXEC \

           --conf spark.driver.cores=$VCORES \

           --conf spark.executor.cores=$VCORES \

           --conf spark.driver.memory=$MEMORY \

           --conf spark.executor.memory=$MEMORY \

           --conf spark.network.timeout=300 \

           --conf spark.kubernetes.allocation.batch.size=3 \

           --conf spark.kubernetes.allocation.batch.delay=1 \

           --conf spark.dynamicAllocation.enabled=true \

           --conf spark.dynamicAllocation.shuffleTracking.enabled=true \

           --conf spark.kubernetes.driver.container.image=${IMAGEDRIVER} \

           --conf spark.kubernetes.executor.container.image=${IMAGEDRIVER} \

           --conf
spark.kubernetes.authenticate.driver.serviceAccountName=spark-bq \

             --conf
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" \

           --conf
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true"
\

           --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/
kubernetes.io/serviceaccount/ca.crt  \

           --conf
spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/
kubernetes.io/serviceaccount/token  \

           $CODE_DIRECTORY_CLOUD/${APPLICATION}



In contrast the standard spark-submit does not have those 4 volcano
specific lines (in bald). This i the output from *spark-submit --verbose*


Spark properties used, including those specified through

 --conf and those from the properties file
/home/hduser/dba/bin/python/spark_on_gke/deployment/src/scripts/properties:

  (spark.kubernetes.executor.secrets.spark-sa,*********(redacted))

  (spark.dynamicAllocation.shuffleTracking.enabled,true)

  (spark.kubernetes.allocation.batch.delay,1)


(spark.kubernetes.driverEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))


*(spark.kubernetes.executor.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.driver.memory,8192m)

  (spark.network.timeout,300)

  (spark.executor.memory,8192m)

  (spark.executor.instances,2)

  (spark.hadoop.fs.gs.project.id,xxx)

  (spark.kubernetes.allocation.batch.size,3)


(spark.hadoop.google.cloud.auth.service.account.json.keyfile,*********(redacted))

  *(spark.kubernetes.scheduler,volcano)*

  (spark.kubernetes.namespace,spark)

  (spark.kubernetes.authenticate.driver.serviceAccountName,spark-bq)

  (spark.kubernetes.executor.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)

  (spark.driver.cores,3)

  (spark.kubernetes.driverEnv.GCS_PROJECT_ID,xxx)


(spark.executor.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  (spark.executorEnv.GCS_PROJECT_ID,xxx)

  (spark.hadoop.google.cloud.auth.service.account.enable,true)

  (spark.driver.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  *(spark.kubernetes.job.queue,queue1)*

  (spark.kubernetes.authenticate.caCertFile,*********(redacted))

  (spark.kubernetes.driver.secrets.spark-sa,*********(redacted))

  (spark.executorEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))

  (spark.kubernetes.authenticate.oauthTokenFile,*********(redacted))

  (spark.dynamicAllocation.enabled,true)

  (spark.kubernetes.driver.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)


*(spark.kubernetes.driver.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.executor.cores,3)

So I ran the spark-submit job 8 times foreach configuration in sequence;
namely the *standard setup followed by the volcano setup*. The timings were
measured from the python code at the start and the end excluding the
cluster creation times, simply:

       start_time = time.time()

    ..code

    end_time = time.time()

    time_elapsed = (end_time - start_time)

    print(f"""Elapsed time in seconds is {time_elapsed}""")

*Results*

 These are the results of each run and timings in seconds


[image: image.png]



At this first instance the volcano compares poorly (avg ~96 seconds
compared to ~ 92 seconds). So with the deviations, I need to investigate
the reasons for the larger fluctuations of time taken for the volcano runs.
Any comments are welcome.


HTH



----------
From: Yikun Jiang <yi...@gmail.com>
Date: Fri, 25 Feb 2022 at 02:51
To: dev <de...@spark.apache.org>
Cc: Dongjoon Hyun <do...@apache.org>, Holden Karau <ho...@pigscanfly.ca>,
William Wang <wa...@gmail.com>, <pi...@gmail.com>,
Hyukjin Kwon <gu...@gmail.com>, <mg...@apache.org>, Weiwei Yang <
wwei@apache.org>, Thomas Graves <tg...@gmail.com>


@dongjoon-hyun @yangwwei Thanks!

@Mich Thanks for testing it, I'm not very professional with GKE,

I'm also not quite sure if it is different in configurations, internal
network, scheduler implementations
itself VS upstream K8S. As far as I know, different K8S vendors also
maintain their own optimizations
in their downstream product.

But you can see some basic integration test results based on upstream K8S
on x86/arm64:
- x86: https://github.com/apache/spark/pull/35422#issuecomment-1035901775
- Arm64: https://github.com/apache/spark/pull/35422#issuecomment-1037039764

As can be seen from the results, for a single job, there is no big
difference between default scheduler
and volcano.

Also custom schedulers such as Volcano, Yunikorn are more for the overall
situation for multiple jobs
and the utilization of the entire K8S cluster.


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Fri, 25 Feb 2022 at 09:50
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>



Hi Yikun,

GKE <https://cloud.google.com/kubernetes-engine> is Google's Kubernetes
engine first in the market and pretty stable.

The cluster deployed is a 3 node GKE with 4 Vcores and 16GB of RAM each.
Autoscaling is on to take nodes from 3 to 6. So it is pretty robust. I did
15 sequences of tests with the following results

[image: image.png]



Now again the readings from the standard spark-submit are pretty stable
with the standard deviation of 3., compared to volcano with the standard
deviation of 13.6. What is the latency
for FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”,
could that be one reason?


Regards,


Mich


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Fri, 25 Feb 2022 at 11:38
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>


spreadsheet with the actual size

[image: 26243d69-ac3a-43f1-b2cc-903f9744237b.png]


Also the spec for GKE cluster build


gcloud beta container \

 --project "xxx" clusters create "spark-on-gke" \

 --zone "europe-west2-c" \

 --no-enable-basic-auth \

 --cluster-version "1.21.6-gke.1500" \

 --release-channel "regular" \

 --machine-type "e2-standard-4" \

 --image-type "COS_CONTAINERD" \

 --disk-type "pd-standard"

 --disk-size "100" \

 --metadata disable-legacy-endpoints=true \

 --scopes "https://www.googleapis.com/auth/devstorage.read_only", \

   "https://www.googleapis.com/auth/logging.write", \

   "https://www.googleapis.com/auth/monitoring",\

   "https://www.googleapis.com/auth/servicecontrol",\

   "https://www.googleapis.com/auth/service.management.readonly",\

   "https://www.googleapis.com/auth/trace.append" \

 --max-pods-per-node "110" \

 --num-nodes "3" \

 --logging=SYSTEM,WORKLOAD \

 --monitoring=SYSTEM \

 --enable-ip-alias \

 --network "projects/xxx/global/networks/default" \

 --subnetwork "projects/xxx/regions/europe-west2/subnetworks/default" \

 --no-enable-intra-node-visibility \

 --default-max-pods-per-node "110" \

 --enable-autoscaling \

 --min-nodes "3" \

 --max-nodes "6" \

 --no-enable-master-authorized-networks \

 --addons
HorizontalPodAutoscaling,HttpLoadBalancing,GcePersistentDiskCsiDriver \

 --enable-autoupgrade \

 --enable-autorepair \

 --max-surge-upgrade 1 \

 --max-unavailable-upgrade 0 \

 --enable-shielded-nodes \

 --node-locations "europe-west2-c"

Fwd: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
I found some of the notes on Volcano and my tests back in Feb 2022. I did
my volcano tests on Spark 3.1.1. The results were not very great then.
Hence I asked in thread from @santosh, if any updated comparisons are
available. I will try the test with Spark 3.4.1 at some point. Maybe some
users have done some tests on Volcano with newer versions of Spark that
they care to share?


Thanks



Forwarded Conversation
Subject: Recap on current status of "SPIP: Support Customized Kubernetes
Schedulers",
------------------------


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Thu, 24 Feb 2022 at 09:16
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>


Hi,

what do expect the performance gain to be by using volcano versus standard
scheduler.

Just to be sure there are two aspects here.


   1. Procuring the Kubernetes cluster
   2. Running the job through spark-submit


Item 1 is left untouched and we should see improvements in item 2 with
Volcano

Thanks



   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.





----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Thu, 24 Feb 2022 at 23:35
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>



I did some preliminary tests without volcana and with volcano addition to
spark-submit.


*setup*


The K8s cluster used was a Google Kubernetes standard cluster with three
nodes with autoscale up to 6 nodes. It runs *spark 3.1.1* with spark-py
dockers also using *spark 3.1.1 with Java 8*.  In every run, it creates a
million rows of random data and inserts them from Spark DF into Google
BigQuery database. The choice of Spark 3.1.1 and Java 8 was for
compatibility for Spark API and the BigQuery.


To keep the systematics the same I used the same cluster with the only
difference being the spark-submit additional lines as below for volcano



         NEXEC=2

         MEMORY="8192m"

         VCORES=3


        FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”

        gcloud config set compute/zone $ZONE

        export PROJECT=$(gcloud info --format='value(config.project)')

        gcloud container clusters get-credentials ${CLUSTER_NAME} --zone
$ZONE

        export KUBERNETES_MASTER_IP=$(gcloud container clusters list
--filter name=${CLUSTER_NAME} --format='value(MASTER_IP)')

        spark-submit --verbose \

           --properties-file ${property_file} \

           --master k8s://https://$KUBERNETES_MASTER_IP:443 \

           --deploy-mode cluster \

           --name sparkBQ \

   *         --conf spark.kubernetes.scheduler=volcano \*

*           --conf spark.kubernetes.driver.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.executor.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.job.queue=queue1 \*

           --py-files $CODE_DIRECTORY_CLOUD/spark_on_gke.zip \

           --conf spark.kubernetes.namespace=$NAMESPACE \

           --conf spark.executor.instances=$NEXEC \

           --conf spark.driver.cores=$VCORES \

           --conf spark.executor.cores=$VCORES \

           --conf spark.driver.memory=$MEMORY \

           --conf spark.executor.memory=$MEMORY \

           --conf spark.network.timeout=300 \

           --conf spark.kubernetes.allocation.batch.size=3 \

           --conf spark.kubernetes.allocation.batch.delay=1 \

           --conf spark.dynamicAllocation.enabled=true \

           --conf spark.dynamicAllocation.shuffleTracking.enabled=true \

           --conf spark.kubernetes.driver.container.image=${IMAGEDRIVER} \

           --conf spark.kubernetes.executor.container.image=${IMAGEDRIVER} \

           --conf
spark.kubernetes.authenticate.driver.serviceAccountName=spark-bq \

             --conf
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" \

           --conf
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true"
\

           --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/
kubernetes.io/serviceaccount/ca.crt  \

           --conf
spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/
kubernetes.io/serviceaccount/token  \

           $CODE_DIRECTORY_CLOUD/${APPLICATION}



In contrast the standard spark-submit does not have those 4 volcano
specific lines (in bald). This i the output from *spark-submit --verbose*


Spark properties used, including those specified through

 --conf and those from the properties file
/home/hduser/dba/bin/python/spark_on_gke/deployment/src/scripts/properties:

  (spark.kubernetes.executor.secrets.spark-sa,*********(redacted))

  (spark.dynamicAllocation.shuffleTracking.enabled,true)

  (spark.kubernetes.allocation.batch.delay,1)


(spark.kubernetes.driverEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))


*(spark.kubernetes.executor.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.driver.memory,8192m)

  (spark.network.timeout,300)

  (spark.executor.memory,8192m)

  (spark.executor.instances,2)

  (spark.hadoop.fs.gs.project.id,xxx)

  (spark.kubernetes.allocation.batch.size,3)


(spark.hadoop.google.cloud.auth.service.account.json.keyfile,*********(redacted))

  *(spark.kubernetes.scheduler,volcano)*

  (spark.kubernetes.namespace,spark)

  (spark.kubernetes.authenticate.driver.serviceAccountName,spark-bq)

  (spark.kubernetes.executor.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)

  (spark.driver.cores,3)

  (spark.kubernetes.driverEnv.GCS_PROJECT_ID,xxx)


(spark.executor.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  (spark.executorEnv.GCS_PROJECT_ID,xxx)

  (spark.hadoop.google.cloud.auth.service.account.enable,true)

  (spark.driver.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  *(spark.kubernetes.job.queue,queue1)*

  (spark.kubernetes.authenticate.caCertFile,*********(redacted))

  (spark.kubernetes.driver.secrets.spark-sa,*********(redacted))

  (spark.executorEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))

  (spark.kubernetes.authenticate.oauthTokenFile,*********(redacted))

  (spark.dynamicAllocation.enabled,true)

  (spark.kubernetes.driver.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)


*(spark.kubernetes.driver.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.executor.cores,3)

So I ran the spark-submit job 8 times foreach configuration in sequence;
namely the *standard setup followed by the volcano setup*. The timings were
measured from the python code at the start and the end excluding the
cluster creation times, simply:

       start_time = time.time()

    ..code

    end_time = time.time()

    time_elapsed = (end_time - start_time)

    print(f"""Elapsed time in seconds is {time_elapsed}""")

*Results*

 These are the results of each run and timings in seconds


[image: image.png]



At this first instance the volcano compares poorly (avg ~96 seconds
compared to ~ 92 seconds). So with the deviations, I need to investigate
the reasons for the larger fluctuations of time taken for the volcano runs.
Any comments are welcome.


HTH



----------
From: Yikun Jiang <yi...@gmail.com>
Date: Fri, 25 Feb 2022 at 02:51
To: dev <de...@spark.apache.org>
Cc: Dongjoon Hyun <do...@apache.org>, Holden Karau <ho...@pigscanfly.ca>,
William Wang <wa...@gmail.com>, <pi...@gmail.com>,
Hyukjin Kwon <gu...@gmail.com>, <mg...@apache.org>, Weiwei Yang <
wwei@apache.org>, Thomas Graves <tg...@gmail.com>


@dongjoon-hyun @yangwwei Thanks!

@Mich Thanks for testing it, I'm not very professional with GKE,

I'm also not quite sure if it is different in configurations, internal
network, scheduler implementations
itself VS upstream K8S. As far as I know, different K8S vendors also
maintain their own optimizations
in their downstream product.

But you can see some basic integration test results based on upstream K8S
on x86/arm64:
- x86: https://github.com/apache/spark/pull/35422#issuecomment-1035901775
- Arm64: https://github.com/apache/spark/pull/35422#issuecomment-1037039764

As can be seen from the results, for a single job, there is no big
difference between default scheduler
and volcano.

Also custom schedulers such as Volcano, Yunikorn are more for the overall
situation for multiple jobs
and the utilization of the entire K8S cluster.


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Fri, 25 Feb 2022 at 09:50
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>



Hi Yikun,

GKE <https://cloud.google.com/kubernetes-engine> is Google's Kubernetes
engine first in the market and pretty stable.

The cluster deployed is a 3 node GKE with 4 Vcores and 16GB of RAM each.
Autoscaling is on to take nodes from 3 to 6. So it is pretty robust. I did
15 sequences of tests with the following results

[image: image.png]



Now again the readings from the standard spark-submit are pretty stable
with the standard deviation of 3., compared to volcano with the standard
deviation of 13.6. What is the latency
for FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”,
could that be one reason?


Regards,


Mich


----------
From: Mich Talebzadeh <mi...@gmail.com>
Date: Fri, 25 Feb 2022 at 11:38
To: Yikun Jiang <yi...@gmail.com>
Cc: dev <de...@spark.apache.org>, Dongjoon Hyun <do...@apache.org>, Holden
Karau <ho...@pigscanfly.ca>, William Wang <wa...@gmail.com>,
Attila Zsolt Piros <pi...@gmail.com>, Hyukjin Kwon <
gurwls223@gmail.com>, <mg...@apache.org>, Weiwei Yang <ww...@apache.org>,
Thomas Graves <tg...@gmail.com>


spreadsheet with the actual size

[image: 26243d69-ac3a-43f1-b2cc-903f9744237b.png]


Also the spec for GKE cluster build


gcloud beta container \

 --project "xxx" clusters create "spark-on-gke" \

 --zone "europe-west2-c" \

 --no-enable-basic-auth \

 --cluster-version "1.21.6-gke.1500" \

 --release-channel "regular" \

 --machine-type "e2-standard-4" \

 --image-type "COS_CONTAINERD" \

 --disk-type "pd-standard"

 --disk-size "100" \

 --metadata disable-legacy-endpoints=true \

 --scopes "https://www.googleapis.com/auth/devstorage.read_only", \

   "https://www.googleapis.com/auth/logging.write", \

   "https://www.googleapis.com/auth/monitoring",\

   "https://www.googleapis.com/auth/servicecontrol",\

   "https://www.googleapis.com/auth/service.management.readonly",\

   "https://www.googleapis.com/auth/trace.append" \

 --max-pods-per-node "110" \

 --num-nodes "3" \

 --logging=SYSTEM,WORKLOAD \

 --monitoring=SYSTEM \

 --enable-ip-alias \

 --network "projects/xxx/global/networks/default" \

 --subnetwork "projects/xxx/regions/europe-west2/subnetworks/default" \

 --no-enable-intra-node-visibility \

 --default-max-pods-per-node "110" \

 --enable-autoscaling \

 --min-nodes "3" \

 --max-nodes "6" \

 --no-enable-master-authorized-networks \

 --addons
HorizontalPodAutoscaling,HttpLoadBalancing,GcePersistentDiskCsiDriver \

 --enable-autoupgrade \

 --enable-autorepair \

 --max-surge-upgrade 1 \

 --max-unavailable-upgrade 0 \

 --enable-shielded-nodes \

 --node-locations "europe-west2-c"

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
spreadsheet with the actual size

[image: 26243d69-ac3a-43f1-b2cc-903f9744237b.png]


Also the spec for GKE cluster build


gcloud beta container \

 --project "xxx" clusters create "spark-on-gke" \

 --zone "europe-west2-c" \

 --no-enable-basic-auth \

 --cluster-version "1.21.6-gke.1500" \

 --release-channel "regular" \

 --machine-type "e2-standard-4" \

 --image-type "COS_CONTAINERD" \

 --disk-type "pd-standard"

 --disk-size "100" \

 --metadata disable-legacy-endpoints=true \

 --scopes "https://www.googleapis.com/auth/devstorage.read_only", \

   "https://www.googleapis.com/auth/logging.write", \

   "https://www.googleapis.com/auth/monitoring",\

   "https://www.googleapis.com/auth/servicecontrol",\

   "https://www.googleapis.com/auth/service.management.readonly",\

   "https://www.googleapis.com/auth/trace.append" \

 --max-pods-per-node "110" \

 --num-nodes "3" \

 --logging=SYSTEM,WORKLOAD \

 --monitoring=SYSTEM \

 --enable-ip-alias \

 --network "projects/xxx/global/networks/default" \

 --subnetwork "projects/xxx/regions/europe-west2/subnetworks/default" \

 --no-enable-intra-node-visibility \

 --default-max-pods-per-node "110" \

 --enable-autoscaling \

 --min-nodes "3" \

 --max-nodes "6" \

 --no-enable-master-authorized-networks \

 --addons
HorizontalPodAutoscaling,HttpLoadBalancing,GcePersistentDiskCsiDriver \

 --enable-autoupgrade \

 --enable-autorepair \

 --max-surge-upgrade 1 \

 --max-unavailable-upgrade 0 \

 --enable-shielded-nodes \

 --node-locations "europe-west2-c"



   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Fri, 25 Feb 2022 at 09:50, Mich Talebzadeh <mi...@gmail.com>
wrote:

>
> Hi Yikun,
>
> GKE <https://cloud.google.com/kubernetes-engine> is Google's Kubernetes
> engine first in the market and pretty stable.
>
> The cluster deployed is a 3 node GKE with 4 Vcores and 16GB of RAM each.
> Autoscaling is on to take nodes from 3 to 6. So it is pretty robust. I did
> 15 sequences of tests with the following results
>
> [image: image.png]
>
>
>
> Now again the readings from the standard spark-submit are pretty stable
> with the standard deviation of 3., compared to volcano with the standard
> deviation of 13.6. What is the latency
> for FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”,
> could that be one reason?
>
>
> Regards,
>
>
> Mich
>
>
>    view my Linkedin profile
> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>
>
>  https://en.everybodywiki.com/Mich_Talebzadeh
>
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
>
> On Fri, 25 Feb 2022 at 02:51, Yikun Jiang <yi...@gmail.com> wrote:
>
>> @dongjoon-hyun @yangwwei Thanks!
>>
>> @Mich Thanks for testing it, I'm not very professional with GKE,
>>
>> I'm also not quite sure if it is different in configurations, internal
>> network, scheduler implementations
>> itself VS upstream K8S. As far as I know, different K8S vendors also
>> maintain their own optimizations
>> in their downstream product.
>>
>> But you can see some basic integration test results based on upstream K8S
>> on x86/arm64:
>> - x86: https://github.com/apache/spark/pull/35422#issuecomment-1035901775
>> - Arm64:
>> https://github.com/apache/spark/pull/35422#issuecomment-1037039764
>>
>> As can be seen from the results, for a single job, there is no big
>> difference between default scheduler
>> and volcano.
>>
>> Also custom schedulers such as Volcano, Yunikorn are more for the overall
>> situation for multiple jobs
>> and the utilization of the entire K8S cluster.
>>
>

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
Hi Yikun,

GKE <https://cloud.google.com/kubernetes-engine> is Google's Kubernetes
engine first in the market and pretty stable.

The cluster deployed is a 3 node GKE with 4 Vcores and 16GB of RAM each.
Autoscaling is on to take nodes from 3 to 6. So it is pretty robust. I did
15 sequences of tests with the following results

[image: image.png]



Now again the readings from the standard spark-submit are pretty stable
with the standard deviation of 3., compared to volcano with the standard
deviation of 13.6. What is the latency
for FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”,
could that be one reason?


Regards,


Mich


   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Fri, 25 Feb 2022 at 02:51, Yikun Jiang <yi...@gmail.com> wrote:

> @dongjoon-hyun @yangwwei Thanks!
>
> @Mich Thanks for testing it, I'm not very professional with GKE,
>
> I'm also not quite sure if it is different in configurations, internal
> network, scheduler implementations
> itself VS upstream K8S. As far as I know, different K8S vendors also
> maintain their own optimizations
> in their downstream product.
>
> But you can see some basic integration test results based on upstream K8S
> on x86/arm64:
> - x86: https://github.com/apache/spark/pull/35422#issuecomment-1035901775
> - Arm64:
> https://github.com/apache/spark/pull/35422#issuecomment-1037039764
>
> As can be seen from the results, for a single job, there is no big
> difference between default scheduler
> and volcano.
>
> Also custom schedulers such as Volcano, Yunikorn are more for the overall
> situation for multiple jobs
> and the utilization of the entire K8S cluster.
>

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Yikun Jiang <yi...@gmail.com>.
@dongjoon-hyun @yangwwei Thanks!

@Mich Thanks for testing it, I'm not very professional with GKE,

I'm also not quite sure if it is different in configurations, internal
network, scheduler implementations
itself VS upstream K8S. As far as I know, different K8S vendors also
maintain their own optimizations
in their downstream product.

But you can see some basic integration test results based on upstream K8S
on x86/arm64:
- x86: https://github.com/apache/spark/pull/35422#issuecomment-1035901775
- Arm64: https://github.com/apache/spark/pull/35422#issuecomment-1037039764

As can be seen from the results, for a single job, there is no big
difference between default scheduler
and volcano.

Also custom schedulers such as Volcano, Yunikorn are more for the overall
situation for multiple jobs
and the utilization of the entire K8S cluster.

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
I did some preliminary tests without volcana and with volcano addition to
spark-submit.


*setup*


The K8s cluster used was a Google Kubernetes standard cluster with three
nodes with autoscale up to 6 nodes. It runs *spark 3.1.1* with spark-py
dockers also using *spark 3.1.1 with Java 8*.  In every run, it creates a
million rows of random data and inserts them from Spark DF into Google
BigQuery database. The choice of Spark 3.1.1 and Java 8 was for
compatibility for Spark API and the BigQuery.


To keep the systematics the same I used the same cluster with the only
difference being the spark-submit additional lines as below for volcano



         NEXEC=2

         MEMORY="8192m"

         VCORES=3


        FEATURES=”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”

        gcloud config set compute/zone $ZONE

        export PROJECT=$(gcloud info --format='value(config.project)')

        gcloud container clusters get-credentials ${CLUSTER_NAME} --zone
$ZONE

        export KUBERNETES_MASTER_IP=$(gcloud container clusters list
--filter name=${CLUSTER_NAME} --format='value(MASTER_IP)')

        spark-submit --verbose \

           --properties-file ${property_file} \

           --master k8s://https://$KUBERNETES_MASTER_IP:443 \

           --deploy-mode cluster \

           --name sparkBQ \

   *         --conf spark.kubernetes.scheduler=volcano \*

*           --conf spark.kubernetes.driver.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.executor.pod.featureSteps=$FEATURES \*

*           --conf spark.kubernetes.job.queue=queue1 \*

           --py-files $CODE_DIRECTORY_CLOUD/spark_on_gke.zip \

           --conf spark.kubernetes.namespace=$NAMESPACE \

           --conf spark.executor.instances=$NEXEC \

           --conf spark.driver.cores=$VCORES \

           --conf spark.executor.cores=$VCORES \

           --conf spark.driver.memory=$MEMORY \

           --conf spark.executor.memory=$MEMORY \

           --conf spark.network.timeout=300 \

           --conf spark.kubernetes.allocation.batch.size=3 \

           --conf spark.kubernetes.allocation.batch.delay=1 \

           --conf spark.dynamicAllocation.enabled=true \

           --conf spark.dynamicAllocation.shuffleTracking.enabled=true \

           --conf spark.kubernetes.driver.container.image=${IMAGEDRIVER} \

           --conf spark.kubernetes.executor.container.image=${IMAGEDRIVER} \

           --conf
spark.kubernetes.authenticate.driver.serviceAccountName=spark-bq \

             --conf
spark.driver.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true" \

           --conf
spark.executor.extraJavaOptions="-Dio.netty.tryReflectionSetAccessible=true"
\

           --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/
kubernetes.io/serviceaccount/ca.crt  \

           --conf
spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/
kubernetes.io/serviceaccount/token  \

           $CODE_DIRECTORY_CLOUD/${APPLICATION}



In contrast the standard spark-submit does not have those 4 volcano
specific lines (in bald). This i the output from *spark-submit --verbose*


Spark properties used, including those specified through

 --conf and those from the properties file
/home/hduser/dba/bin/python/spark_on_gke/deployment/src/scripts/properties:

  (spark.kubernetes.executor.secrets.spark-sa,*********(redacted))

  (spark.dynamicAllocation.shuffleTracking.enabled,true)

  (spark.kubernetes.allocation.batch.delay,1)


(spark.kubernetes.driverEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))


*(spark.kubernetes.executor.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.driver.memory,8192m)

  (spark.network.timeout,300)

  (spark.executor.memory,8192m)

  (spark.executor.instances,2)

  (spark.hadoop.fs.gs.project.id,xxx)

  (spark.kubernetes.allocation.batch.size,3)


(spark.hadoop.google.cloud.auth.service.account.json.keyfile,*********(redacted))

  *(spark.kubernetes.scheduler,volcano)*

  (spark.kubernetes.namespace,spark)

  (spark.kubernetes.authenticate.driver.serviceAccountName,spark-bq)

  (spark.kubernetes.executor.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)

  (spark.driver.cores,3)

  (spark.kubernetes.driverEnv.GCS_PROJECT_ID,xxx)


(spark.executor.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  (spark.executorEnv.GCS_PROJECT_ID,xxx)

  (spark.hadoop.google.cloud.auth.service.account.enable,true)

  (spark.driver.extraJavaOptions,-Dio.netty.tryReflectionSetAccessible=true)

  *(spark.kubernetes.job.queue,queue1)*

  (spark.kubernetes.authenticate.caCertFile,*********(redacted))

  (spark.kubernetes.driver.secrets.spark-sa,*********(redacted))

  (spark.executorEnv.GOOGLE_APPLICATION_CREDENTIALS,*********(redacted))

  (spark.kubernetes.authenticate.oauthTokenFile,*********(redacted))

  (spark.dynamicAllocation.enabled,true)

  (spark.kubernetes.driver.container.image,
eu.gcr.io/xxx/spark-py:3.1.1-scala_2.12-8-jre-slim-buster-java8PlusPackages)


*(spark.kubernetes.driver.pod.featureSteps,”org.apache.spark.deploy.k8s.features.VolcanoFeatureStep”)*

  (spark.executor.cores,3)

So I ran the spark-submit job 8 times foreach configuration in sequence;
namely the *standard setup followed by the volcano setup*. The timings were
measured from the python code at the start and the end excluding the
cluster creation times, simply:

       start_time = time.time()

    ..code

    end_time = time.time()

    time_elapsed = (end_time - start_time)

    print(f"""Elapsed time in seconds is {time_elapsed}""")

*Results*

 These are the results of each run and timings in seconds


[image: image.png]



At this first instance the volcano compares poorly (avg ~96 seconds
compared to ~ 92 seconds). So with the deviations, I need to investigate
the reasons for the larger fluctuations of time taken for the volcano runs.
Any comments are welcome.


HTH



   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Thu, 24 Feb 2022 at 09:16, Mich Talebzadeh <mi...@gmail.com>
wrote:

> Hi,
>
> what do expect the performance gain to be by using volcano versus standard
> scheduler.
>
> Just to be sure there are two aspects here.
>
>
>    1. Procuring the Kubernetes cluster
>    2. Running the job through spark-submit
>
>
> Item 1 is left untouched and we should see improvements in item 2 with
> Volcano
>
> Thanks
>
>
>
>    view my Linkedin profile
> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>
>
>  https://en.everybodywiki.com/Mich_Talebzadeh
>
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
>
> On Thu, 24 Feb 2022 at 03:35, Yikun Jiang <yi...@gmail.com> wrote:
>
>> First, much thanks for all your help (Spark/Volcano/Yunikorn community)
>> to make this SPIP happen!
>>
>> Especially,@dongjoon-hyun @holdenk @william-wang @attilapiros
>> @HyukjinKwon @martin-g @yangwwei @tgravescs
>>
>> The SPIP is near the end of the stage. It can be said that it is beta
>> available at the basic level.
>>
>> I also draft a simple slide to show how to use and help you understand
>> what we have done:
>>
>> https://docs.google.com/presentation/d/1XDsTWPcsBe4PQ-1MlBwd9pRl8mySdziE_dJE6iATNw8
>>
>> Below are also some recap to help you understand current implementation
>> and next step on SPIP:
>>
>> *# Existing work*
>> *## Basic part:*
>> - SPARK-36059 <https://issues.apache.org/jira/browse/SPARK-36059> *New
>> configuration:* ability to specify "schedulerName" in
>> driver/executor for Spark on K8S
>> - SPARK-37331 <https://issues.apache.org/jira/browse/SPARK-37331> *New
>> workflow:*ability to create pre-populated resources before driver pod
>>  for Spark on K8S
>> - SPARK-37145 <https://issues.apache.org/jira/browse/SPARK-37145> *New
>> developer API:* support user feature step with configuration for Spark
>> on K8S
>> - *(reviewing)* *New Job Configurations* for Spark on K8S:
>>   - SPARK-38188 <https://issues.apache.org/jira/browse/SPARK-38188>:
>> spark.kubernetes.job.queue
>>   - SPARK-38187 <https://issues.apache.org/jira/browse/SPARK-38187>:
>> spark.kubernetes.job.[minCPU|minMemory]
>>   - SPARK-38189 <https://issues.apache.org/jira/browse/SPARK-38189>:
>> spark.kubernetes.job.priorityClassName
>>
>> *## Volcano Part:*
>> - SPARK-37258 <https://issues.apache.org/jira/browse/SPARK-37258> *New
>> volcano extension* in kubernetes-client fabric8io/kubernetes-client#3579
>> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
>> profile: *-Pvolcano
>> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
>> Feature Step:* VolcanoFeatureStep
>> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
>> integration test:*
>>  *- Passed on x86 and Arm64 (Linux on Huawei Kunpeng 920 and MacOS on
>> Apple Silicon M1).*
>>  - Test basic volcano workflow
>>  - Test all existing tests based on the volcano.
>>
>> *## Yunikorn Part:*
>> @yangwwei  will also make the efforts for Yunikorn module feature step
>> since this week.
>> I will help to complete the yunikorn integration based on previous
>> experience.
>>
>> *# Next Plan*
>> There are also 3 main tasks to be completed before v3.3 code freeze:
>> 1. (reviewing) SPARK-38188
>> <https://issues.apache.org/jira/browse/SPARK-38188>: Support queue
>> scheduling configuration
>> https://github.com/apache/spark/pull/35553
>> 2. (reviewing) SPARK-38187
>> <https://issues.apache.org/jira/browse/SPARK-38187>: Support resource
>> reservation (minCPU/minMemory configuration)
>> https://github.com/apache/spark/pull/35640
>> 3. (reviewing) SPARK-38187
>> <https://issues.apache.org/jira/browse/SPARK-38187>: Support priority
>> scheduling (priorityClass configuration):
>> https://issues.apache.org/jira/browse/SPARK-38189
>> https://github.com/apache/spark/pull/35639
>> 4. (WIP) SPARK-37809 <https://issues.apache.org/jira/browse/SPARK-37809>:
>> Yunikorn integration
>>
>> Also several misc work is gonna be completed before 3.3:
>> 1. Integrated volcano deploy into integration test (x86 and arm)
>> - Add it to spark kubernetes integration test once cross compile support:
>> https://github.com/volcano-sh/volcano/pull/1571
>> 2. Complete doc and test guideline.
>>
>> Please feel free to contact me if you have any other concerns! Thanks!
>>
>> [1] https://issues.apache.org/jira/browse/SPARK-36057
>>
>

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Mich Talebzadeh <mi...@gmail.com>.
Hi,

what do expect the performance gain to be by using volcano versus standard
scheduler.

Just to be sure there are two aspects here.


   1. Procuring the Kubernetes cluster
   2. Running the job through spark-submit


Item 1 is left untouched and we should see improvements in item 2 with
Volcano

Thanks



   view my Linkedin profile
<https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>


 https://en.everybodywiki.com/Mich_Talebzadeh



*Disclaimer:* Use it at your own risk. Any and all responsibility for any
loss, damage or destruction of data or any other property which may arise
from relying on this email's technical content is explicitly disclaimed.
The author will in no case be liable for any monetary damages arising from
such loss, damage or destruction.




On Thu, 24 Feb 2022 at 03:35, Yikun Jiang <yi...@gmail.com> wrote:

> First, much thanks for all your help (Spark/Volcano/Yunikorn community) to
> make this SPIP happen!
>
> Especially,@dongjoon-hyun @holdenk @william-wang @attilapiros @HyukjinKwon
> @martin-g @yangwwei @tgravescs
>
> The SPIP is near the end of the stage. It can be said that it is beta
> available at the basic level.
>
> I also draft a simple slide to show how to use and help you understand
> what we have done:
>
> https://docs.google.com/presentation/d/1XDsTWPcsBe4PQ-1MlBwd9pRl8mySdziE_dJE6iATNw8
>
> Below are also some recap to help you understand current implementation
> and next step on SPIP:
>
> *# Existing work*
> *## Basic part:*
> - SPARK-36059 <https://issues.apache.org/jira/browse/SPARK-36059> *New
> configuration:* ability to specify "schedulerName" in driver/executor for
> Spark on K8S
> - SPARK-37331 <https://issues.apache.org/jira/browse/SPARK-37331> *New
> workflow:*ability to create pre-populated resources before driver pod
>  for Spark on K8S
> - SPARK-37145 <https://issues.apache.org/jira/browse/SPARK-37145> *New
> developer API:* support user feature step with configuration for Spark on
> K8S
> - *(reviewing)* *New Job Configurations* for Spark on K8S:
>   - SPARK-38188 <https://issues.apache.org/jira/browse/SPARK-38188>:
> spark.kubernetes.job.queue
>   - SPARK-38187 <https://issues.apache.org/jira/browse/SPARK-38187>:
> spark.kubernetes.job.[minCPU|minMemory]
>   - SPARK-38189 <https://issues.apache.org/jira/browse/SPARK-38189>:
> spark.kubernetes.job.priorityClassName
>
> *## Volcano Part:*
> - SPARK-37258 <https://issues.apache.org/jira/browse/SPARK-37258> *New
> volcano extension* in kubernetes-client fabric8io/kubernetes-client#3579
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> profile: *-Pvolcano
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> Feature Step:* VolcanoFeatureStep
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> integration test:*
>  *- Passed on x86 and Arm64 (Linux on Huawei Kunpeng 920 and MacOS on
> Apple Silicon M1).*
>  - Test basic volcano workflow
>  - Test all existing tests based on the volcano.
>
> *## Yunikorn Part:*
> @yangwwei  will also make the efforts for Yunikorn module feature step
> since this week.
> I will help to complete the yunikorn integration based on previous
> experience.
>
> *# Next Plan*
> There are also 3 main tasks to be completed before v3.3 code freeze:
> 1. (reviewing) SPARK-38188
> <https://issues.apache.org/jira/browse/SPARK-38188>: Support queue
> scheduling configuration
> https://github.com/apache/spark/pull/35553
> 2. (reviewing) SPARK-38187
> <https://issues.apache.org/jira/browse/SPARK-38187>: Support resource
> reservation (minCPU/minMemory configuration)
> https://github.com/apache/spark/pull/35640
> 3. (reviewing) SPARK-38187
> <https://issues.apache.org/jira/browse/SPARK-38187>: Support priority
> scheduling (priorityClass configuration):
> https://issues.apache.org/jira/browse/SPARK-38189
> https://github.com/apache/spark/pull/35639
> 4. (WIP) SPARK-37809 <https://issues.apache.org/jira/browse/SPARK-37809>:
> Yunikorn integration
>
> Also several misc work is gonna be completed before 3.3:
> 1. Integrated volcano deploy into integration test (x86 and arm)
> - Add it to spark kubernetes integration test once cross compile support:
> https://github.com/volcano-sh/volcano/pull/1571
> 2. Complete doc and test guideline.
>
> Please feel free to contact me if you have any other concerns! Thanks!
>
> [1] https://issues.apache.org/jira/browse/SPARK-36057
>

Re: Recap on current status of "SPIP: Support Customized Kubernetes Schedulers"

Posted by Weiwei Yang <ww...@apache.org>.
Thank you, Yikun.
I am working on SPARK-37809
<https://issues.apache.org/jira/browse/SPARK-37809> and SPARK-38310
<https://issues.apache.org/jira/browse/SPARK-38310>. They are the major
stuff for the yunikorn part.
Keep in mind we also need to add the documents.
Thanks for building up the common things, great work.

On Wed, Feb 23, 2022 at 7:35 PM Yikun Jiang <yi...@gmail.com> wrote:

> First, much thanks for all your help (Spark/Volcano/Yunikorn community) to
> make this SPIP happen!
>
> Especially,@dongjoon-hyun @holdenk @william-wang @attilapiros @HyukjinKwon
> @martin-g @yangwwei @tgravescs
>
> The SPIP is near the end of the stage. It can be said that it is beta
> available at the basic level.
>
> I also draft a simple slide to show how to use and help you understand
> what we have done:
>
> https://docs.google.com/presentation/d/1XDsTWPcsBe4PQ-1MlBwd9pRl8mySdziE_dJE6iATNw8
>
> Below are also some recap to help you understand current implementation
> and next step on SPIP:
>
> *# Existing work*
> *## Basic part:*
> - SPARK-36059 <https://issues.apache.org/jira/browse/SPARK-36059> *New
> configuration:* ability to specify "schedulerName" in driver/executor for
> Spark on K8S
> - SPARK-37331 <https://issues.apache.org/jira/browse/SPARK-37331> *New
> workflow:*ability to create pre-populated resources before driver pod
>  for Spark on K8S
> - SPARK-37145 <https://issues.apache.org/jira/browse/SPARK-37145> *New
> developer API:* support user feature step with configuration for Spark on
> K8S
> - *(reviewing)* *New Job Configurations* for Spark on K8S:
>   - SPARK-38188 <https://issues.apache.org/jira/browse/SPARK-38188>:
> spark.kubernetes.job.queue
>   - SPARK-38187 <https://issues.apache.org/jira/browse/SPARK-38187>:
> spark.kubernetes.job.[minCPU|minMemory]
>   - SPARK-38189 <https://issues.apache.org/jira/browse/SPARK-38189>:
> spark.kubernetes.job.priorityClassName
>
> *## Volcano Part:*
> - SPARK-37258 <https://issues.apache.org/jira/browse/SPARK-37258> *New
> volcano extension* in kubernetes-client fabric8io/kubernetes-client#3579
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> profile: *-Pvolcano
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> Feature Step:* VolcanoFeatureStep
> - SPARK-36061 <https://issues.apache.org/jira/browse/SPARK-36061> *New
> integration test:*
>  *- Passed on x86 and Arm64 (Linux on Huawei Kunpeng 920 and MacOS on
> Apple Silicon M1).*
>  - Test basic volcano workflow
>  - Test all existing tests based on the volcano.
>
> *## Yunikorn Part:*
> @yangwwei  will also make the efforts for Yunikorn module feature step
> since this week.
> I will help to complete the yunikorn integration based on previous
> experience.
>
> *# Next Plan*
> There are also 3 main tasks to be completed before v3.3 code freeze:
> 1. (reviewing) SPARK-38188
> <https://issues.apache.org/jira/browse/SPARK-38188>: Support queue
> scheduling configuration
> https://github.com/apache/spark/pull/35553
> 2. (reviewing) SPARK-38187
> <https://issues.apache.org/jira/browse/SPARK-38187>: Support resource
> reservation (minCPU/minMemory configuration)
> https://github.com/apache/spark/pull/35640
> 3. (reviewing) SPARK-38187
> <https://issues.apache.org/jira/browse/SPARK-38187>: Support priority
> scheduling (priorityClass configuration):
> https://issues.apache.org/jira/browse/SPARK-38189
> https://github.com/apache/spark/pull/35639
> 4. (WIP) SPARK-37809 <https://issues.apache.org/jira/browse/SPARK-37809>:
> Yunikorn integration
>
> Also several misc work is gonna be completed before 3.3:
> 1. Integrated volcano deploy into integration test (x86 and arm)
> - Add it to spark kubernetes integration test once cross compile support:
> https://github.com/volcano-sh/volcano/pull/1571
> 2. Complete doc and test guideline.
>
> Please feel free to contact me if you have any other concerns! Thanks!
>
> [1] https://issues.apache.org/jira/browse/SPARK-36057
>