-
Notifications
You must be signed in to change notification settings - Fork 28.8k
[SPARK-23146][WIP] Support client mode for Kubernetes in Out-Cluster mode #20451
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
Changes from 12 commits
26a0126
e75c8ff
66a07eb
5e05410
3985bc6
d3b2ec4
d102ac2
136e957
a07c9df
3164fa6
11c1ea5
a248675
58ca9ab
4022277
e8b54a4
13fa7b2
ab992ca
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,40 @@ | ||
/* | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. done |
||
* 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. | ||
*/ | ||
package org.apache.spark.deploy.k8s | ||
|
||
private[spark] object OptionRequirements { | ||
|
||
def requireBothOrNeitherDefined( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. not used, and if it were, perhaps would be clearer as (opt1, opt2) match {
case (Some(val1), None) =>
...
case (None, Some(val2)) =>
...
case _ => // ok
} There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is part of unused code that could be fixed in other PR (e.g. #21462). Will double check after merging master, but normally, client-mode does not impact this. |
||
opt1: Option[_], | ||
opt2: Option[_], | ||
errMessageWhenFirstIsMissing: String, | ||
errMessageWhenSecondIsMissing: String): Unit = { | ||
requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) | ||
requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) | ||
} | ||
|
||
def requireSecondIfFirstIsDefined( | ||
opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { | ||
opt1.foreach { _ => | ||
require(opt2.isDefined, errMessageWhenSecondIsMissing) | ||
} | ||
} | ||
|
||
def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { | ||
opt1.foreach { _ => require(opt2.isEmpty, errMessage) } | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,7 +20,7 @@ import java.io.File | |
|
||
import com.google.common.base.Charsets | ||
import com.google.common.io.Files | ||
import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} | ||
import io.fabric8.kubernetes.client.{Config => Fabric8Client, ConfigBuilder, DefaultKubernetesClient, KubernetesClient} | ||
import io.fabric8.kubernetes.client.utils.HttpClientUtils | ||
import okhttp3.Dispatcher | ||
|
||
|
@@ -36,6 +36,21 @@ import org.apache.spark.util.ThreadUtils | |
private[spark] object SparkKubernetesClientFactory { | ||
|
||
def createKubernetesClient( | ||
master: String, | ||
namespace: Option[String], | ||
kubernetesAuthConfPrefix: String, | ||
sparkConf: SparkConf, | ||
maybeServiceAccountToken: Option[File], | ||
maybeServiceAccountCaCert: Option[File]): KubernetesClient = { | ||
new java.io.File(Fabric8Client.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH).exists() match { | ||
case true => createInClusterKubernetesClient(master, namespace, kubernetesAuthConfPrefix, | ||
sparkConf, maybeServiceAccountToken, maybeServiceAccountCaCert) | ||
case false => createOutClusterKubernetesClient(master, namespace, kubernetesAuthConfPrefix, | ||
sparkConf, maybeServiceAccountToken, maybeServiceAccountCaCert) | ||
} | ||
} | ||
|
||
private def createInClusterKubernetesClient( | ||
master: String, | ||
namespace: Option[String], | ||
kubernetesAuthConfPrefix: String, | ||
|
@@ -88,6 +103,56 @@ private[spark] object SparkKubernetesClientFactory { | |
new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) | ||
} | ||
|
||
def createOutClusterKubernetesClient( | ||
master: String, | ||
namespace: Option[String], | ||
kubernetesAuthConfPrefix: String, | ||
sparkConf: SparkConf, | ||
maybeServiceAccountToken: Option[File], | ||
maybeServiceAccountCaCert: Option[File]): KubernetesClient = { | ||
val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" | ||
val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" | ||
val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) | ||
.map(new File(_)) | ||
.orElse(maybeServiceAccountToken) | ||
val oauthTokenValue = sparkConf.getOption(oauthTokenConf) | ||
OptionRequirements.requireNandDefined( | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since it's only used once I'm not sure it warrents a separate file/method for checking Options. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I did not introduce |
||
oauthTokenFile, | ||
oauthTokenValue, | ||
s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + | ||
s" value $oauthTokenConf.") | ||
|
||
val caCertFile = sparkConf | ||
.getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") | ||
.orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) | ||
val clientKeyFile = sparkConf | ||
.getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") | ||
val clientCertFile = sparkConf | ||
.getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") | ||
val dispatcher = new Dispatcher( | ||
ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) | ||
val config = new ConfigBuilder() | ||
.withApiVersion("v1") | ||
.withMasterUrl(master) | ||
.withWebsocketPingInterval(0) | ||
.withOption(oauthTokenValue) { | ||
(token, configBuilder) => configBuilder.withOauthToken(token) | ||
}.withOption(caCertFile) { | ||
(file, configBuilder) => configBuilder.withCaCertFile(file) | ||
}.withOption(clientKeyFile) { | ||
(file, configBuilder) => configBuilder.withClientKeyFile(file) | ||
}.withOption(clientCertFile) { | ||
(file, configBuilder) => configBuilder.withClientCertFile(file) | ||
}.withOption(namespace) { | ||
(ns, configBuilder) => configBuilder.withNamespace(ns) | ||
}.build() | ||
val baseHttpClient = HttpClientUtils.createHttpClient(config) | ||
val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() | ||
.dispatcher(dispatcher) | ||
.build() | ||
new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) | ||
} | ||
|
||
private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) | ||
extends AnyVal { | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,41 +20,65 @@ import java.io.File | |
import java.util.concurrent.TimeUnit | ||
|
||
import com.google.common.cache.CacheBuilder | ||
import io.fabric8.kubernetes.client.Config | ||
|
||
import org.apache.spark.{SparkContext, SparkException} | ||
import org.apache.spark.deploy.k8s.{KubernetesUtils, SparkKubernetesClientFactory} | ||
import io.fabric8.kubernetes.client.{Config, KubernetesClient} | ||
import org.apache.spark.{SparkConf, SparkContext, SparkException} | ||
import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory | ||
import org.apache.spark.deploy.k8s.Config._ | ||
import org.apache.spark.deploy.k8s.Constants._ | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} | ||
import org.apache.spark.util.{SystemClock, ThreadUtils} | ||
|
||
private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { | ||
trait ManagerSpecificHandlers { | ||
def createKubernetesClient(sparkConf: SparkConf): KubernetesClient | ||
} | ||
|
||
override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") | ||
private[spark] class KubernetesClusterManager extends ExternalClusterManager | ||
with ManagerSpecificHandlers with Logging { | ||
|
||
override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { | ||
if (masterURL.startsWith("k8s") && | ||
sc.deployMode == "client" && | ||
!sc.conf.get(KUBERNETES_DRIVER_SUBMIT_CHECK).getOrElse(false)) { | ||
throw new SparkException("Client mode is currently not supported for Kubernetes.") | ||
class InClusterHandlers extends ManagerSpecificHandlers { | ||
override def createKubernetesClient(sparkConf: SparkConf): KubernetesClient = | ||
SparkKubernetesClientFactory.createKubernetesClient( | ||
KUBERNETES_MASTER_INTERNAL_URL, | ||
Some(sparkConf.get(KUBERNETES_NAMESPACE)), | ||
KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, | ||
sparkConf, | ||
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), | ||
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) | ||
} | ||
|
||
class OutClusterHandlers extends ManagerSpecificHandlers { | ||
override def createKubernetesClient(sparkConf: SparkConf): KubernetesClient = | ||
SparkKubernetesClientFactory.createKubernetesClient( | ||
sparkConf.get("spark.master").replace("k8s://", ""), | ||
Some(sparkConf.get(KUBERNETES_NAMESPACE)), | ||
KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The name of this prefix There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the call to
I would favor the second and remove the first. For the config place, I remember that the fabric8 k8s client does also some inspection to see if it is in or out cluster, and loads the config form the default place (depending the case), with possiblity to specify other places for the cert, token... (this is what we give as property to the end-user). |
||
sparkConf, | ||
None, | ||
None) | ||
} | ||
|
||
val modeHandler: ManagerSpecificHandlers = { | ||
new java.io.File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH).exists() match { | ||
case true => new InClusterHandlers() | ||
case false => new OutClusterHandlers() | ||
} | ||
} | ||
|
||
override def createKubernetesClient(sparkConf: SparkConf): KubernetesClient = | ||
modeHandler.createKubernetesClient(sparkConf) | ||
|
||
override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") | ||
|
||
override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { | ||
new TaskSchedulerImpl(sc) | ||
} | ||
|
||
override def createSchedulerBackend( | ||
sc: SparkContext, | ||
masterURL: String, | ||
scheduler: TaskScheduler): SchedulerBackend = { | ||
val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( | ||
KUBERNETES_MASTER_INTERNAL_URL, | ||
Some(sc.conf.get(KUBERNETES_NAMESPACE)), | ||
KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX, | ||
sc.conf, | ||
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), | ||
Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) | ||
val kubernetesClient = createKubernetesClient(sc.getConf) | ||
|
||
val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( | ||
"kubernetes-executor-requests") | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX
was replaced byKUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX
. Please revert this addition.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
done