diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md
index 79e01a35e2c57..a5da80a68d32d 100644
--- a/docs/running-on-kubernetes.md
+++ b/docs/running-on-kubernetes.md
@@ -1356,15 +1356,6 @@ See the [configuration page](configuration.html) for information on Spark config
3.3.0 |
-
- spark.kubernetes.job.queue |
- (none) |
-
- The name of the queue to which the job is submitted. This info will be stored in configuration
- and passed to specific feature step.
- |
- 3.3.0 |
-
spark.kubernetes.configMap.maxSize |
1572864 |
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
index e66ecf4312bb2..ff17ef51fe630 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala
@@ -306,13 +306,6 @@ private[spark] object Config extends Logging {
.stringConf
.createOptional
- val KUBERNETES_JOB_QUEUE = ConfigBuilder("spark.kubernetes.job.queue")
- .doc("The name of the queue to which the job is submitted. This info " +
- "will be stored in configuration and passed to specific feature step.")
- .version("3.3.0")
- .stringConf
- .createOptional
-
val KUBERNETES_EXECUTOR_REQUEST_CORES =
ConfigBuilder("spark.kubernetes.executor.request.cores")
.doc("Specify the cpu request for each executor pod")
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala
index 5fd0fc69ea2df..393edd2871ea0 100644
--- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala
@@ -32,7 +32,6 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
private lazy val podGroupName = s"${kubernetesConf.appId}-podgroup"
private lazy val namespace = kubernetesConf.namespace
- private lazy val queue = kubernetesConf.get(KUBERNETES_JOB_QUEUE)
private var priorityClassName: Option[String] = None
override def init(config: KubernetesDriverConf): Unit = {
@@ -60,7 +59,6 @@ private[spark] class VolcanoFeatureStep extends KubernetesDriverCustomFeatureCon
var spec = pg.getSpec
if (spec == null) spec = new PodGroupSpec
- queue.foreach(spec.setQueue(_))
priorityClassName.foreach(spec.setPriorityClassName(_))
pg.setSpec(spec)
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala
index e7f1e316a6d67..9f6bedb17626d 100644
--- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala
@@ -41,16 +41,6 @@ class VolcanoFeatureStepSuite extends SparkFunSuite {
assert(podGroup.getMetadata.getName === s"${kubernetesConf.appId}-podgroup")
}
- test("SPARK-38818: Support `spark.kubernetes.job.queue`") {
- val sparkConf = new SparkConf()
- .set(KUBERNETES_JOB_QUEUE.key, "queue1")
- val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf)
- val step = new VolcanoFeatureStep()
- step.init(kubernetesConf)
- val podGroup = step.getAdditionalPreKubernetesResources().head.asInstanceOf[PodGroup]
- assert(podGroup.getSpec.getQueue === "queue1")
- }
-
test("SPARK-36061: Executor Pod with Volcano PodGroup") {
val sparkConf = new SparkConf()
val kubernetesConf = KubernetesTestConf.createExecutorConf(sparkConf)
diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue-driver-podgroup-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue-driver-podgroup-template.yml
new file mode 100644
index 0000000000000..591000a0d02d3
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue-driver-podgroup-template.yml
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+apiVersion: scheduling.volcano.sh/v1beta1
+kind: PodGroup
+spec:
+ queue: queue
diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue0-driver-podgroup-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue0-driver-podgroup-template.yml
new file mode 100644
index 0000000000000..faba21abe1ec2
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue0-driver-podgroup-template.yml
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+apiVersion: scheduling.volcano.sh/v1beta1
+kind: PodGroup
+spec:
+ queue: queue0
diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue1-driver-podgroup-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue1-driver-podgroup-template.yml
new file mode 100644
index 0000000000000..280656450ea06
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/volcano/queue1-driver-podgroup-template.yml
@@ -0,0 +1,20 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+apiVersion: scheduling.volcano.sh/v1beta1
+kind: PodGroup
+spec:
+ queue: queue1
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala
index 803a8d3f194d0..ce5f86345eb45 100644
--- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala
@@ -210,7 +210,12 @@ private[spark] trait VolcanoTestsSuite extends BeforeAndAfterEach { k8sSuite: Ku
.set(KUBERNETES_SCHEDULER_NAME.key, "volcano")
.set(KUBERNETES_DRIVER_POD_FEATURE_STEPS.key, VOLCANO_FEATURE_STEP)
.set(KUBERNETES_EXECUTOR_POD_FEATURE_STEPS.key, VOLCANO_FEATURE_STEP)
- queue.foreach(conf.set(KUBERNETES_JOB_QUEUE.key, _))
+ queue.foreach { q =>
+ conf.set(KUBERNETES_DRIVER_PODGROUP_TEMPLATE_FILE.key,
+ new File(
+ getClass.getResource(s"/volcano/$q-driver-podgroup-template.yml").getFile
+ ).getAbsolutePath)
+ }
groupLoc.foreach { locator =>
conf.set(s"${KUBERNETES_DRIVER_LABEL_PREFIX}spark-group-locator", locator)
conf.set(s"${KUBERNETES_EXECUTOR_LABEL_PREFIX}spark-group-locator", locator)