diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java index c034e6c04a5f2..e9e94c1855a53 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java @@ -17,64 +17,13 @@ package org.apache.spark.network.shuffle.kubernetes; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.shuffle.ExternalShuffleClient; -import org.apache.spark.network.shuffle.protocol.RegisterDriver; -import org.apache.spark.network.util.TransportConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import java.io.Closeable; import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * A client for talking to the external shuffle service in Kubernetes cluster mode. - * - * This is used by the each Spark executor to register with a corresponding external - * shuffle service on the cluster. The purpose is for cleaning up shuffle files - * reliably if the application exits unexpectedly. - */ -public class KubernetesExternalShuffleClient extends ExternalShuffleClient { - private static final Logger logger = LoggerFactory - .getLogger(KubernetesExternalShuffleClient.class); - - /** - * Creates an Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}. - * Please refer to docs on {@link ExternalShuffleClient} for more information. - */ - public KubernetesExternalShuffleClient( - TransportConf conf, - SecretKeyHolder secretKeyHolder, - boolean authEnabled, - long registrationTimeoutMs) { - super(conf, secretKeyHolder, authEnabled, registrationTimeoutMs); - } - - public void registerDriverWithShuffleService(String host, int port) - throws IOException, InterruptedException { - checkInit(); - ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer(); - TransportClient client = clientFactory.createClient(host, port); - client.sendRpc(registerDriver, new RegisterDriverCallback()); - } - private class RegisterDriverCallback implements RpcResponseCallback { - @Override - public void onSuccess(ByteBuffer response) { - logger.info("Successfully registered app " + appId + " with external shuffle service."); - } +public interface KubernetesExternalShuffleClient extends Closeable { - @Override - public void onFailure(Throwable e) { - logger.warn("Unable to register app " + appId + " with external shuffle service. " + - "Please manually remove shuffle data after driver exit. Error: " + e); - } - } + void init(String appId); - @Override - public void close() { - super.close(); - } + void registerDriverWithShuffleService(String host, int port) + throws IOException, InterruptedException; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java new file mode 100644 index 0000000000000..b33ffd0d97d4e --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java @@ -0,0 +1,78 @@ +/* + * 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.network.shuffle.kubernetes; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.shuffle.ExternalShuffleClient; +import org.apache.spark.network.shuffle.protocol.RegisterDriver; +import org.apache.spark.network.util.TransportConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * A client for talking to the external shuffle service in Kubernetes cluster mode. + * + * This is used by the each Spark executor to register with a corresponding external + * shuffle service on the cluster. The purpose is for cleaning up shuffle files + * reliably if the application exits unexpectedly. + */ +public class KubernetesExternalShuffleClientImpl + extends ExternalShuffleClient implements KubernetesExternalShuffleClient { + + private static final Logger logger = LoggerFactory + .getLogger(KubernetesExternalShuffleClientImpl.class); + + /** + * Creates a Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}. + * Please refer to docs on {@link ExternalShuffleClient} for more information. + */ + public KubernetesExternalShuffleClientImpl( + TransportConf conf, + SecretKeyHolder secretKeyHolder, + boolean saslEnabled, + long registrationTimeoutMs) { + super(conf, secretKeyHolder, saslEnabled, registrationTimeoutMs); + } + + @Override + public void registerDriverWithShuffleService(String host, int port) + throws IOException, InterruptedException { + checkInit(); + ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer(); + TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(registerDriver, new RegisterDriverCallback()); + } + + private class RegisterDriverCallback implements RpcResponseCallback { + @Override + public void onSuccess(ByteBuffer response) { + logger.info("Successfully registered app " + appId + " with external shuffle service."); + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Unable to register app " + appId + " with external shuffle service. " + + "Please manually remove shuffle data after driver exit. Error: " + e); + } + } +} diff --git a/conf/k8s-shuffle-service-rbac.yaml b/conf/k8s-shuffle-service-rbac.yaml new file mode 100644 index 0000000000000..7ba4fe8e2f15c --- /dev/null +++ b/conf/k8s-shuffle-service-rbac.yaml @@ -0,0 +1,80 @@ +# +# 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: v1 +kind: ServiceAccount +metadata: + name: spark-shuffle-service-service-account + namespace: default + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +--- +apiVersion: extensions/v1beta1 +kind: PodSecurityPolicy +metadata: + name: spark-shuffle-service-pod-security-policy + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +spec: + privileged: false + fsGroup: + rule: RunAsAny + runAsUser: + rule: RunAsAny + volumes: + - "hostPath" + - "secret" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: ClusterRole +metadata: + name: spark-shuffle-service-role + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +rules: +- apiGroups: + - "extensions" + resources: + - "podsecuritypolicies" + resourceNames: + - "spark-shuffle-service-pod-security-policy" + verbs: + - "use" +- apiGroups: + - "" # "" indicates the core API group + resources: + - "pods" + verbs: + - "get" + - "list" + - "watch" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: ClusterRoleBinding +metadata: + name: spark-shuffle-service-role-binding +subjects: +- kind: ServiceAccount + name: spark-shuffle-service-service-account + namespace: default +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: spark-shuffle-service-role diff --git a/conf/k8s-spark-rbac.yaml b/conf/k8s-spark-rbac.yaml new file mode 100644 index 0000000000000..40381d39f2b07 --- /dev/null +++ b/conf/k8s-spark-rbac.yaml @@ -0,0 +1,55 @@ +# +# 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: v1 +kind: ServiceAccount +metadata: + name: spark + namespace: default +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: Role +metadata: + namespace: default + name: spark-role +rules: +- apiGroups: + - "" # "" indicates the core API group + resources: + - "pods" + verbs: + - "*" +- apiGroups: + - "" # "" indicates the core API group + resources: + - "services" + verbs: + - "*" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: RoleBinding +metadata: + name: spark-role-binding + namespace: default +subjects: +- kind: ServiceAccount + name: spark + namespace: default +roleRef: + kind: Role + name: spark-role + apiGroup: rbac.authorization.k8s.io diff --git a/conf/kubernetes-resource-staging-server.yaml b/conf/kubernetes-resource-staging-server.yaml index 80d59b8091903..18b446574fdaa 100644 --- a/conf/kubernetes-resource-staging-server.yaml +++ b/conf/kubernetes-resource-staging-server.yaml @@ -15,7 +15,7 @@ # limitations under the License. # --- -apiVersion: extensions/v1beta1 +apiVersion: apps/v1beta1 kind: Deployment metadata: name: spark-resource-staging-server @@ -32,14 +32,14 @@ spec: name: spark-resource-staging-server-config containers: - name: spark-resource-staging-server - image: kubespark/spark-resource-staging-server:v2.2.0-kubernetes-0.3.0 + image: kubespark/spark-resource-staging-server:v2.2.0-kubernetes-0.4.0 resources: requests: cpu: 100m memory: 256Mi limits: cpu: 100m - memory: 256Mi + memory: 1Gi volumeMounts: - name: resource-staging-server-properties mountPath: '/etc/spark-resource-staging-server' diff --git a/conf/kubernetes-shuffle-service.yaml b/conf/kubernetes-shuffle-service.yaml index 8ab0b362ea32e..b22adb265fd18 100644 --- a/conf/kubernetes-shuffle-service.yaml +++ b/conf/kubernetes-shuffle-service.yaml @@ -32,16 +32,16 @@ spec: volumes: - name: temp-volume hostPath: - path: '/var/tmp' # change this path according to your cluster configuration. + path: '/tmp/spark-local' # change this path according to your cluster configuration. containers: - name: shuffle # This is an official image that is built # from the dockerfiles/shuffle directory # in the spark distribution. - image: kubespark/spark-shuffle:v2.2.0-kubernetes-0.3.0 + image: kubespark/spark-shuffle:v2.2.0-kubernetes-0.4.0 imagePullPolicy: IfNotPresent volumeMounts: - - mountPath: '/tmp' + - mountPath: '/tmp/spark-local' name: temp-volume # more volumes can be mounted here. # The spark job must be configured to use these diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c0b0a3f5f6963..a49004bdf2348 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -715,7 +715,7 @@ object SparkSubmit extends CommandLineUtils with Logging { } if (isKubernetesCluster) { - childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" + childMainClass = "org.apache.spark.deploy.k8s.submit.Client" if (args.isPython) { childArgs ++= Array("--primary-py-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 02b0610091450..064da0fd074d8 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -262,7 +262,7 @@ package object config { private[spark] val DRIVER_HOST_ADDRESS = ConfigBuilder("spark.driver.host") .doc("Address of driver endpoints.") .stringConf - .createWithDefault(Utils.localHostName()) + .createWithDefault(Utils.localCanonicalHostName()) private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") .doc("Address where to bind network listen sockets on the driver.") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4a6c64d754ecd..930e09d90c2f5 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -941,6 +941,13 @@ private[spark] object Utils extends Logging { customHostname = Some(hostname) } + /** + * Get the local machine's FQDN. + */ + def localCanonicalHostName(): String = { + customHostname.getOrElse(localIpAddress.getCanonicalHostName) + } + /** * Get the local machine's hostname. */ diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index cbaab84379db0..7b01cf4c3c3d8 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -24,6 +24,8 @@ should give you a list of pods and configmaps (if any) respectively. * You must have a spark distribution with Kubernetes support. This may be obtained from the [release tarball](https://github.com/apache-spark-on-k8s/spark/releases) or by [building Spark with Kubernetes support](../resource-managers/kubernetes/README.md#building-spark-with-kubernetes-support). +* You must have [Kubernetes DNS](https://kubernetes.io/docs/concepts/services-networking/dns-pod-service/) configured in +your cluster. ## Driver & Executor Images @@ -59,7 +61,7 @@ to the registry. For example, if the registry host is `registry-host` and the registry is listening on port 5000: cd $SPARK_HOME - docker build -t registry-host:5000/spark-base:latest -f dockerfiles/driver/spark-base . + docker build -t registry-host:5000/spark-base:latest -f dockerfiles/spark-base/Dockerfile . docker build -t registry-host:5000/spark-driver:latest -f dockerfiles/driver/Dockerfile . docker build -t registry-host:5000/spark-executor:latest -f dockerfiles/executor/Dockerfile . docker build -t registry-host:5000/spark-init:latest -f dockerfiles/init-container/Dockerfile . @@ -222,7 +224,7 @@ Below is an example submission: local:///opt/spark/examples/src/main/python/pi.py 100 ``` -## Dynamic Executor Scaling +## Dynamic Allocation in Kubernetes Spark on Kubernetes supports Dynamic Allocation with cluster mode. This mode requires running an external shuffle service. This is typically a [daemonset](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) @@ -245,6 +247,7 @@ the command may then look like the following: --class org.apache.spark.examples.GroupByTest \ --master k8s://: \ --kubernetes-namespace default \ + --conf spark.local.dir=/tmp/spark-local --conf spark.app.name=group-by-test \ --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:latest \ --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:latest \ @@ -254,6 +257,14 @@ the command may then look like the following: --conf spark.kubernetes.shuffle.labels="app=spark-shuffle-service,spark-version=2.2.0" \ local:///opt/spark/examples/jars/spark-examples_2.11-2.2.0-k8s-0.3.0.jar 10 400000 2 +The external shuffle service has to mount directories that can be shared with the executor pods. The provided example +YAML spec mounts a hostPath volume to the external shuffle service pods, but these hostPath volumes must also be mounted +into the executors. When using the external shuffle service, the directories specified in the `spark.local.dir` +configuration are mounted as hostPath volumes into all of the executor containers. To ensure that one does not +accidentally mount the incorrect hostPath volumes, the value of `spark.local.dir` must be specified in your +application's configuration when using Kubernetes, even though it defaults to the JVM's temporary directory when using +other cluster managers. + ## Advanced ### Securing the Resource Staging Server with TLS @@ -606,48 +617,6 @@ from the other deployment modes. See the [configuration page](configuration.html myIdentifier. Multiple annotations can be added by setting multiple configurations with this prefix. - - spark.kubernetes.driver.labels - (none) - - Deprecated. Use spark.kubernetes.driver.label. instead which supports = - and , characters in label values. - Custom labels that will be added to the driver pod. This should be a comma-separated list of label key-value pairs, - where each label is in the format key=value. Note that Spark also adds its own labels to the driver pod - for bookkeeping purposes. - - - - spark.kubernetes.driver.annotations - (none) - - Deprecated. Use spark.kubernetes.driver.annotation. instead which supports - = and , characters in annotation values. - Custom annotations that will be added to the driver pod. This should be a comma-separated list of label key-value - pairs, where each annotation is in the format key=value. - - - - spark.kubernetes.executor.labels - (none) - - Deprecated. Use spark.kubernetes.executor.label. instead which supports - = and , characters in label values. - Custom labels that will be added to the executor pods. This should be a comma-separated list of label key-value - pairs, where each label is in the format key=value. Note that Spark also adds its own labels to the - executor pods for bookkeeping purposes. - - - - spark.kubernetes.executor.annotations - (none) - - Deprecated. Use spark.kubernetes.executor.annotation. instead which supports - = and , characters in annotation values. - Custom annotations that will be added to the executor pods. This should be a comma-separated list of annotation - key-value pairs, where each annotation is in the format key=value. - - spark.kubernetes.driver.pod.name (none) @@ -800,6 +769,22 @@ from the other deployment modes. See the [configuration page](configuration.html the Driver process. The user can specify multiple of these to set multiple environment variables. + + spark.kubernetes.driver.secrets.[SecretName] + (none) + + Mounts the Kubernetes secret named SecretName onto the path specified by the value + in the driver Pod. The user can specify multiple instances of this for multiple secrets. + + + + spark.kubernetes.executor.secrets.[SecretName] + (none) + + Mounts the Kubernetes secret named SecretName onto the path specified by the value + in the executor Pods. The user can specify multiple instances of this for multiple secrets. + + diff --git a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index 55e7e38b28a08..81d14766ffb8d 100644 --- a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1 +1 @@ -org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterManager +org.apache.spark.scheduler.cluster.k8s.KubernetesClusterManager diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala index a6f0ca502f6f0..364882d542fa6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStream, OutputStream} import java.util.zip.{GZIPInputStream, GZIPOutputStream} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala similarity index 65% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 1a008c236d00f..601533615ce9a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.OptionalConfigEntry object ConfigurationUtils extends Logging { def parseKeyValuePairs( @@ -41,31 +40,6 @@ object ConfigurationUtils extends Logging { }).getOrElse(Map.empty[String, String]) } - def combinePrefixedKeyValuePairsWithDeprecatedConf( - sparkConf: SparkConf, - prefix: String, - deprecatedConf: OptionalConfigEntry[String], - configType: String): Map[String, String] = { - val deprecatedKeyValuePairsString = sparkConf.get(deprecatedConf) - deprecatedKeyValuePairsString.foreach { _ => - logWarning(s"Configuration with key ${deprecatedConf.key} is deprecated. Use" + - s" configurations with prefix $prefix instead.") - } - val fromDeprecated = parseKeyValuePairs( - deprecatedKeyValuePairsString, - deprecatedConf.key, - configType) - val fromPrefix = sparkConf.getAllWithPrefix(prefix) - val combined = fromDeprecated.toSeq ++ fromPrefix - combined.groupBy(_._1).foreach { - case (key, values) => - require(values.size == 1, - s"Cannot have multiple values for a given $configType key, got key $key with" + - s" values $values") - } - combined.toMap - } - def parsePrefixedKeyValuePairs( sparkConf: SparkConf, prefix: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala index 265b8f197a102..99947498d6a89 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait InitContainerResourceStagingServerSecretPlugin { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala index aba94e6969529..833af9b3b5638 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s case class KubernetesCredentials( oauthTokenBase64: Option[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala index c61f4f1d44acf..661c3a40a15a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File import java.nio.ByteBuffer @@ -29,8 +29,8 @@ import scala.collection.mutable import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala index eda43de0a9a5b..89053de5b9558 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s private[spark] object OptionRequirements { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala index 36b1b07dc6bc4..cdec3a155a597 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index d2729a2db2fa0..c90303b6ba506 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File @@ -25,7 +25,7 @@ import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.ThreadUtils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala index 2df7ac7a204c4..35246506ecb33 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ /** * This is separated out from the init-container steps API because this component can be reused to diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index 53a184cba7a4d..0e35e04ff5803 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.util.concurrent.TimeUnit import org.apache.spark.{SPARK_VERSION => sparkVersion} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit @@ -115,42 +115,14 @@ package object config extends Logging { private[spark] val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." private[spark] val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." - private[spark] val KUBERNETES_DRIVER_LABELS = - ConfigBuilder("spark.kubernetes.driver.labels") - .doc("Custom labels that will be added to the driver pod. This should be a comma-separated" + - " list of label key-value pairs, where each label is in the format key=value. Note that" + - " Spark also adds its own labels to the driver pod for bookkeeping purposes.") - .stringConf - .createOptional - private[spark] val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." - private[spark] val KUBERNETES_DRIVER_ANNOTATIONS = - ConfigBuilder("spark.kubernetes.driver.annotations") - .doc("Custom annotations that will be added to the driver pod. This should be a" + - " comma-separated list of annotation key-value pairs, where each annotation is in the" + - " format key=value.") - .stringConf - .createOptional - - private[spark] val KUBERNETES_EXECUTOR_LABELS = - ConfigBuilder("spark.kubernetes.executor.labels") - .doc("Custom labels that will be added to the executor pods. This should be a" + - " comma-separated list of label key-value pairs, where each label is in the format" + - " key=value.") - .stringConf - .createOptional - - private[spark] val KUBERNETES_EXECUTOR_ANNOTATIONS = - ConfigBuilder("spark.kubernetes.executor.annotations") - .doc("Custom annotations that will be added to the executor pods. This should be a" + - " comma-separated list of annotation key-value pairs, where each annotation is in the" + - " format key=value.") - .stringConf - .createOptional + private[spark] val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." + private[spark] val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." private[spark] val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") @@ -185,12 +157,6 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val KUBERNETES_SHUFFLE_DIR = - ConfigBuilder("spark.kubernetes.shuffle.dir") - .doc("Path to the shared shuffle directories.") - .stringConf - .createOptional - private[spark] val KUBERNETES_SHUFFLE_APISERVER_URI = ConfigBuilder("spark.kubernetes.shuffle.apiServer.url") .doc("URL to the Kubernetes API server that the shuffle service will monitor for Spark pods.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index 2c2ccf31b9dd9..95d7f284f86da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s package object constants { // Labels @@ -44,11 +44,11 @@ package object constants { private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" // Default and fixed ports - private[spark] val SUBMISSION_SERVER_PORT = 7077 private[spark] val DEFAULT_DRIVER_PORT = 7078 private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 private[spark] val DEFAULT_UI_PORT = 4040 private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private[spark] val DRIVER_PORT_NAME = "driver-rpc-port" private[spark] val EXECUTOR_PORT_NAME = "executor" // Environment Variables @@ -102,4 +102,5 @@ package object constants { private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 private[spark] val MEMORY_OVERHEAD_MIN_MIB = 384L + private[spark] val GENERATED_LOCAL_DIR_MOUNT_ROOT = "/mnt/tmp/spark-local" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 9573bc222505f..0190d3eccfe3b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.{Collections, UUID} @@ -24,10 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +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.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala index 434919208ba2e..94adcc5857d2a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.lang.Boolean diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala similarity index 80% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index b66da0b154698..3203cebba71e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -14,16 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.LocalDirectoryMountConfigurationStep +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.util.Utils +import org.apache.spark.util.{SystemClock, Utils} /** * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. @@ -72,10 +73,9 @@ private[spark] class DriverConfigurationStepsOrchestrator( .getOrElse(Array.empty[String]) ++ additionalMainAppPythonFile.toSeq ++ additionalPythonFiles - val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( submissionSparkConf, KUBERNETES_DRIVER_LABEL_PREFIX, - KUBERNETES_DRIVER_LABELS, "label") require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + @@ -83,6 +83,11 @@ private[spark] class DriverConfigurationStepsOrchestrator( val allDriverLabels = driverCustomLabels ++ Map( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX, + "driver secrets") + val initialSubmissionStep = new BaseDriverConfigurationStep( kubernetesAppId, kubernetesResourceNamePrefix, @@ -92,8 +97,17 @@ private[spark] class DriverConfigurationStepsOrchestrator( mainClass, appArgs, submissionSparkConf) + val driverAddressStep = new DriverAddressConfigurationStep( + kubernetesResourceNamePrefix, + allDriverLabels, + submissionSparkConf, + new SystemClock) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + + val localDirectoryMountConfigurationStep = new LocalDirectoryMountConfigurationStep( + submissionSparkConf) + val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -113,7 +127,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( // Then, indicate to the outer block that the init-container should not handle // those local files simply by filtering them out. val sparkFilesWithoutLocal = KubernetesFileUtils.getNonSubmitterLocalFiles(sparkFiles) - val smallFilesSecretName = s"${kubernetesAppId}-submitted-files" + val smallFilesSecretName = s"$kubernetesAppId-submitted-files" val mountSmallFilesBootstrap = new MountSmallFilesBootstrapImpl( smallFilesSecretName, MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH) val mountSmallLocalFilesStep = new MountSmallLocalFilesStep( @@ -153,17 +167,29 @@ private[spark] class DriverConfigurationStepsOrchestrator( } else { (filesDownloadPath, Seq.empty[DriverConfigurationStep]) } + val dependencyResolutionStep = new DependencyResolutionStep( sparkJars, sparkFiles, jarsDownloadPath, localFilesDownloadPath) + + val mountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + Some(new MountSecretsStep(mountSecretsBootstrap)) + } else { + None + } + Seq( initialSubmissionStep, + driverAddressStep, kubernetesCredentialsStep, - dependencyResolutionStep) ++ + dependencyResolutionStep, + localDirectoryMountConfigurationStep) ++ submittedDependenciesBootstrapSteps ++ - pythonStep.toSeq + pythonStep.toSeq ++ + mountSecretsStep.toSeq } private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala index 837ec0e8c867e..55ed06d7ca415 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] object InitContainerUtil { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index ec591923f1472..00ff72ec0efb1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 4a8a7308b9fe4..c645b008d736d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.concurrent.{CountDownLatch, TimeUnit} @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils -private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { +private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { def awaitCompletion(): Unit } @@ -39,7 +39,7 @@ private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { * @param maybeLoggingInterval ms between each state request. If provided, must be a positive * number. */ -private[kubernetes] class LoggingPodStatusWatcherImpl( +private[k8s] class LoggingPodStatusWatcherImpl( appId: String, maybeLoggingInterval: Option[Long]) extends LoggingPodStatusWatcher with Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala index 436d531a850ff..f806e65974fcc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit private[spark] sealed trait MainAppResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala new file mode 100644 index 0000000000000..e83dcdcdc0637 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala @@ -0,0 +1,67 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} + +/** + * Bootstraps a driver or executor pod with needed secrets mounted. + */ +private[spark] trait MountSecretsBootstrap { + + /** + * Mounts Kubernetes secrets as secret volumes into the given container in the given pod. + * + * @param pod the pod into which the secret volumes are being added. + * @param container the container into which the secret volumes are being mounted. + * @return the updated pod and container with the secrets mounted. + */ + def mountSecrets(pod: Pod, container: Container): (Pod, Container) +} + +private[spark] class MountSecretsBootstrapImpl( + secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap { + + override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { + var podBuilder = new PodBuilder(pod) + secretNamesToMountPaths.keys.foreach(name => + podBuilder = podBuilder + .editOrNewSpec() + .addNewVolume() + .withName(secretVolumeName(name)) + .withNewSecret() + .withSecretName(name) + .endSecret() + .endVolume() + .endSpec()) + + var containerBuilder = new ContainerBuilder(container) + secretNamesToMountPaths.foreach(namePath => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(namePath._1)) + .withMountPath(namePath._2) + .endVolumeMount() + ) + + (podBuilder.build(), containerBuilder.build()) + } + + private def secretVolumeName(secretName: String): String = { + secretName + "-volume" + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala index 79919b511ec37..2ace625b4d6a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait MountSmallFilesBootstrap { def mountSmallFilesSecret(pod: Pod, container: Container): (Pod, Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala index 8103272c27518..d1d20711abffe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.StringWriter import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala index 83d7a28f5ca10..7b829351a28ba 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{File, FileOutputStream} import javax.ws.rs.core.MediaType @@ -27,8 +27,8 @@ import okhttp3.RequestBody import retrofit2.Call import org.apache.spark.{SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} import org.apache.spark.util.Utils private[spark] trait SubmittedDependencyUploader { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala index 225972c1057f2..b02cb21372678 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit case class SubmittedResourceIdAndSecret(resourceId: String, resourceSecret: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala index b76c77a656d2d..563662ef80d11 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, PodBuilder, QuantityBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Represents the initial setup required for the driver. @@ -67,10 +67,9 @@ private[spark] class BaseDriverConfigurationStep( .build() } val driverCustomAnnotations = ConfigurationUtils - .combinePrefixedKeyValuePairsWithDeprecatedConf( + .parsePrefixedKeyValuePairs( submissionSparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX, - KUBERNETES_DRIVER_ANNOTATIONS, "annotation") require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala index 090240420119e..c2c76c2ba1960 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStep +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStep import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala new file mode 100644 index 0000000000000..615261e80f302 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala @@ -0,0 +1,102 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.ServiceBuilder +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.Clock + +/** + * Allows the driver to be reachable by executor pods through a headless service. The service's + * ports should correspond to the ports that the executor will reach the pod at for RPC. + */ +private[spark] class DriverAddressConfigurationStep( + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + submissionSparkConf: SparkConf, + clock: Clock) extends DriverConfigurationStep with Logging { + import DriverAddressConfigurationStep._ + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, + s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's hostname" + + s" will be managed via a Kubernetes service.") + require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, + s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be" + + s" managed via a Kubernetes service.") + + val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" + val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { + preferredServiceName + } else { + val randomServiceId = clock.getTimeMillis() + val shorterServiceName = s"spark-$randomServiceId$DRIVER_SVC_POSTFIX" + logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is" + + s" too long (must be <= 63 characters). Falling back to use $shorterServiceName" + + s" as the driver service's name.") + shorterServiceName + } + + val driverPort = submissionSparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverBlockManagerPort = submissionSparkConf.getInt( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + val driverService = new ServiceBuilder() + .withNewMetadata() + .withName(resolvedServiceName) + .endMetadata() + .withNewSpec() + .withClusterIP("None") + .withSelector(driverLabels.asJava) + .addNewPort() + .withName(DRIVER_PORT_NAME) + .withPort(driverPort) + .withNewTargetPort(driverPort) + .endPort() + .addNewPort() + .withName(BLOCK_MANAGER_PORT_NAME) + .withPort(driverBlockManagerPort) + .withNewTargetPort(driverBlockManagerPort) + .endPort() + .endSpec() + .build() + + val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) + val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, driverHostname) + .set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, driverHostname) + .set("spark.driver.port", driverPort.toString) + .set( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, driverBlockManagerPort) + + driverSpec.copy( + driverSparkConf = resolvedSparkConf, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(driverService)) + } +} + +private[spark] object DriverAddressConfigurationStep { + val DRIVER_BIND_ADDRESS_KEY = org.apache.spark.internal.config.DRIVER_BIND_ADDRESS.key + val DRIVER_HOST_KEY = org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key + val DRIVER_SVC_POSTFIX = "-driver-svc" + val MAX_SERVICE_NAME_LENGTH = 63 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala index 8070e32371f94..706b02db57672 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps /** * Represents a step in preparing the Kubernetes driver. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala index 74df2de480bf4..da5686099fbbb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import java.nio.charset.StandardCharsets @@ -25,8 +25,8 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala index b4248338cc8de..f8d1c909ac794 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} /** * Configures the init-container that bootstraps dependencies into the driver pod. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala index 3ec4b6c4df10f..f76c0a8c9c6bd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala new file mode 100644 index 0000000000000..3f9ba8af74162 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala @@ -0,0 +1,98 @@ +/* + * 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.submit.submitsteps + +import java.nio.file.Paths +import java.util.UUID + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, VolumeBuilder, VolumeMountBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.constants._ + +/** + * Configures local directories that the driver and executors should use for temporary storage. + * + * Note that we have different semantics for scratch space in Kubernetes versus the other cluster + * managers. In Kubernetes, we cannot allow the local directories to resolve to the Java temporary + * directory. This is because we will mount either emptyDir volumes for both the driver and + * executors, or hostPath volumes for the executors and an emptyDir for the driver. In either + * case, the mount paths need to be directories that do not exist in the base container images. + * But the Java temporary directory is typically a directory like /tmp which exists in most + * container images. + * + * The solution is twofold: + * - When not using an external shuffle service, a reasonable default is to create a new directory + * with a random name and set that to be the value of `spark.local.dir`. + * - When using the external shuffle service, it is risky to assume that the user intends to mount + * the JVM temporary directory into the pod as a hostPath volume. We therefore enforce that + * spark.local.dir must be set in dynamic allocation mode so that the user explicitly sets the + * paths that have to be mounted. + */ +private[spark] class LocalDirectoryMountConfigurationStep( + submissionSparkConf: SparkConf, + randomDirProvider: () => String = () => s"spark-${UUID.randomUUID()}") + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val configuredLocalDirs = submissionSparkConf.getOption("spark.local.dir") + val isUsingExternalShuffle = submissionSparkConf.get( + org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED) + val resolvedLocalDirsSingleString = if (isUsingExternalShuffle) { + require(configuredLocalDirs.isDefined, "spark.local.dir must be provided explicitly when" + + " using the external shuffle service in Kubernetes. These directories should map to" + + " the paths that are mounted into the external shuffle service pods.") + configuredLocalDirs.get + } else { + // If we don't use the external shuffle service, local directories should be randomized if + // not provided. + configuredLocalDirs.getOrElse(s"$GENERATED_LOCAL_DIR_MOUNT_ROOT/${randomDirProvider()}") + } + val resolvedLocalDirs = resolvedLocalDirsSingleString.split(",") + // It's worth noting that we always use an emptyDir volume for the directories on the driver, + // because the driver does not need a hostPath to share its scratch space with any other pod. + // The driver itself will decide on whether to use a hostPath volume or an emptyDir volume for + // these directories on the executors. (see ExecutorPodFactory and + // KubernetesExternalClusterManager) + val localDirVolumes = resolvedLocalDirs.zipWithIndex.map { case (dir, index) => + new VolumeBuilder() + .withName(s"spark-local-dir-$index-${Paths.get(dir).getFileName.toString}") + .withNewEmptyDir().endEmptyDir() + .build() + } + val localDirVolumeMounts = localDirVolumes.zip(resolvedLocalDirs).map { + case (volume, path) => + new VolumeMountBuilder() + .withName(volume.getName) + .withMountPath(path) + .build() + } + val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone().set( + "spark.local.dir", resolvedLocalDirsSingleString) + driverSpec.copy( + driverPod = new PodBuilder(driverSpec.driverPod) + .editSpec() + .addToVolumes(localDirVolumes: _*) + .endSpec() + .build(), + driverContainer = new ContainerBuilder(driverSpec.driverContainer) + .addToVolumeMounts(localDirVolumeMounts: _*) + .build(), + driverSparkConf = resolvedDriverSparkConf + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala new file mode 100644 index 0000000000000..21c7cafd987cd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala @@ -0,0 +1,37 @@ +/* + * 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.submit.submitsteps + +import org.apache.spark.deploy.k8s.submit.MountSecretsBootstrap + +/** + * A driver configuration step for mounting user-specified secrets onto user-specified paths. + * + * @param mountSecretsBootstrap a utility actually handling mounting of the secrets. + */ +private[spark] class MountSecretsStep( + mountSecretsBootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets(driverSpec.driverPod, driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithSecretsMounted, + driverContainer = driverContainerWithSecretsMounted + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala index 6f9796a70eeb6..52503b1b3910e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,8 +22,8 @@ import com.google.common.io.{BaseEncoding, Files} import io.fabric8.kubernetes.api.model.SecretBuilder import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStep( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala index 024d643ddf9fd..f0003924e9c6d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class PythonStep( primaryPyFile: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala index 60bf27beacaaf..dc335ac8f076e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class BaseInitContainerConfigurationStep( sparkJars: Seq[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala index 7b7622c3d4f8b..a3bdaa8e93344 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer /** * Represents a step in preparing the init-container for the driver and executors. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala similarity index 88% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index e4ea5235af18f..07a45dc1f1613 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} +import org.apache.spark.deploy.k8s.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{KubernetesFileUtils, SubmittedDependencyUploaderImpl} +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} import org.apache.spark.util.Utils /** @@ -62,6 +62,12 @@ private[spark] class InitContainerConfigurationStepsOrchestrator( submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) .getOrElse(false) + + OptionRequirements.requireSecondIfFirstIsDefined( + KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkJars).headOption, + resourceStagingServerUri, + "Local JARs were provided, however no resource staging server URI was found.") + OptionRequirements.requireNandDefined( maybeResourceStagingServerInternalClientCert, maybeResourceStagingServerInternalTrustStore, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala index 5b5ac3c1f17c2..5fcb0495f0b19 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala index 7aa27a1de6811..b803b67978397 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File @@ -24,10 +24,10 @@ import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.SubmittedDependencyUploader import org.apache.spark.internal.config.OptionalConfigEntry import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala index d050e0a41a15a..a40ced5a55be7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala index 0e274678ad6f0..1c78b307c2de5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.util.concurrent.TimeUnit @@ -29,8 +29,8 @@ import scala.concurrent.duration.Duration import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SSLOptions} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.kubernetes.{CompressionUtils, KubernetesCredentials} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{CompressionUtils, KubernetesCredentials} +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala index 17f90118e150d..777ed70d82c22 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStreamReader} import java.security.{KeyStore, PrivateKey} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala index 0b97317eba8b1..9a63830966dbc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File @@ -31,8 +31,8 @@ import org.glassfish.jersey.server.ResourceConfig import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{SystemClock, ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala index cb1e65421c013..30c5d3917d74a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.security.SecureRandom @@ -24,8 +24,8 @@ import com.google.common.io.Files import org.apache.commons.lang3.RandomStringUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.OptionRequirements -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.OptionRequirements +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging private[spark] trait ResourceStagingServerSslOptionsProvider { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala index b9d283a99ade9..b48b2ca4859d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.InputStream import javax.ws.rs.{Consumes, GET, HeaderParam, Path, PathParam, POST, Produces} @@ -22,8 +22,8 @@ import javax.ws.rs.core.{MediaType, StreamingOutput} import org.glassfish.jersey.media.multipart.FormDataParam -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Service that receives application data that can be retrieved later on. This is primarily used diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala index 7bc21c21619e1..ebf37fd277cd6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, OutputStream} import java.security.SecureRandom @@ -26,8 +26,8 @@ import com.google.common.io.{BaseEncoding, ByteStreams, Files} import scala.collection.concurrent.TrieMap import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala index 5fbf0f9c43970..53803f46163e6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import okhttp3.{RequestBody, ResponseBody} import retrofit2.Call import retrofit2.http.{Multipart, Path, Streaming} -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala index 5046cb479054c..f5d41a5012bc1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.FileInputStream import java.net.{InetSocketAddress, URI} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala index 9e2b8a780df29..4376de42bd751 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala index 81f394800f803..f4038ae05fd99 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala index 5d9db728483fa..38636ac7d992d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala index 4061bc36764d7..9a9691371b3f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.module.scala.JsonScalaEnumeration diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala index 0c0d428e035dc..7cb819dc90804 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, IOException} import java.security.SecureRandom diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala new file mode 100644 index 0000000000000..2b35fd6a513f5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala @@ -0,0 +1,61 @@ +/* + * 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.scheduler.cluster.k8s + +import java.nio.file.Paths + +import io.fabric8.kubernetes.api.model.{Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +private[spark] trait ExecutorLocalDirVolumeProvider { + def getExecutorLocalDirVolumesWithMounts: Seq[(Volume, VolumeMount)] +} + +private[spark] class ExecutorLocalDirVolumeProviderImpl( + sparkConf: SparkConf, + kubernetesExternalShuffleManager: Option[KubernetesExternalShuffleManager]) + extends ExecutorLocalDirVolumeProvider { + override def getExecutorLocalDirVolumesWithMounts: Seq[(Volume, VolumeMount)] = { + kubernetesExternalShuffleManager.map(_.getExecutorShuffleDirVolumesWithMounts) + .getOrElse { + // If we're not using the external shuffle manager, we should use emptyDir volumes for + // shuffle directories since it's important for disk I/O for these directories to be + // performant. If the user has not provided a local directory, instead of using the + // Java temporary directory, we create one instead, because we want to avoid + // mounting an emptyDir which overlaps with an existing path in the Docker image. + // Java's temporary directory path is typically /tmp or a similar path, which is + // likely to exist in most images. + val resolvedLocalDirs = Utils.getConfiguredLocalDirs(sparkConf) + val localDirVolumes = resolvedLocalDirs.zipWithIndex.map { case (dir, index) => + new VolumeBuilder() + .withName(s"spark-local-dir-$index-${Paths.get(dir).getFileName.toString}") + .withNewEmptyDir().endEmptyDir() + .build() + } + val localDirVolumeMounts = localDirVolumes.zip(resolvedLocalDirs).map { + case (volume, path) => + new VolumeMountBuilder() + .withName(volume.getName) + .withMountPath(path) + .build() + } + localDirVolumes.zip(localDirVolumeMounts) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala new file mode 100644 index 0000000000000..98a0d879b6a58 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -0,0 +1,274 @@ +/* + * 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.scheduler.cluster.k8s + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder, VolumeBuilder, VolumeMountBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} +import org.apache.spark.util.Utils + +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are +// common across all executors. Then, pass in dynamic parameters into createExecutorPod. +private[spark] trait ExecutorPodFactory { + def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod +} + +private[spark] class ExecutorPodFactoryImpl( + sparkConf: SparkConf, + nodeAffinityExecutorPodModifier: NodeAffinityExecutorPodModifier, + mountSecretsBootstrap: Option[MountSecretsBootstrap], + mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], + executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], + executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], + executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider) + extends ExecutorPodFactory { + + import ExecutorPodFactoryImpl._ + + private val executorExtraClasspath = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + + private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_EXECUTOR_LABEL_PREFIX, + "executor label") + require( + !executorLabels.contains(SPARK_APP_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") + require( + !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + + s" Spark.") + + private val executorAnnotations = + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, + "executor annotation") + private val nodeSelector = + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_NODE_SELECTOR_PREFIX, + "node selector") + + private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) + private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) + private val blockmanagerPort = sparkConf + .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + private val kubernetesDriverPodName = sparkConf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) + + private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) + + private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) + private val executorMemoryString = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_MEMORY.key, + org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) + + private val memoryOverheadMiB = sparkConf + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) + private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB + + private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d) + private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) + + override def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod = { + val name = s"$executorPodNamePrefix-exec-$executorId" + + // hostname must be no longer than 63 characters, so take the last 63 characters of the pod + // name as the hostname. This preserves uniqueness since the end of name contains + // executorId and applicationId + val hostname = name.substring(Math.max(0, name.length - 63)) + val resolvedExecutorLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ + executorLabels + val executorMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryMiB}Mi") + .build() + val executorMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryWithOverhead}Mi") + .build() + val executorCpuQuantity = new QuantityBuilder(false) + .withAmount(executorCores.toString) + .build() + val executorExtraClasspathEnv = executorExtraClasspath.map { cp => + new EnvVarBuilder() + .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) + .withValue(cp) + .build() + } + val executorExtraJavaOptionsEnv = sparkConf + .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + .map { opts => + val delimitedOpts = Utils.splitCommandString(opts) + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) + val executorEnv = (Seq( + (ENV_EXECUTOR_PORT, executorPort.toString), + (ENV_DRIVER_URL, driverUrl), + // Executor backend expects integral value for executor cores, so round it up to an int. + (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, applicationId), + (ENV_EXECUTOR_ID, executorId), + (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs) + .map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build() + ) ++ Seq( + new EnvVarBuilder() + .withName(ENV_EXECUTOR_POD_IP) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) + .build() + ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq + val requiredPorts = Seq( + (EXECUTOR_PORT_NAME, executorPort), + (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + .map(port => { + new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build() + }) + val shuffleVolumesWithMounts = executorLocalDirVolumeProvider + .getExecutorLocalDirVolumesWithMounts + + val executorContainer = new ContainerBuilder() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .endResources() + .addAllToEnv(executorEnv.asJava) + .withPorts(requiredPorts.asJava) + .addAllToVolumeMounts(shuffleVolumesWithMounts.map(_._2).asJava) + .build() + + val executorPod = new PodBuilder() + .withNewMetadata() + .withName(name) + .withLabels(resolvedExecutorLabels.asJava) + .withAnnotations(executorAnnotations.asJava) + .withOwnerReferences() + .addNewOwnerReference() + .withController(true) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withHostname(hostname) + .withRestartPolicy("Never") + .withNodeSelector(nodeSelector.asJava) + .addAllToVolumes(shuffleVolumesWithMounts.map(_._1).asJava) + .endSpec() + .build() + + val containerWithExecutorLimitCores = executorLimitCores.map { + limitCores => + val executorCpuLimitQuantity = new QuantityBuilder(false) + .withAmount(limitCores) + .build() + new ContainerBuilder(executorContainer) + .editResources() + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .build() + }.getOrElse(executorContainer) + + val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = + mountSecretsBootstrap.map {bootstrap => + bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores) + }.getOrElse((executorPod, containerWithExecutorLimitCores)) + val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = + mountSmallFilesBootstrap.map { bootstrap => + bootstrap.mountSmallFilesSecret( + withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) + }.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)) + val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = + executorInitContainerBootstrap.map { bootstrap => + val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + withMaybeSmallFilesMountedPod, + new ContainerBuilder().build(), + withMaybeSmallFilesMountedContainer)) + + val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => + plugin.mountResourceStagingServerSecretIntoInitContainer( + podWithDetachedInitContainer.initContainer) + }.getOrElse(podWithDetachedInitContainer.initContainer) + + val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( + podWithDetachedInitContainer.pod, resolvedInitContainer) + + val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => + plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) + }.getOrElse(podWithAttachedInitContainer) + + (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) + }.getOrElse((withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer)) + + val executorPodWithNodeAffinity = + nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( + executorPodWithInitContainer, nodeToLocalTaskCount) + + new PodBuilder(executorPodWithNodeAffinity) + .editSpec() + .addToContainers(initBootstrappedExecutorContainer) + .endSpec() + .build() + } +} + +private object ExecutorPodFactoryImpl { + private val DEFAULT_STATIC_PORT = 10000 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala similarity index 87% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala index e04ab9e541963..8d6d92ac7b412 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala @@ -14,19 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.net.InetAddress /** * Gets full host names of given IP addresses from DNS. */ -private[kubernetes] trait InetAddressUtil { +private[k8s] trait InetAddressUtil { def getFullHostName(ipAddress: String): String } -private[kubernetes] object InetAddressUtilImpl extends InetAddressUtil { +private[k8s] object InetAddressUtilImpl extends InetAddressUtil { // NOTE: This does issue a network call to DNS. Caching is done internally by the InetAddress // class for both hits and misses. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala similarity index 65% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index b89e81bcb0be9..6ae76c4a7bda3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -14,19 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} import org.apache.spark.internal.Logging +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { @@ -51,6 +54,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) val maybeExecutorInitContainerSecretMountPath = sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) + val executorInitContainerSecretVolumePlugin = for { initContainerSecretName <- maybeExecutorInitContainerSecretName initContainerSecretMountPath <- maybeExecutorInitContainerSecretMountPath @@ -59,10 +63,11 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit initContainerSecretName, initContainerSecretMountPath) } + // Only set up the bootstrap if they've provided both the config map key and the config map // name. The config map might not be provided if init-containers aren't being used to // bootstrap dependencies. - val executorInitContainerbootStrap = for { + val executorInitContainerBootstrap = for { configMap <- maybeInitContainerConfigMap configMapKey <- maybeInitContainerConfigMapKey } yield { @@ -75,12 +80,22 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } + val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath } yield { new MountSmallFilesBootstrapImpl(secretName, secretMountPath) } + + val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(sparkConf, + KUBERNETES_EXECUTOR_SECRETS_PREFIX, "executor secrets") + val mountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + } else { + None + } + if (maybeInitContainerConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") @@ -89,6 +104,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit logWarning("The executor's init-container config map key was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") } + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), @@ -96,13 +112,42 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf, Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) - new KubernetesClusterSchedulerBackend( - sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], - sc, - executorInitContainerbootStrap, - executorInitContainerSecretVolumePlugin, + + val kubernetesShuffleManager = if (sparkConf.get( + org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED)) { + val kubernetesExternalShuffleClient = new KubernetesExternalShuffleClientImpl( + SparkTransportConf.fromSparkConf(sparkConf, "shuffle"), + sc.env.securityManager, + sc.env.securityManager.isAuthenticationEnabled(), + sparkConf.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) + Some(new KubernetesExternalShuffleManagerImpl( + sparkConf, + kubernetesClient, + kubernetesExternalShuffleClient)) + } else None + + val executorLocalDirVolumeProvider = new ExecutorLocalDirVolumeProviderImpl( + sparkConf, kubernetesShuffleManager) + val executorPodFactory = new ExecutorPodFactoryImpl( + sparkConf, + NodeAffinityExecutorPodModifierImpl, + mountSecretBootstrap, mountSmallFilesBootstrap, - kubernetesClient) + executorInitContainerBootstrap, + executorInitContainerSecretVolumePlugin, + executorLocalDirVolumeProvider) + val allocatorExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") + val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( + "kubernetes-executor-requests") + new KubernetesClusterSchedulerBackend( + scheduler.asInstanceOf[TaskSchedulerImpl], + sc.env.rpcEnv, + executorPodFactory, + kubernetesShuffleManager, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala new file mode 100644 index 0000000000000..687e89d659fd8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,490 @@ +/* + * 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.scheduler.cluster.k8s + +import java.io.Closeable +import java.net.InetAddress +import java.util.Collections +import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, ThreadPoolExecutor, TimeUnit} +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} +import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils + +private[spark] class KubernetesClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + rpcEnv: RpcEnv, + executorPodFactory: ExecutorPodFactory, + shuffleManager: Option[KubernetesExternalShuffleManager], + kubernetesClient: KubernetesClient, + allocatorExecutor: ScheduledExecutorService, + requestExecutorsService: ExecutorService) + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + + import KubernetesClusterSchedulerBackend._ + + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) + private val RUNNING_EXECUTOR_PODS_LOCK = new Object + // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK. + private val runningExecutorsToPods = new mutable.HashMap[String, Pod] + // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK. + private val runningPodsToExecutors = new mutable.HashMap[String, String] + private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]() + private val podsWithKnownExitReasons = new ConcurrentHashMap[String, ExecutorExited]() + private val disconnectedPodsByExecutorIdPendingRemoval = new ConcurrentHashMap[String, Pod]() + + private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) + + private val kubernetesDriverPodName = conf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse( + throw new SparkException("Must specify the driver pod name")) + private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( + requestExecutorsService) + + private val driverPod = try { + kubernetesClient.pods().inNamespace(kubernetesNamespace). + withName(kubernetesDriverPodName).get() + } catch { + case throwable: Throwable => + logError(s"Executor cannot find driver pod.", throwable) + throw new SparkException(s"Executor cannot find driver pod", throwable) + } + + override val minRegisteredRatio = + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + 0.8 + } else { + super.minRegisteredRatio + } + + private val executorWatchResource = new AtomicReference[Closeable] + protected var totalExpectedExecutors = new AtomicInteger(0) + + private val driverUrl = RpcEndpointAddress( + conf.get("spark.driver.host"), + conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + + private val initialExecutors = getInitialTargetExecutorNumber() + + private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) + require(podAllocationInterval > 0, s"Allocation batch delay " + + s"$KUBERNETES_ALLOCATION_BATCH_DELAY " + + s"is $podAllocationInterval, should be a positive integer") + + private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) + require(podAllocationSize > 0, s"Allocation batch size " + + s"$KUBERNETES_ALLOCATION_BATCH_SIZE " + + s"is $podAllocationSize, should be a positive integer") + + private val allocatorRunnable = new Runnable { + + // Maintains a map of executor id to count of checks performed to learn the loss reason + // for an executor. + private val executorReasonCheckAttemptCounts = new mutable.HashMap[String, Int] + + override def run(): Unit = { + handleDisconnectedExecutors() + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) { + logDebug("Waiting for pending executors before scaling") + } else if (totalExpectedExecutors.get() <= runningExecutorsToPods.size) { + logDebug("Maximum allowed executor limit reached. Not scaling up further.") + } else { + val nodeToLocalTaskCount = getNodesWithLocalTaskCounts + for (i <- 0 until math.min( + totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { + val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) + runningExecutorsToPods.put(executorId, pod) + runningPodsToExecutors.put(pod.getMetadata.getName, executorId) + logInfo( + s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") + } + } + } + } + + def handleDisconnectedExecutors(): Unit = { + // For each disconnected executor, synchronize with the loss reasons that may have been found + // by the executor pod watcher. If the loss reason was discovered by the watcher, + // inform the parent class with removeExecutor. + disconnectedPodsByExecutorIdPendingRemoval.keys().asScala.foreach { case (executorId) => + val executorPod = disconnectedPodsByExecutorIdPendingRemoval.get(executorId) + val knownExitReason = Option(podsWithKnownExitReasons.remove( + executorPod.getMetadata.getName)) + knownExitReason.fold { + removeExecutorOrIncrementLossReasonCheckCount(executorId) + } { executorExited => + logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) + removeExecutor(executorId, executorExited) + // We keep around executors that have exit conditions caused by the application. This + // allows them to be debugged later on. Otherwise, mark them as to be deleted from the + // the API server. + if (!executorExited.exitCausedByApp) { + deleteExecutorFromClusterAndDataStructures(executorId) + } + } + } + } + + def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = { + val reasonCheckCount = executorReasonCheckAttemptCounts.getOrElse(executorId, 0) + if (reasonCheckCount >= MAX_EXECUTOR_LOST_REASON_CHECKS) { + removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons.")) + deleteExecutorFromClusterAndDataStructures(executorId) + } else { + executorReasonCheckAttemptCounts.put(executorId, reasonCheckCount + 1) + } + } + + def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { + disconnectedPodsByExecutorIdPendingRemoval.remove(executorId) + executorReasonCheckAttemptCounts -= executorId + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.remove(executorId).map { pod => + kubernetesClient.pods().delete(pod) + runningPodsToExecutors.remove(pod.getMetadata.getName) + }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) + } + } + } + + private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { + if (Utils.isDynamicAllocationEnabled(conf)) { + val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) + val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) + val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1) + require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, + s"initial executor number $initialNumExecutors must between min executor number " + + s"$minNumExecutors and max executor number $maxNumExecutors") + + initialNumExecutors + } else { + conf.getInt("spark.executor.instances", defaultNumExecutors) + } + + } + + override def applicationId(): String = conf.get("spark.app.id", super.applicationId()) + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio + } + + override def start(): Unit = { + super.start() + executorWatchResource.set( + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .watch(new ExecutorPodsWatcher())) + + allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) + shuffleManager.foreach(_.start(applicationId())) + + if (!Utils.isDynamicAllocationEnabled(conf)) { + doRequestTotalExecutors(initialExecutors) + } + } + + override def stop(): Unit = { + // stop allocation of new resources and caches. + allocatorExecutor.shutdown() + shuffleManager.foreach(_.stop()) + + // send stop message to executors so they shut down cleanly + super.stop() + + // then delete the executor pods + // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. + // When using Utils.tryLogNonFatalError some of the code fails but without any logs or + // indication as to why. + try { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_)) + runningExecutorsToPods.clear() + runningPodsToExecutors.clear() + } + executorPodsByIPs.clear() + val resource = executorWatchResource.getAndSet(null) + if (resource != null) { + resource.close() + } + } catch { + case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) + } + try { + logInfo("Closing kubernetes client") + kubernetesClient.close() + } catch { + case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) + } + } + + /** + * @return A map of K8s cluster nodes to the number of tasks that could benefit from data + * locality if an executor launches on the cluster node. + */ + private def getNodesWithLocalTaskCounts() : Map[String, Int] = { + val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ + KubernetesClusterSchedulerBackend.this.synchronized { + hostToLocalTaskCount + } + for (pod <- executorPodsByIPs.values().asScala) { + // Remove cluster nodes that are running our executors already. + // TODO: This prefers spreading out executors across nodes. In case users want + // consolidating executors on fewer nodes, introduce a flag. See the spark.deploy.spreadOut + // flag that Spark standalone has: https://spark.apache.org/docs/latest/spark-standalone.html + nodeToLocalTaskCount.remove(pod.getSpec.getNodeName).nonEmpty || + nodeToLocalTaskCount.remove(pod.getStatus.getHostIP).nonEmpty || + nodeToLocalTaskCount.remove( + InetAddress.getByName(pod.getStatus.getHostIP).getCanonicalHostName).nonEmpty + } + nodeToLocalTaskCount.toMap[String, Int] + } + + /** + * Allocates a new executor pod + * + * @param nodeToLocalTaskCount A map of K8s cluster nodes to the number of tasks that could + * benefit from data locality if an executor launches on the cluster + * node. + * @return A tuple of the new executor name and the Pod data structure. + */ + private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val executorPod = executorPodFactory.createExecutorPod( + executorId, + applicationId(), + driverUrl, + conf.getExecutorEnv, + driverPod, + nodeToLocalTaskCount) + try { + (executorId, kubernetesClient.pods.create(executorPod)) + } catch { + case throwable: Throwable => + logError("Failed to allocate executor pod.", throwable) + throw throwable + } + } + + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { + totalExpectedExecutors.set(requestedTotal) + true + } + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + for (executor <- executorIds) { + val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) + maybeRemovedExecutor.foreach { executorPod => + kubernetesClient.pods().delete(executorPod) + disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) + runningPodsToExecutors.remove(executorPod.getMetadata.getName) + } + if (maybeRemovedExecutor.isEmpty) { + logWarning(s"Unable to remove pod for unknown executor $executor") + } + } + } + true + } + + def getExecutorPodByIP(podIP: String): Option[Pod] = { + // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't + // want to be switching to scala.collection.concurrent.Map on + // executorPodsByIPs. + val pod = executorPodsByIPs.get(podIP) + Option(pod) + } + + private class ExecutorPodsWatcher extends Watcher[Pod] { + + private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 + + override def eventReceived(action: Action, pod: Pod): Unit = { + if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" + && pod.getMetadata.getDeletionTimestamp == null) { + val podIP = pod.getStatus.getPodIP + val clusterNodeName = pod.getSpec.getNodeName + logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") + executorPodsByIPs.put(podIP, pod) + } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || + action == Action.DELETED || action == Action.ERROR) { + val podName = pod.getMetadata.getName + val podIP = pod.getStatus.getPodIP + logDebug(s"Executor pod $podName at IP $podIP was at $action.") + if (podIP != null) { + executorPodsByIPs.remove(podIP) + } + if (action == Action.ERROR) { + logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) + handleErroredPod(pod) + } else if (action == Action.DELETED) { + logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) + handleDeletedPod(pod) + } + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Executor pod watch closed.", cause) + } + + def getExecutorExitStatus(pod: Pod): Int = { + val containerStatuses = pod.getStatus.getContainerStatuses + if (!containerStatuses.isEmpty) { + // we assume the first container represents the pod status. This assumption may not hold + // true in the future. Revisit this if side-car containers start running inside executor + // pods. + getExecutorExitStatus(containerStatuses.get(0)) + } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS + } + + def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { + Option(containerStatus.getState).map(containerState => + Option(containerState.getTerminated).map(containerStateTerminated => + containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE) + ).getOrElse(UNKNOWN_EXIT_CODE) + } + + def isPodAlreadyReleased(pod: Pod): Boolean = { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + !runningPodsToExecutors.contains(pod.getMetadata.getName) + } + } + + def handleErroredPod(pod: Pod): Unit = { + val containerExitStatus = getExecutorExitStatus(pod) + // container was probably actively killed by the driver. + val exitReason = if (isPodAlreadyReleased(pod)) { + ExecutorExited(containerExitStatus, exitCausedByApp = false, + s"Container in pod " + pod.getMetadata.getName + + " exited from explicit termination request.") + } else { + val containerExitReason = containerExitStatus match { + case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE => + memLimitExceededLogMessage(pod.getStatus.getReason) + case _ => + // Here we can't be sure that that exit was caused by the application but this seems + // to be the right default since we know the pod was not explicitly deleted by + // the user. + s"Pod ${pod.getMetadata.getName}'s executor container exited with exit status" + + s" code $containerExitStatus." + } + ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) + } + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) + } + + def handleDeletedPod(pod: Pod): Unit = { + val exitMessage = if (isPodAlreadyReleased(pod)) { + s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request." + } else { + s"Pod ${pod.getMetadata.getName} deleted or lost." + } + val exitReason = ExecutorExited( + getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) + } + } + + override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { + new KubernetesDriverEndpoint(rpcEnv, properties) + } + + private class KubernetesDriverEndpoint( + rpcEnv: RpcEnv, + sparkProperties: Seq[(String, String)]) + extends DriverEndpoint(rpcEnv, sparkProperties) { + + override def onDisconnected(rpcAddress: RpcAddress): Unit = { + addressToExecutorId.get(rpcAddress).foreach { executorId => + if (disableExecutor(executorId)) { + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.get(executorId).foreach { pod => + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) + } + } + } + } + } + + override def receiveAndReply( + context: RpcCallContext): PartialFunction[Any, Unit] = { + new PartialFunction[Any, Unit]() { + override def isDefinedAt(msg: Any): Boolean = { + msg match { + case RetrieveSparkAppConfig(_) => + shuffleManager.isDefined + case _ => false + } + } + + override def apply(msg: Any): Unit = { + msg match { + case RetrieveSparkAppConfig(executorId) if shuffleManager.isDefined => + val runningExecutorPod = RUNNING_EXECUTOR_PODS_LOCK.synchronized { + kubernetesClient + .pods() + .withName(runningExecutorsToPods(executorId).getMetadata.getName) + .get() + } + val shuffleSpecificProperties = shuffleManager.get + .getShuffleServiceConfigurationForExecutor(runningExecutorPod) + val reply = SparkAppConfig( + sparkProperties ++ shuffleSpecificProperties, + SparkEnv.get.securityManager.getIOEncryptionKey(), + hadoopDelegationCreds) + context.reply(reply) + } + } + }.orElse(super.receiveAndReply(context)) + } + } +} + +private object KubernetesClusterSchedulerBackend { + private val VMEM_EXCEEDED_EXIT_CODE = -103 + private val PMEM_EXCEEDED_EXIT_CODE = -104 + private val UNKNOWN_EXIT_CODE = -111 + // Number of times we are allowed check for the loss reason for an executor before we give up + // and assume the executor failed for good, and attribute it to a framework fault. + val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 + + def memLimitExceededLogMessage(diagnostics: String): String = { + s"Pod/Container killed for exceeding memory limits. $diagnostics" + + " Consider boosting spark executor memory overhead." + } +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala new file mode 100644 index 0000000000000..388e2b17f4fdd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala @@ -0,0 +1,162 @@ +/* + * 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.scheduler.cluster.k8s + +import io.fabric8.kubernetes.api.model.{Pod, Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.internal.readiness.Readiness +import org.apache.commons.io.FilenameUtils +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.internal.Logging +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient +import org.apache.spark.util.Utils + +private[spark] trait KubernetesExternalShuffleManager { + + def start(appId: String): Unit + + def stop(): Unit + + /** + * Returns the properties that should be applied for this executor pod, given that + * this executor will need to communicate with an external shuffle service. + * + * In practice, this seq will always have a size of 1, but since this method's semantics are that + * the returned values are key-value pairs to apply as properties, it is clearer to express + * this as a collection. + */ + def getShuffleServiceConfigurationForExecutor(executorPod: Pod): Seq[(String, String)] + + def getExecutorShuffleDirVolumesWithMounts: Seq[(Volume, VolumeMount)] + +} + +private[spark] class KubernetesExternalShuffleManagerImpl( + sparkConf: SparkConf, + client: KubernetesClient, + shuffleClient: KubernetesExternalShuffleClient) + extends KubernetesExternalShuffleManager with Logging { + + private val shuffleNamespace = sparkConf.get(KUBERNETES_SHUFFLE_NAMESPACE) + private val shufflePodLabels = ConfigurationUtils.parseKeyValuePairs( + sparkConf.get(KUBERNETES_SHUFFLE_LABELS), + KUBERNETES_SHUFFLE_LABELS.key, + "shuffle-labels") + if (shufflePodLabels.isEmpty) { + throw new SparkException(s"Dynamic allocation enabled " + + s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified") + } + private val externalShufflePort = sparkConf.getInt("spark.shuffle.service.port", 7337) + private val shuffleDirs = Utils.getConfiguredLocalDirs(sparkConf) + private var shufflePodCache = scala.collection.mutable.Map[String, String]() + private var watcher: Watch = _ + + override def start(appId: String): Unit = { + // seed the initial cache. + val pods = client.pods() + .inNamespace(shuffleNamespace) + .withLabels(shufflePodLabels.asJava) + .list() + pods.getItems.asScala.foreach { + pod => + if (Readiness.isReady(pod)) { + addShufflePodToCache(pod) + } else { + logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + + s"on node ${pod.getSpec.getNodeName}") + } + } + + watcher = client + .pods() + .inNamespace(shuffleNamespace) + .withLabels(shufflePodLabels.asJava) + .watch(new Watcher[Pod] { + override def eventReceived(action: Watcher.Action, p: Pod): Unit = { + action match { + case Action.DELETED | Action.ERROR => + shufflePodCache.remove(p.getSpec.getNodeName) + case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => + addShufflePodToCache(p) + } + } + override def onClose(e: KubernetesClientException): Unit = {} + }) + shuffleClient.init(appId) + } + + private def addShufflePodToCache(pod: Pod): Unit = shufflePodCache.synchronized { + if (shufflePodCache.contains(pod.getSpec.getNodeName)) { + val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get + logError(s"Ambiguous specification of shuffle service pod. " + + s"Found multiple matching pods: ${pod.getMetadata.getName}, " + + s"${registeredPodName} on ${pod.getSpec.getNodeName}") + + throw new SparkException(s"Ambiguous specification of shuffle service pod. " + + s"Found multiple matching pods: ${pod.getMetadata.getName}, " + + s"${registeredPodName} on ${pod.getSpec.getNodeName}") + } else { + shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP + } + } + + override def stop(): Unit = { + watcher.close() + shuffleClient.close() + } + + override def getShuffleServiceConfigurationForExecutor(executorPod: Pod) + : Seq[(String, String)] = { + val nodeName = executorPod.getSpec.getNodeName + val shufflePodIp = shufflePodCache.synchronized { + shufflePodCache.get(nodeName).getOrElse( + throw new SparkException(s"Unable to find shuffle pod on node $nodeName")) + } + // Inform the shuffle pod about this application so it can watch. + shuffleClient.registerDriverWithShuffleService(shufflePodIp, externalShufflePort) + Seq((SPARK_SHUFFLE_SERVICE_HOST.key, shufflePodIp)) + } + + override def getExecutorShuffleDirVolumesWithMounts(): Seq[(Volume, VolumeMount)] = { + // TODO: Using hostPath for the local directory will also make it such that the + // other uses of the local directory - broadcasting and caching - will also write + // to the directory that the shuffle service is aware of. It would be better for + // these directories to be separate so that the lifetime of the non-shuffle scratch + // space is tied to an emptyDir instead of the hostPath. This requires a change in + // core Spark as well. + shuffleDirs.zipWithIndex.map { + case (shuffleDir, shuffleDirIndex) => + val volumeName = s"$shuffleDirIndex-${FilenameUtils.getBaseName(shuffleDir)}" + val volume = new VolumeBuilder() + .withName(volumeName) + .withNewHostPath(shuffleDir) + .build() + val volumeMount = new VolumeMountBuilder() + .withName(volumeName) + .withMountPath(shuffleDir) + .build() + (volume, volumeMount) + } + } +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala index 20136679e9b84..ec56d662195f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, TaskSet, TaskSetManager} import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala index 44d01a5d76b01..39c521978c5b1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} private[spark] class KubernetesTaskSetManager( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala new file mode 100644 index 0000000000000..a3573854835b4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala @@ -0,0 +1,85 @@ +/* + * 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.scheduler.cluster.k8s + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} + +import org.apache.spark.deploy.k8s.constants.ANNOTATION_EXECUTOR_NODE_AFFINITY +import org.apache.spark.internal.Logging + +// Applies a node affinity annotation to executor pods so that pods can be placed optimally for +// locality. +private[spark] trait NodeAffinityExecutorPodModifier { + def addNodeAffinityAnnotationIfUseful( + baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod +} + +private[spark] object NodeAffinityExecutorPodModifierImpl + extends NodeAffinityExecutorPodModifier with Logging { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) + + private def scaleToRange( + value: Int, + baseMin: Double, + baseMax: Double, + rangeMin: Double, + rangeMax: Double): Int = { + (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt + } + override def addNodeAffinityAnnotationIfUseful( + baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { + if (nodeToTaskCount.nonEmpty) { + val taskTotal = nodeToTaskCount.foldLeft(0)(_ + _._2) + // Normalize to node affinity weights in 1 to 100 range. + val nodeToWeight = nodeToTaskCount.map { + case (node, taskCount) => + (node, scaleToRange(taskCount, 1, taskTotal, rangeMin = 1, rangeMax = 100)) + } + val weightToNodes = nodeToWeight.groupBy(_._2).mapValues(_.keys) + // @see https://kubernetes.io/docs/concepts/configuration/assign-pod-node + val nodeAffinityJson = OBJECT_MAPPER.writeValueAsString(SchedulerAffinity(NodeAffinity( + preferredDuringSchedulingIgnoredDuringExecution = + for ((weight, nodes) <- weightToNodes) yield { + WeightedPreference( + weight, + Preference(Array(MatchExpression("kubernetes.io/hostname", "In", nodes)))) + }))) + // TODO: Use non-annotation syntax when we switch to K8s version 1.6. + logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") + new PodBuilder(baseExecutorPod) + .editMetadata() + .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) + .endMetadata() + .build() + } else { + baseExecutorPod + } + } +} + +// These case classes model K8s node affinity syntax fo +// preferredDuringSchedulingIgnoredDuringExecution. +// see https://kubernetes.io/docs/concepts/configuration/assign-pod-node +private case class SchedulerAffinity(nodeAffinity: NodeAffinity) +private case class NodeAffinity( + preferredDuringSchedulingIgnoredDuringExecution: Iterable[WeightedPreference]) +private case class WeightedPreference(weight: Int, preference: Preference) +private case class Preference(matchExpressions: Array[MatchExpression]) +private case class MatchExpression(key: String, operator: String, values: Iterable[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala index 29a7dc982a5af..0b45aa0c117cc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic @@ -25,14 +25,14 @@ import org.apache.log4j.{Level, Logger} /** * Finds rack names that cluster nodes belong to in order to support HDFS rack locality. */ -private[kubernetes] trait RackResolverUtil { +private[k8s] trait RackResolverUtil { def isConfigured() : Boolean def resolveRack(hadoopConfiguration: Configuration, host: String): Option[String] } -private[kubernetes] class RackResolverUtilImpl(hadoopConfiguration: Configuration) +private[k8s] class RackResolverUtilImpl(hadoopConfiguration: Configuration) extends RackResolverUtil { val scriptPlugin : String = classOf[ScriptBasedMapping].getCanonicalName diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala new file mode 100644 index 0000000000000..789e5f336edc8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala @@ -0,0 +1,22 @@ +/* + * 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.scheduler.cluster.k8s + +private[spark] case class ShuffleServiceConfig( + shuffleNamespace: String, + shuffleLabels: Map[String, String], + shuffleDirs: Seq[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala deleted file mode 100644 index 49cac4d81a0b1..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ /dev/null @@ -1,818 +0,0 @@ -/* - * 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.scheduler.cluster.kubernetes - -import java.io.Closeable -import java.net.InetAddress -import java.util.Collections -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} -import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} - -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} -import io.fabric8.kubernetes.client.Watcher.Action -import org.apache.commons.io.FilenameUtils -import scala.collection.{concurrent, mutable} -import scala.collection.JavaConverters._ -import scala.concurrent.{ExecutionContext, Future} - -import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} -import org.apache.spark.internal.config -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient -import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} -import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.{ThreadUtils, Utils} - -private[spark] class KubernetesClusterSchedulerBackend( - scheduler: TaskSchedulerImpl, - val sc: SparkContext, - executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], - executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], - mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], - kubernetesClient: KubernetesClient) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - - import KubernetesClusterSchedulerBackend._ - - private val RUNNING_EXECUTOR_PODS_LOCK = new Object - // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK. - private val runningExecutorsToPods = new mutable.HashMap[String, Pod] - // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK. - private val runningPodsToExecutors = new mutable.HashMap[String, String] - // TODO(varun): Get rid of this lock object by my making the underlying map a concurrent hash map. - private val EXECUTOR_PODS_BY_IPS_LOCK = new Object - // Indexed by executor IP addrs and guarded by EXECUTOR_PODS_BY_IPS_LOCK - private val executorPodsByIPs = new mutable.HashMap[String, Pod] - private val failedPods: concurrent.Map[String, ExecutorExited] = new - ConcurrentHashMap[String, ExecutorExited]().asScala - private val executorsToRemove = Collections.newSetFromMap[String]( - new ConcurrentHashMap[String, java.lang.Boolean]()).asScala - - private val executorExtraClasspath = conf.get( - org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) - private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - - private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - conf, - KUBERNETES_EXECUTOR_LABEL_PREFIX, - KUBERNETES_EXECUTOR_LABELS, - "executor label") - require( - !executorLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is" + - s" reserved for Spark.") - require( - !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - s" Spark.") - - private val executorAnnotations = - ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - conf, - KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, - KUBERNETES_EXECUTOR_ANNOTATIONS, - "executor annotation") - private val nodeSelector = - ConfigurationUtils.parsePrefixedKeyValuePairs( - conf, - KUBERNETES_NODE_SELECTOR_PREFIX, - "node-selector") - private var shufflePodCache: Option[ShufflePodCache] = None - private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) - private val dockerImagePullPolicy = conf.get(DOCKER_IMAGE_PULL_POLICY) - private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) - private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) - private val blockmanagerPort = conf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - - private val kubernetesDriverPodName = conf - .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse( - throw new SparkException("Must specify the driver pod name")) - private val executorPodNamePrefix = conf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - - private val executorMemoryMiB = conf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) - private val executorMemoryString = conf.get( - org.apache.spark.internal.config.EXECUTOR_MEMORY.key, - org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) - - private val memoryOverheadMiB = conf - .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) - private val executorMemoryWithOverheadMiB = executorMemoryMiB + memoryOverheadMiB - - private val executorCores = conf.getDouble("spark.executor.cores", 1d) - private val executorLimitCores = conf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) - - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) - - private val driverPod = try { - kubernetesClient.pods().inNamespace(kubernetesNamespace). - withName(kubernetesDriverPodName).get() - } catch { - case throwable: Throwable => - logError(s"Executor cannot find driver pod.", throwable) - throw new SparkException(s"Executor cannot find driver pod", throwable) - } - - private val shuffleServiceConfig: Option[ShuffleServiceConfig] = - if (Utils.isDynamicAllocationEnabled(sc.conf)) { - val shuffleNamespace = conf.get(KUBERNETES_SHUFFLE_NAMESPACE) - val parsedShuffleLabels = ConfigurationUtils.parseKeyValuePairs( - conf.get(KUBERNETES_SHUFFLE_LABELS), KUBERNETES_SHUFFLE_LABELS.key, - "shuffle-labels") - if (parsedShuffleLabels.isEmpty) { - throw new SparkException(s"Dynamic allocation enabled " + - s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified") - } - - val shuffleDirs = conf.get(KUBERNETES_SHUFFLE_DIR).map { - _.split(",") - }.getOrElse(Utils.getConfiguredLocalDirs(conf)) - Some( - ShuffleServiceConfig(shuffleNamespace, - parsedShuffleLabels, - shuffleDirs)) - } else { - None - } - - // A client for talking to the external shuffle service - private val kubernetesExternalShuffleClient: Option[KubernetesExternalShuffleClient] = { - if (Utils.isDynamicAllocationEnabled(sc.conf)) { - Some(getShuffleClient()) - } else { - None - } - } - - override val minRegisteredRatio = - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - 0.8 - } else { - super.minRegisteredRatio - } - - private val executorWatchResource = new AtomicReference[Closeable] - protected var totalExpectedExecutors = new AtomicInteger(0) - - - private val driverUrl = RpcEndpointAddress( - sc.getConf.get("spark.driver.host"), - sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString - - private val initialExecutors = getInitialTargetExecutorNumber() - - private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) - require(podAllocationInterval > 0, s"Allocation batch delay " + - s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " + - s"is ${podAllocationInterval}, should be a positive integer") - - private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) - require(podAllocationSize > 0, s"Allocation batch size " + - s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + - s"is ${podAllocationSize}, should be a positive integer") - - private val allocator = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") - - private val allocatorRunnable: Runnable = new Runnable { - - // Number of times we are allowed check for the loss reason for an executor before we give up - // and assume the executor failed for good, and attribute it to a framework fault. - private val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 - private val executorsToRecover = new mutable.HashSet[String] - // Maintains a map of executor id to count of checks performed to learn the loss reason - // for an executor. - private val executorReasonChecks = new mutable.HashMap[String, Int] - - override def run(): Unit = { - removeFailedExecutors() - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) { - logDebug("Waiting for pending executors before scaling") - } else if (totalExpectedExecutors.get() <= runningExecutorsToPods.size) { - logDebug("Maximum allowed executor limit reached. Not scaling up further.") - } else { - val nodeToLocalTaskCount = getNodesWithLocalTaskCounts - for (i <- 0 until math.min( - totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { - val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) - runningExecutorsToPods.put(executorId, pod) - runningPodsToExecutors.put(pod.getMetadata.getName, executorId) - logInfo( - s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}") - } - } - } - } - - def removeFailedExecutors(): Unit = { - val localRunningExecutorsToPods = RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.toMap - } - executorsToRemove.foreach { case (executorId) => - localRunningExecutorsToPods.get(executorId).map { pod: Pod => - failedPods.get(pod.getMetadata.getName).map { executorExited: ExecutorExited => - logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) - removeExecutor(executorId, executorExited) - if (!executorExited.exitCausedByApp) { - executorsToRecover.add(executorId) - } - }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId)) - }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId)) - - executorsToRecover.foreach(executorId => { - executorsToRemove -= executorId - executorReasonChecks -= executorId - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.remove(executorId).map { pod: Pod => - kubernetesClient.pods().delete(pod) - runningPodsToExecutors.remove(pod.getMetadata.getName) - }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) - } - }) - executorsToRecover.clear() - } - } - - def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = { - val reasonCheckCount = executorReasonChecks.getOrElse(executorId, 0) - if (reasonCheckCount > MAX_EXECUTOR_LOST_REASON_CHECKS) { - removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons")) - executorsToRecover.add(executorId) - executorReasonChecks -= executorId - } else { - executorReasonChecks.put(executorId, reasonCheckCount + 1) - } - } - } - - private val objectMapper = new ObjectMapper().registerModule(DefaultScalaModule) - - private def getShuffleClient(): KubernetesExternalShuffleClient = { - new KubernetesExternalShuffleClient( - SparkTransportConf.fromSparkConf(conf, "shuffle"), - sc.env.securityManager, - sc.env.securityManager.isAuthenticationEnabled(), - conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) - } - - private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { - if (Utils.isDynamicAllocationEnabled(conf)) { - val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) - val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) - val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1) - require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors, - s"initial executor number $initialNumExecutors must between min executor number " + - s"$minNumExecutors and max executor number $maxNumExecutors") - - initialNumExecutors - } else { - conf.getInt("spark.executor.instances", defaultNumExecutors) - } - - } - - override def applicationId(): String = conf.get("spark.app.id", super.applicationId()) - - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio - } - - override def start(): Unit = { - super.start() - executorWatchResource.set( - kubernetesClient - .pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId()) - .watch(new ExecutorPodsWatcher())) - - allocator.scheduleWithFixedDelay( - allocatorRunnable, 0, podAllocationInterval, TimeUnit.SECONDS) - - if (!Utils.isDynamicAllocationEnabled(sc.conf)) { - doRequestTotalExecutors(initialExecutors) - } else { - shufflePodCache = shuffleServiceConfig - .map { config => new ShufflePodCache( - kubernetesClient, config.shuffleNamespace, config.shuffleLabels) } - shufflePodCache.foreach(_.start()) - kubernetesExternalShuffleClient.foreach(_.init(applicationId())) - } - } - - override def stop(): Unit = { - // stop allocation of new resources and caches. - allocator.shutdown() - shufflePodCache.foreach(_.stop()) - kubernetesExternalShuffleClient.foreach(_.close()) - - // send stop message to executors so they shut down cleanly - super.stop() - - // then delete the executor pods - // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context. - // When using Utils.tryLogNonFatalError some of the code fails but without any logs or - // indication as to why. - try { - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_)) - runningExecutorsToPods.clear() - runningPodsToExecutors.clear() - } - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.clear() - } - val resource = executorWatchResource.getAndSet(null) - if (resource != null) { - resource.close() - } - } catch { - case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) - } - try { - logInfo("Closing kubernetes client") - kubernetesClient.close() - } catch { - case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) - } - } - - /** - * @return A map of K8s cluster nodes to the number of tasks that could benefit from data - * locality if an executor launches on the cluster node. - */ - private def getNodesWithLocalTaskCounts() : Map[String, Int] = { - val executorPodsWithIPs = EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.values.toList // toList makes a defensive copy. - } - val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ - KubernetesClusterSchedulerBackend.this.synchronized { - hostToLocalTaskCount - } - for (pod <- executorPodsWithIPs) { - // Remove cluster nodes that are running our executors already. - // TODO: This prefers spreading out executors across nodes. In case users want - // consolidating executors on fewer nodes, introduce a flag. See the spark.deploy.spreadOut - // flag that Spark standalone has: https://spark.apache.org/docs/latest/spark-standalone.html - nodeToLocalTaskCount.remove(pod.getSpec.getNodeName).nonEmpty || - nodeToLocalTaskCount.remove(pod.getStatus.getHostIP).nonEmpty || - nodeToLocalTaskCount.remove( - InetAddress.getByName(pod.getStatus.getHostIP).getCanonicalHostName).nonEmpty - } - nodeToLocalTaskCount.toMap[String, Int] - } - - private def addNodeAffinityAnnotationIfUseful( - baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { - def scaleToRange(value: Int, baseMin: Double, baseMax: Double, - rangeMin: Double, rangeMax: Double): Int = - (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt - - if (nodeToTaskCount.nonEmpty) { - val taskTotal = nodeToTaskCount.foldLeft(0)(_ + _._2) - // Normalize to node affinity weights in 1 to 100 range. - val nodeToWeight = nodeToTaskCount.map{ - case (node, taskCount) => - (node, scaleToRange(taskCount, 1, taskTotal, rangeMin = 1, rangeMax = 100))} - val weightToNodes = nodeToWeight.groupBy(_._2).mapValues(_.keys) - // see https://kubernetes.io/docs/concepts/configuration/assign-pod-node - val nodeAffinityJson = objectMapper.writeValueAsString(SchedulerAffinity(NodeAffinity( - preferredDuringSchedulingIgnoredDuringExecution = - for ((weight, nodes) <- weightToNodes) yield - WeightedPreference(weight, - Preference(Array(MatchExpression("kubernetes.io/hostname", "In", nodes)))) - ))) - // TODO: Use non-annotation syntax when we switch to K8s version 1.6. - logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") - new PodBuilder(baseExecutorPod).editMetadata() - .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) - .endMetadata() - .build() - } else { - baseExecutorPod - } - } - - /** - * Allocates a new executor pod - * - * @param nodeToLocalTaskCount A map of K8s cluster nodes to the number of tasks that could - * benefit from data locality if an executor launches on the cluster - * node. - * @return A tuple of the new executor name and the Pod data structure. - */ - private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { - val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val name = s"$executorPodNamePrefix-exec-$executorId" - - // hostname must be no longer than 63 characters, so take the last 63 characters of the pod - // name as the hostname. This preserves uniqueness since the end of name contains - // executorId and applicationId - val hostname = name.substring(Math.max(0, name.length - 63)) - val resolvedExecutorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> applicationId(), - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorLabels - val executorMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryMiB}Mi") - .build() - val executorMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryWithOverheadMiB}Mi") - .build() - val executorCpuQuantity = new QuantityBuilder(false) - .withAmount(executorCores.toString) - .build() - val executorExtraClasspathEnv = executorExtraClasspath.map { cp => - new EnvVarBuilder() - .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) - .withValue(cp) - .build() - } - val executorExtraJavaOptionsEnv = conf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() - } - }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = (Seq( - (ENV_EXECUTOR_PORT, executorPort.toString), - (ENV_DRIVER_URL, driverUrl), - // Executor backend expects integral value for executor cores, so round it up to an int. - (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), - (ENV_EXECUTOR_MEMORY, executorMemoryString), - (ENV_APPLICATION_ID, applicationId()), - (ENV_EXECUTOR_ID, executorId), - (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ sc.executorEnvs.toSeq) - .map(env => new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() - ) ++ Seq( - new EnvVarBuilder() - .withName(ENV_EXECUTOR_POD_IP) - .withValueFrom(new EnvVarSourceBuilder() - .withNewFieldRef("v1", "status.podIP") - .build()) - .build() - ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq - val requiredPorts = Seq( - (EXECUTOR_PORT_NAME, executorPort), - (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) - .map(port => { - new ContainerPortBuilder() - .withName(port._1) - .withContainerPort(port._2) - .build() - }) - - val executorContainer = new ContainerBuilder() - .withName(s"executor") - .withImage(executorDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .withNewResources() - .addToRequests("memory", executorMemoryQuantity) - .addToLimits("memory", executorMemoryLimitQuantity) - .addToRequests("cpu", executorCpuQuantity) - .endResources() - .addAllToEnv(executorEnv.asJava) - .withPorts(requiredPorts.asJava) - .build() - - val executorPod = new PodBuilder() - .withNewMetadata() - .withName(name) - .withLabels(resolvedExecutorLabels.asJava) - .withAnnotations(executorAnnotations.asJava) - .withOwnerReferences() - .addNewOwnerReference() - .withController(true) - .withApiVersion(driverPod.getApiVersion) - .withKind(driverPod.getKind) - .withName(driverPod.getMetadata.getName) - .withUid(driverPod.getMetadata.getUid) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .withHostname(hostname) - .withRestartPolicy("Never") - .withNodeSelector(nodeSelector.asJava) - .endSpec() - .build() - - val containerWithExecutorLimitCores = executorLimitCores.map { - limitCores => - val executorCpuLimitQuantity = new QuantityBuilder(false) - .withAmount(limitCores) - .build() - new ContainerBuilder(executorContainer) - .editResources() - .addToLimits("cpu", executorCpuLimitQuantity) - .endResources() - .build() - }.getOrElse(executorContainer) - - val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => - new ContainerBuilder(container) - .addNewVolumeMount() - .withName(FilenameUtils.getBaseName(dir)) - .withMountPath(dir) - .endVolumeMount() - .build() - } - }.getOrElse(containerWithExecutorLimitCores) - val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => - new PodBuilder(builder) - .editSpec() - .addNewVolume() - .withName(FilenameUtils.getBaseName(dir)) - .withNewHostPath() - .withPath(dir) - .endHostPath() - .endVolume() - .endSpec() - .build() - } - }.getOrElse(executorPod) - val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = - mountSmallFilesBootstrap.map { bootstrap => - bootstrap.mountSmallFilesSecret( - withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) - }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) - val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = - executorInitContainerBootstrap.map { bootstrap => - val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( - PodWithDetachedInitContainer( - withMaybeSmallFilesMountedPod, - new ContainerBuilder().build(), - withMaybeSmallFilesMountedContainer)) - - val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => - plugin.mountResourceStagingServerSecretIntoInitContainer( - podWithDetachedInitContainer.initContainer) - }.getOrElse(podWithDetachedInitContainer.initContainer) - - val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( - podWithDetachedInitContainer.pod, resolvedInitContainer) - - val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => - plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) - }.getOrElse(podWithAttachedInitContainer) - - (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) - }.getOrElse((withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer)) - - val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( - executorPodWithInitContainer, nodeToLocalTaskCount) - val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) - .editSpec() - .addToContainers(initBootstrappedExecutorContainer) - .endSpec() - .build() - try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) - } catch { - case throwable: Throwable => - logError("Failed to allocate executor pod.", throwable) - throw throwable - } - } - - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { - totalExpectedExecutors.set(requestedTotal) - true - } - - override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - for (executor <- executorIds) { - runningExecutorsToPods.remove(executor) match { - case Some(pod) => - kubernetesClient.pods().delete(pod) - runningPodsToExecutors.remove(pod.getMetadata.getName) - case None => logWarning(s"Unable to remove pod for unknown executor $executor") - } - } - } - true - } - - def getExecutorPodByIP(podIP: String): Option[Pod] = { - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.get(podIP) - } - } - - private class ExecutorPodsWatcher extends Watcher[Pod] { - - private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1 - - override def eventReceived(action: Action, pod: Pod): Unit = { - if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running" - && pod.getMetadata.getDeletionTimestamp == null) { - val podIP = pod.getStatus.getPodIP - val clusterNodeName = pod.getSpec.getNodeName - logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs += ((podIP, pod)) - } - } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || - action == Action.DELETED || action == Action.ERROR) { - val podName = pod.getMetadata.getName - val podIP = pod.getStatus.getPodIP - logDebug(s"Executor pod $podName at IP $podIP was at $action.") - if (podIP != null) { - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs -= podIP - } - } - if (action == Action.ERROR) { - logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) - handleErroredPod(pod) - } else if (action == Action.DELETED) { - logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason) - handleDeletedPod(pod) - } - } - } - - override def onClose(cause: KubernetesClientException): Unit = { - logDebug("Executor pod watch closed.", cause) - } - - def getExecutorExitStatus(pod: Pod): Int = { - val containerStatuses = pod.getStatus.getContainerStatuses - if (!containerStatuses.isEmpty) { - // we assume the first container represents the pod status. This assumption may not hold - // true in the future. Revisit this if side-car containers start running inside executor - // pods. - getExecutorExitStatus(containerStatuses.get(0)) - } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS - } - - def getExecutorExitStatus(containerStatus: ContainerStatus): Int = { - Option(containerStatus.getState).map(containerState => - Option(containerState.getTerminated).map(containerStateTerminated => - containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE) - ).getOrElse(UNKNOWN_EXIT_CODE) - } - - def isPodAlreadyReleased(pod: Pod): Boolean = { - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - !runningPodsToExecutors.contains(pod.getMetadata.getName) - } - } - - def handleErroredPod(pod: Pod): Unit = { - val alreadyReleased = isPodAlreadyReleased(pod) - val containerExitStatus = getExecutorExitStatus(pod) - // container was probably actively killed by the driver. - val exitReason = if (alreadyReleased) { - ExecutorExited(containerExitStatus, exitCausedByApp = false, - s"Container in pod " + pod.getMetadata.getName + - " exited from explicit termination request.") - } else { - val containerExitReason = containerExitStatus match { - case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE => - memLimitExceededLogMessage(pod.getStatus.getReason) - case _ => - // Here we can't be sure that that exit was caused by the application but this seems - // to be the right default since we know the pod was not explicitly deleted by - // the user. - "Pod exited with following container exit status code " + containerExitStatus - } - ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) - } - failedPods.put(pod.getMetadata.getName, exitReason) - } - - def handleDeletedPod(pod: Pod): Unit = { - val exitReason = ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, - "Pod " + pod.getMetadata.getName + " deleted or lost.") - failedPods.put(pod.getMetadata.getName, exitReason) - } - } - - override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new KubernetesDriverEndpoint(rpcEnv, properties) - } - - private class KubernetesDriverEndpoint( - rpcEnv: RpcEnv, - sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { - private val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337) - - override def onDisconnected(rpcAddress: RpcAddress): Unit = { - addressToExecutorId.get(rpcAddress).foreach { executorId => - if (disableExecutor(executorId)) { - executorsToRemove.add(executorId) - } - } - } - - override def receiveAndReply( - context: RpcCallContext): PartialFunction[Any, Unit] = { - new PartialFunction[Any, Unit]() { - override def isDefinedAt(msg: Any): Boolean = { - msg match { - case RetrieveSparkAppConfig(_) => - Utils.isDynamicAllocationEnabled(sc.conf) - case _ => false - } - } - - override def apply(msg: Any): Unit = { - msg match { - case RetrieveSparkAppConfig(executorId) => - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - var resolvedProperties = sparkProperties - val runningExecutorPod = kubernetesClient - .pods() - .withName(runningExecutorsToPods(executorId).getMetadata.getName) - .get() - val nodeName = runningExecutorPod.getSpec.getNodeName - val shufflePodIp = shufflePodCache.get.getShufflePodForExecutor(nodeName) - - // Inform the shuffle pod about this application so it can watch. - kubernetesExternalShuffleClient.foreach( - _.registerDriverWithShuffleService(shufflePodIp, externalShufflePort)) - - resolvedProperties = resolvedProperties ++ Seq( - (SPARK_SHUFFLE_SERVICE_HOST.key, shufflePodIp)) - - val reply = SparkAppConfig( - resolvedProperties, - SparkEnv.get.securityManager.getIOEncryptionKey(), - hadoopDelegationCreds) - context.reply(reply) - } - } - } - }.orElse(super.receiveAndReply(context)) - } - } -} -case class ShuffleServiceConfig( - shuffleNamespace: String, - shuffleLabels: Map[String, String], - shuffleDirs: Seq[String]) - -private object KubernetesClusterSchedulerBackend { - private val DEFAULT_STATIC_PORT = 10000 - private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) - private val VMEM_EXCEEDED_EXIT_CODE = -103 - private val PMEM_EXCEEDED_EXIT_CODE = -104 - private val UNKNOWN_EXIT_CODE = -111 - - def memLimitExceededLogMessage(diagnostics: String): String = { - s"Pod/Container killed for exceeding memory limits. $diagnostics" + - " Consider boosting spark executor memory overhead." - } -} - -/** - * These case classes model K8s node affinity syntax for - * preferredDuringSchedulingIgnoredDuringExecution. - * - * see https://kubernetes.io/docs/concepts/configuration/assign-pod-node - */ -case class SchedulerAffinity(nodeAffinity: NodeAffinity) -case class NodeAffinity(preferredDuringSchedulingIgnoredDuringExecution: - Iterable[WeightedPreference]) -case class WeightedPreference(weight: Int, preference: Preference) -case class Preference(matchExpressions: Array[MatchExpression]) -case class MatchExpression(key: String, operator: String, values: Iterable[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala deleted file mode 100644 index 15e02664589eb..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * 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.scheduler.cluster.kubernetes - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.Pod -import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} -import io.fabric8.kubernetes.client.Watcher.Action -import io.fabric8.kubernetes.client.internal.readiness.Readiness - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging - -private[spark] class ShufflePodCache ( - client: KubernetesClient, - dsNamespace: String, - dsLabels: Map[String, String]) extends Logging { - - private var shufflePodCache = scala.collection.mutable.Map[String, String]() - private var watcher: Watch = _ - - def start(): Unit = { - // seed the initial cache. - val pods = client.pods() - .inNamespace(dsNamespace).withLabels(dsLabels.asJava).list() - pods.getItems.asScala.foreach { - pod => - if (Readiness.isReady(pod)) { - addShufflePodToCache(pod) - } else { - logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + - s"on node ${pod.getSpec.getNodeName}") - } - } - - watcher = client - .pods() - .inNamespace(dsNamespace) - .withLabels(dsLabels.asJava) - .watch(new Watcher[Pod] { - override def eventReceived(action: Watcher.Action, p: Pod): Unit = { - action match { - case Action.DELETED | Action.ERROR => - shufflePodCache.remove(p.getSpec.getNodeName) - case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => - addShufflePodToCache(p) - } - } - override def onClose(e: KubernetesClientException): Unit = {} - }) - } - - private def addShufflePodToCache(pod: Pod): Unit = { - if (shufflePodCache.contains(pod.getSpec.getNodeName)) { - val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get - logError(s"Ambiguous specification of shuffle service pod. " + - s"Found multiple matching pods: ${pod.getMetadata.getName}, " + - s"${registeredPodName} on ${pod.getSpec.getNodeName}") - - throw new SparkException(s"Ambiguous specification of shuffle service pod. " + - s"Found multiple matching pods: ${pod.getMetadata.getName}, " + - s"${registeredPodName} on ${pod.getSpec.getNodeName}") - } else { - shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP - } - } - - def stop(): Unit = { - watcher.close() - } - - def getShufflePodForExecutor(executorNode: String): String = { - shufflePodCache.get(executorNode) - .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) - } -} - diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala index 5ded567c84287..b4e9ee104fa52 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ private val INIT_CONTAINER_SECRET_NAME = "init-secret" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala index 8de0f56f007dc..0a41cd5d41d0f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileOutputStream, OutputStreamWriter} import java.math.BigInteger @@ -30,7 +30,7 @@ import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3Certi import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder -import org.apache.spark.deploy.kubernetes.submit.{KeyAndCertPem, KeyStoreAndTrustStore} +import org.apache.spark.deploy.k8s.submit.{KeyAndCertPem, KeyStoreAndTrustStore} import org.apache.spark.util.Utils private[spark] object SSLUtils { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala index 86420e5a9edfd..78abea5754e56 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { private val INIT_CONTAINER_IMAGE = "spark-init:latest" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 0100dce454a3f..041f51e912002 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import com.google.common.collect.Iterables import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} @@ -29,8 +29,8 @@ import org.scalatest.mock.MockitoSugar._ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} class ClientSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala similarity index 70% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index c168e7b5407ba..1199f033cf06a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, LocalDirectoryMountConfigurationStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -29,6 +29,9 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" private val APP_ARGS = Array("arg1", "arg2") private val ADDITIONAL_PYTHON_FILES = Seq("local:///var/apps/python/py1.py") + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" test("Base submission steps without an init-container or python files.") { val sparkConf = new SparkConf(false) @@ -47,8 +50,10 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], - classOf[DependencyResolutionStep]) + classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep]) } test("Submission steps with an init-container.") { @@ -69,8 +74,10 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[InitContainerBootstrapStep]) } @@ -90,8 +97,10 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[PythonStep]) } @@ -111,11 +120,38 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[MountSmallLocalFilesStep]) } + test("Submission steps with driver secrets to mount") { + val sparkConf = new SparkConf(false) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], + classOf[MountSecretsStep]) + } + private def validateStepTypes( orchestrator: DriverConfigurationStepsOrchestrator, types: Class[_ <: DriverConfigurationStep]*): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala similarity index 82% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala index aea7c6798b5c4..2a568c95d21b3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala @@ -14,14 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.internal.config +import org.apache.spark.deploy.k8s.KubernetesExternalShuffleService +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite { @@ -33,11 +32,11 @@ private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite SPARK_CONF, new SecurityManager(SPARK_CONF)) - val shuffleClient = new KubernetesExternalShuffleClient( + val shuffleClient = new KubernetesExternalShuffleClientImpl( SparkTransportConf.fromSparkConf(SPARK_CONF, "shuffle"), new SecurityManager(SPARK_CONF), false, - SPARK_CONF.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) + SPARK_CONF.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) shuffleService.start() shuffleClient.init("newapp") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala new file mode 100644 index 0000000000000..0921228c795c0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala @@ -0,0 +1,46 @@ +/* + * 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.submit + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkFunSuite + +private[spark] class MountSecretsBootstrapSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("Mounts all given secrets") { + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val driverContainer = new ContainerBuilder().build() + val driverPod = new PodBuilder().build() + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets(driverPod, driverContainer) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala index 5240128743b76..83181c1fe3c4e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala new file mode 100644 index 0000000000000..8388c16ded268 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala @@ -0,0 +1,36 @@ +/* + * 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.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] object SecretVolumeUtils { + + def podHasVolume(driverPod: Pod, volumeName: String): Boolean = { + driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName) + } + + def containerHasVolume( + driverContainer: Container, + volumeName: String, + mountPath: String): Boolean = { + driverContainer.getVolumeMounts.asScala.exists(volumeMount => + volumeMount.getName == volumeName && volumeMount.getMountPath == mountPath) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala index 96fa92c254297..42114f1a6f82b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.UUID @@ -31,8 +31,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Response} import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} import org.apache.spark.util.Utils private[spark] class SubmittedDependencyUploaderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala similarity index 90% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala index 5e3718800d928..330d8e87a9fe9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { @@ -34,8 +34,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { private val APP_ARGS = Array("arg1", "arg2") private val CUSTOM_ANNOTATION_KEY = "customAnnotation" private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "customAnnotationDeprecated" - private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "customAnnotationDeprecatedValue" private val DRIVER_CUSTOM_ENV_KEY1 = "customDriverEnv1" private val DRIVER_CUSTOM_ENV_KEY2 = "customDriverEnv2" @@ -49,8 +47,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L) .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") .set(s"spark.kubernetes.driver.annotation.$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) - .set("spark.kubernetes.driver.annotations", - s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2") @@ -98,7 +94,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) val expectedAnnotations = Map( CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, - DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, SPARK_APP_NAME_ANNOTATION -> APP_NAME) assert(driverPodMetadata.getAnnotations.asScala === expectedAnnotations) assert(preparedDriverSpec.driverPod.getSpec.getRestartPolicy === "Never") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala index 3f7ec61074b0c..02b2820acaf3f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilde import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala new file mode 100644 index 0000000000000..2b9570fdd625c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala @@ -0,0 +1,181 @@ +/* + * 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.submit.submitsteps + +import io.fabric8.kubernetes.api.model.Service +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.util.Clock + +private[spark] class DriverAddressConfigurationStepSuite + extends SparkFunSuite with BeforeAndAfter { + + private val SHORT_RESOURCE_NAME_PREFIX = + "a" * (DriverAddressConfigurationStep.MAX_SERVICE_NAME_LENGTH - + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX.length) + + private val LONG_RESOURCE_NAME_PREFIX = + "a" * (DriverAddressConfigurationStep.MAX_SERVICE_NAME_LENGTH - + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX.length + 1) + private val DRIVER_LABELS = Map( + "label1key" -> "label1value", + "label2key" -> "label2value") + + @Mock + private var clock: Clock = _ + + private var sparkConf: SparkConf = _ + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf(false) + } + + test("Headless service has a port for the driver RPC and the block manager.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + assert(resolvedDriverSpec.otherKubernetesResources.size === 1) + assert(resolvedDriverSpec.otherKubernetesResources.head.isInstanceOf[Service]) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + verifyService( + 9000, + 8080, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}", + driverService) + } + + test("Hostname and ports are set according to the service name.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Ports should resolve to defaults in SparkConf and in the service.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf, + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + verifyService( + DEFAULT_DRIVER_PORT, + DEFAULT_BLOCKMANAGER_PORT, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}", + resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service]) + assert(resolvedDriverSpec.driverSparkConf.get("spark.driver.port") === + DEFAULT_DRIVER_PORT.toString) + assert(resolvedDriverSpec.driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT) === DEFAULT_BLOCKMANAGER_PORT) + } + + test("Long prefixes should switch to using a generated name.") { + val configurationStep = new DriverAddressConfigurationStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + when(clock.getTimeMillis()).thenReturn(10000) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + val expectedServiceName = s"spark-10000${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}" + assert(driverService.getMetadata.getName === expectedServiceName) + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Disallow bind address and driver host to be set explicitly.") { + val configurationStep = new DriverAddressConfigurationStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), + clock) + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver bind address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverAddressConfigurationStep.DRIVER_BIND_ADDRESS_KEY} is" + + s" not supported in Kubernetes mode, as the driver's hostname will be managed via" + + s" a Kubernetes service.") + } + sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) + sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver host address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverAddressConfigurationStep.DRIVER_HOST_KEY} is" + + s" not supported in Kubernetes mode, as the driver's hostname will be managed via" + + s" a Kubernetes service.") + } + } + + private def verifyService( + driverPort: Int, + blockManagerPort: Int, + expectedServiceName: String, + service: Service): Unit = { + assert(service.getMetadata.getName === expectedServiceName) + assert(service.getSpec.getClusterIP === "None") + assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) + assert(service.getSpec.getPorts.size() === 2) + val driverServicePorts = service.getSpec.getPorts.asScala + assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) + assert(driverServicePorts.head.getPort.intValue() === driverPort) + assert(driverServicePorts.head.getTargetPort.getIntVal === driverPort) + assert(driverServicePorts(1).getName === BLOCK_MANAGER_PORT_NAME) + assert(driverServicePorts(1).getPort.intValue() === blockManagerPort) + assert(driverServicePorts(1).getTargetPort.getIntVal === blockManagerPort) + } + + private def verifySparkConfHostNames( + driverSparkConf: SparkConf, expectedHostName: String): Unit = { + assert(driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_HOST_ADDRESS) === expectedHostName) + assert(driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) === expectedHostName) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala index 3d5664713a2b8..b29edca05f4a0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -25,8 +25,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils private[spark] class DriverKubernetesCredentialsStepSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala index b11b487111496..ee3b4229b16c1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.StringReader import java.util.Properties @@ -26,9 +26,9 @@ import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala new file mode 100644 index 0000000000000..a0a010ffe916e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala @@ -0,0 +1,46 @@ +/* + * 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.submit.submitsteps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, SecretVolumeUtils} + +private[spark] class MountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("Mounts all given secrets") { + val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false)) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val mountSecretsStep = new MountSecretsStep(mountSecretsBootstrap) + val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec) + val driverPodWithSecretsMounted = configuredDriverSpec.driverPod + val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala index 5e49ab2fc0e36..812031a306063 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.{File, RandomAccessFile} @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrap +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.MountSmallFilesBootstrap import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStepSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala index ce0dcee6acc46..66218f130b9c4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index fe1af4bc5be2a..65df5fcebe382 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model._ import org.mockito.{Mock, MockitoAnnotations} @@ -26,8 +26,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ private val SPARK_JARS = Seq( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala similarity index 57% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala index 1cc8007803457..3810a324f99b5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "namespace" @@ -44,9 +44,79 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" private val STAGING_SERVER_URI = "http://localhost:8000" + test ("error thrown if local jars provided without resource staging server") { + val sparkConf = new SparkConf(true) + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + assert(sparkConf.get(RESOURCE_STAGING_SERVER_URI).isEmpty) + + val thrown = intercept[IllegalArgumentException] { + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + } + + assert(thrown.getMessage contains "Local JARs were provided, however no resource staging" + + " server URI was found.") + } + + test ("error not thrown with non-local jars and resource staging server provided") { + val sparkConf = new SparkConf(true) + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 2) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps(1).isInstanceOf[SubmittedResourcesInitContainerConfigurationStep]) + } + + test ("error not thrown with non-local jars and no resource staging server provided") { + val sparkConf = new SparkConf(true) + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 1) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + } + test ("including step to contact resource staging server") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) @@ -71,13 +141,12 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("not including steps because no contact to resource staging server") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) val orchestrator = new InitContainerConfigurationStepsOrchestrator( NAMESPACE, APP_RESOURCE_PREFIX, - SPARK_JARS, + SPARK_JARS.take(1), SPARK_FILES, JARS_DOWNLOAD_PATH, FILES_DOWNLOAD_PATH, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala index 2edaba93fe07f..1488c0d00b7a5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File import java.util.UUID @@ -31,10 +31,10 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} import org.apache.spark.util.Utils class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala new file mode 100644 index 0000000000000..5ce199a5df857 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala @@ -0,0 +1,91 @@ +/* + * 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.kubernetes.submit.submitsteps + +import java.nio.file.Paths + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{KubernetesDriverSpec, LocalDirectoryMountConfigurationStep} + +private[spark] class LocalDirectoryMountConfigurationStepSuite extends SparkFunSuite { + + test("When using the external shuffle service, the local directories must be provided.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, true) + val configurationStep = new LocalDirectoryMountConfigurationStep(sparkConf) + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The configuration step should have failed without local dirs.") + } catch { + case e: Throwable => + assert(e.getMessage === "requirement failed: spark.local.dir must be provided explicitly" + + " when using the external shuffle service in Kubernetes. These directories should map" + + " to the paths that are mounted into the external shuffle service pods.") + } + } + + test("When not using the external shuffle service, a random directory should be set" + + " for local dirs if one is not provided.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, false) + val configurationStep = new LocalDirectoryMountConfigurationStep( + sparkConf, () => "local-dir") + val resolvedDriverSpec = configurationStep.configureDriver( + KubernetesDriverSpec.initialSpec(sparkConf)) + testLocalDirsMatch(resolvedDriverSpec, Seq(s"$GENERATED_LOCAL_DIR_MOUNT_ROOT/local-dir")) + } + + test("When not using the external shuffle service, provided local dirs should be mounted as" + + " emptyDirs.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, false) + .set("spark.local.dir", "/mnt/tmp/spark-local,/var/tmp/spark-local") + val configurationStep = new LocalDirectoryMountConfigurationStep( + sparkConf) + val resolvedDriverSpec = configurationStep.configureDriver( + KubernetesDriverSpec.initialSpec(sparkConf)) + testLocalDirsMatch(resolvedDriverSpec, Seq("/mnt/tmp/spark-local", "/var/tmp/spark-local")) + } + + private def testLocalDirsMatch( + resolvedDriverSpec: KubernetesDriverSpec, expectedLocalDirs: Seq[String]): Unit = { + assert(resolvedDriverSpec.driverSparkConf.get("spark.local.dir").split(",") === + expectedLocalDirs) + expectedLocalDirs + .zip(resolvedDriverSpec.driverPod.getSpec.getVolumes.asScala) + .zipWithIndex + .foreach { + case ((dir, volume), index) => + assert(volume.getEmptyDir != null) + val fileName = Paths.get(dir).getFileName.toString + assert(volume.getName === s"spark-local-dir-$index-$fileName") + } + + expectedLocalDirs + .zip(resolvedDriverSpec.driverContainer.getVolumeMounts.asScala) + .zipWithIndex + .foreach { + case ((dir, volumeMount), index) => + val fileName = Paths.get(dir).getFileName.toString + assert(volumeMount.getName === s"spark-local-dir-$index-$fileName") + assert(volumeMount.getMountPath === dir) + } + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala index f2fdf026390cd..4c80637da7843 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayOutputStream, File} import java.util.UUID @@ -33,8 +33,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Callback, Response} import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.Utils class KubernetesSparkDependencyDownloadInitContainerSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala index 3bb318d713a54..1ab7a7f4f966d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, StringWriter} import java.security.KeyStore @@ -25,7 +25,7 @@ import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.util.Utils class ResourceStagingServerSslOptionsProviderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala index 1bcd85a611e00..4e6b4abfb6874 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.net.ServerSocket import javax.ws.rs.core.MediaType @@ -29,7 +29,7 @@ import org.scalatest.mock.MockitoSugar.mock import retrofit2.Call import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala index 8b398a9891f34..d9b5c4ac5b9c5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala index 6b5737ebf2e23..b58046da67800 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayInputStream, File} import java.nio.file.Paths diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala new file mode 100644 index 0000000000000..f3baf5b9f739a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala @@ -0,0 +1,64 @@ +/* + * 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.scheduler.cluster.k8s + +import io.fabric8.kubernetes.api.model.{VolumeBuilder, VolumeMountBuilder} +import org.mockito.Mockito.{verify, verifyNoMoreInteractions, when} +import org.scalatest.mock.MockitoSugar.mock + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class ExecutorLocalDirVolumeProviderSuite extends SparkFunSuite { + + test("Delegates to the external shuffle manager implementation if present.") { + val externalShuffleManager = mock[KubernetesExternalShuffleManager] + val mockVolume = new VolumeBuilder() + .withName("local-dir") + .withNewHostPath("/tmp/spark-local-dirs") + .build() + val mockVolumeMount = new VolumeMountBuilder() + .withName("local-dir") + .withMountPath("/tmp/spark-local-dirs-mount") + .build() + when(externalShuffleManager.getExecutorShuffleDirVolumesWithMounts) + .thenReturn(Seq((mockVolume, mockVolumeMount))) + val volumeProvider = new ExecutorLocalDirVolumeProviderImpl( + new SparkConf(false), Some(externalShuffleManager)) + assert(volumeProvider.getExecutorLocalDirVolumesWithMounts === + Seq((mockVolume, mockVolumeMount))) + verify(externalShuffleManager).getExecutorShuffleDirVolumesWithMounts + verifyNoMoreInteractions(externalShuffleManager) + } + + test("Provides EmptyDir volumes for each local dir in spark.local.dirs.") { + val localDirs = Seq("/tmp/test-local-dir-1", "/tmp/test-local-dir-2") + val sparkConf = new SparkConf(false).set("spark.local.dir", localDirs.mkString(",")) + val volumeProvider = new ExecutorLocalDirVolumeProviderImpl(sparkConf, None) + val localDirVolumesWithMounts = volumeProvider.getExecutorLocalDirVolumesWithMounts + assert(localDirVolumesWithMounts.size === 2) + localDirVolumesWithMounts.zip(localDirs).zipWithIndex.foreach { + case (((localDirVolume, localDirVolumeMount), expectedDirMountPath), index) => + val dirName = expectedDirMountPath.substring( + expectedDirMountPath.lastIndexOf('/') + 1, expectedDirMountPath.length) + assert(localDirVolume.getName === s"spark-local-dir-$index-$dirName") + assert(localDirVolume.getEmptyDir != null) + assert(localDirVolumeMount.getName === localDirVolume.getName) + assert(localDirVolumeMount.getMountPath === expectedDirMountPath) + case unknown => throw new IllegalArgumentException("Unexpected object: $unknown") + } + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala new file mode 100644 index 0000000000000..bb09cb801b5a9 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -0,0 +1,300 @@ +/* + * 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.scheduler.cluster.k8s + +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder, VolumeBuilder, VolumeMountBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import org.mockito.{AdditionalAnswers, Mock, Mockito, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito._ +import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrap, MountSmallFilesBootstrapImpl} + +class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { + private val driverPodName: String = "driver-pod" + private val driverPodUid: String = "driver-uid" + private val driverUrl: String = "driver-url" + private val executorPrefix: String = "base" + private val executorImage: String = "executor-image" + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(driverPodName) + .withUid(driverPodUid) + .endMetadata() + .withNewSpec() + .withNodeName("some-node") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private var baseConf: SparkConf = _ + + @Mock + private var nodeAffinityExecutorPodModifier: NodeAffinityExecutorPodModifier = _ + + @Mock + private var executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider = _ + + before { + MockitoAnnotations.initMocks(this) + baseConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, driverPodName) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix) + .set(EXECUTOR_DOCKER_IMAGE, executorImage) + when(nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( + any(classOf[Pod]), + any(classOf[Map[String, Int]]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + when(executorLocalDirVolumeProvider.getExecutorLocalDirVolumesWithMounts).thenReturn(Seq.empty) + } + private var kubernetesClient: KubernetesClient = _ + + test("basic executor pod has reasonable defaults") { + val factory = new ExecutorPodFactoryImpl( + baseConf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + // The executor pod name and default labels. + assert(executor.getMetadata.getName === s"$executorPrefix-exec-1") + assert(executor.getMetadata.getLabels.size() === 3) + + // There is exactly 1 container with no volume mounts and default memory limits. + // Default memory limit is 1024M + 384M (minimum overhead constant). + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getImage === executorImage) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.isEmpty) + assert(executor.getSpec.getContainers.get(0).getResources.getLimits.size() === 1) + assert(executor.getSpec.getContainers.get(0).getResources + .getLimits.get("memory").getAmount === "1408Mi") + + // The pod has no node selector, volumes. + assert(executor.getSpec.getNodeSelector.isEmpty) + assert(executor.getSpec.getVolumes.isEmpty) + + checkEnv(executor, Map()) + checkOwnerReferences(executor, driverPodUid) + } + + test("executor pod hostnames get truncated to 63 characters") { + val conf = baseConf.clone() + conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, + "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getHostname.length === 63) + } + + test("secrets get mounted") { + val conf = baseConf.clone() + + val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + Some(secretsBootstrap), + None, + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + // check volume mounted. + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container bootstrap step adds an init container") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[SparkPodInitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainerAndVolumes( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + Some(initContainerBootstrap), + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getMetadata.getAnnotations.size() === 1) + assert(executor.getMetadata.getAnnotations.containsKey(INIT_CONTAINER_ANNOTATION)) + checkOwnerReferences(executor, driverPodUid) + } + + test("The local dir volume provider's returned volumes and volume mounts should be added.") { + Mockito.reset(executorLocalDirVolumeProvider) + val localDirVolume = new VolumeBuilder() + .withName("local-dir") + .withNewEmptyDir().endEmptyDir() + .build() + val localDirVolumeMount = new VolumeMountBuilder() + .withName("local-dir") + .withMountPath("/tmp") + .build() + when(executorLocalDirVolumeProvider.getExecutorLocalDirVolumesWithMounts) + .thenReturn(Seq((localDirVolume, localDirVolumeMount))) + val factory = new ExecutorPodFactoryImpl( + baseConf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + assert(executor.getSpec.getVolumes.size === 1) + assert(executor.getSpec.getVolumes.contains(localDirVolume)) + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.contains(localDirVolumeMount)) + } + + test("Small-files add a secret & secret volume mount to the container") { + val conf = baseConf.clone() + + val smallFiles = new MountSmallFilesBootstrapImpl("secret1", "/var/secret1") + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + Some(smallFiles), + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getName === "submitted-files") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + checkEnv(executor, Map("SPARK_MOUNTED_FILES_FROM_SECRET_DIR" -> "/var/secret1")) + } + + test("classpath and extra java options get translated into environment variables") { + val conf = baseConf.clone() + conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") + conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + checkEnv(executor, + Map("SPARK_JAVA_OPT_0" -> "foo=bar", + "SPARK_EXECUTOR_EXTRA_CLASSPATH" -> "bar=baz", + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + } + + // There is always exactly one controller reference, and it points to the driver pod. + private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { + assert(executor.getMetadata.getOwnerReferences.size() === 1) + assert(executor.getMetadata.getOwnerReferences.get(0).getUid === driverPodUid) + assert(executor.getMetadata.getOwnerReferences.get(0).getController === true) + } + + // Check that the expected environment variables are present. + private def checkEnv(executor: Pod, additionalEnvVars: Map[String, String]): Unit = { + val defaultEnvs = Map( + ENV_EXECUTOR_ID -> "1", + ENV_DRIVER_URL -> "dummy", + ENV_EXECUTOR_CORES -> "1", + ENV_EXECUTOR_MEMORY -> "1g", + ENV_APPLICATION_ID -> "dummy", + ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", + ENV_EXECUTOR_POD_IP -> null, + ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) + val mapEnvs = executor.getSpec.getContainers.get(0).getEnv.asScala.map { + x => (x.getName, x.getValue) + }.toMap + assert(defaultEnvs === mapEnvs) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..a9a2937869edd --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -0,0 +1,383 @@ +/* + * 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.scheduler.cluster.k8s + +import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} + +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource} +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito.{doNothing, never, times, verify, when} +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ +import scala.concurrent.Future + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpoint, RpcEndpointAddress, RpcEndpointRef, RpcEnv, RpcTimeout} +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend + +private[spark] class KubernetesClusterSchedulerBackendSuite + extends SparkFunSuite with BeforeAndAfter { + + private val APP_ID = "test-spark-app" + private val DRIVER_POD_NAME = "spark-driver-pod" + private val NAMESPACE = "test-namespace" + private val SPARK_DRIVER_HOST = "localhost" + private val SPARK_DRIVER_PORT = 7077 + private val POD_ALLOCATION_INTERVAL = 60L + private val DRIVER_URL = RpcEndpointAddress( + SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + private val FIRST_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod1") + .endMetadata() + .withNewSpec() + .withNodeName("node1") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private val SECOND_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod2") + .endMetadata() + .withNewSpec() + .withNodeName("node2") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.101") + .endStatus() + .build() + + private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + private type LABELLED_PODS = FilterWatchListDeletable[ + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + private type IN_NAMESPACE_PODS = NonNamespaceOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var sparkContext: SparkContext = _ + + @Mock + private var listenerBus: LiveListenerBus = _ + + @Mock + private var taskSchedulerImpl: TaskSchedulerImpl = _ + + @Mock + private var allocatorExecutor: ScheduledExecutorService = _ + + @Mock + private var requestExecutorsService: ExecutorService = _ + + @Mock + private var executorPodFactory: ExecutorPodFactory = _ + + @Mock + private var shuffleManager: KubernetesExternalShuffleManager = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var podsWithLabelOperations: LABELLED_PODS = _ + + @Mock + private var podsInNamespace: IN_NAMESPACE_PODS = _ + + @Mock + private var podsWithDriverName: PodResource[Pod, DoneablePod] = _ + + @Mock + private var rpcEnv: RpcEnv = _ + + @Mock + private var driverEndpointRef: RpcEndpointRef = _ + + @Mock + private var executorPodsWatch: Watch = _ + + private var sparkConf: SparkConf = _ + private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _ + private var allocatorRunnable: ArgumentCaptor[Runnable] = _ + private var requestExecutorRunnable: ArgumentCaptor[Runnable] = _ + private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ + + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(DRIVER_POD_NAME) + .addToLabels(SPARK_APP_ID_LABEL, APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .endMetadata() + .build() + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf() + .set("spark.app.id", APP_ID) + .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) + .set(KUBERNETES_NAMESPACE, NAMESPACE) + .set("spark.driver.host", SPARK_DRIVER_HOST) + .set("spark.driver.port", SPARK_DRIVER_PORT.toString) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) + allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) + when(sparkContext.conf).thenReturn(sparkConf) + when(sparkContext.listenerBus).thenReturn(listenerBus) + when(taskSchedulerImpl.sc).thenReturn(sparkContext) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations) + when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture())) + .thenReturn(executorPodsWatch) + when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace) + when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName) + when(podsWithDriverName.get()).thenReturn(driverPod) + when(allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable.capture(), + mockitoEq(0L), + mockitoEq(POD_ALLOCATION_INTERVAL), + mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + // Creating Futures in Scala backed by a Java executor service resolves to running + // ExecutorService#execute (as opposed to submit) + doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) + when(rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + .thenReturn(driverEndpointRef) + when(driverEndpointRef.ask[Boolean] + (any(classOf[Any])) + (any())).thenReturn(mock[Future[Boolean]]) + } + + test("Basic lifecycle expectations when starting and stopping the scheduler.") { + val scheduler = newSchedulerBackend(true) + scheduler.start() + verify(shuffleManager).start(APP_ID) + assert(executorPodsWatcherArgument.getValue != null) + assert(allocatorRunnable.getValue != null) + scheduler.stop() + verify(shuffleManager).stop() + verify(executorPodsWatch).close() + } + + test("Static allocation should request executors upon first allocator run.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Killing executors deletes the executor pods") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + scheduler.doKillExecutors(Seq("2")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations).delete(SECOND_EXECUTOR_POD) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should be requested in batches.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations, never()).create(SECOND_EXECUTOR_POD) + val registerFirstExecutorMessage = RegisterExecutor( + "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + allocatorRunnable.getValue.run() + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Deleting executors and then running an allocator pass after finding the loss reason" + + " should only delete the pod once.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + scheduler.doRequestTotalExecutors(0) + requestExecutorRunnable.getAllValues.asScala.last.run() + scheduler.doKillExecutors(Seq("1")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + + val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0) + executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) + allocatorRunnable.getValue.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef, times(1)).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 0, + exitCausedByApp = false, + s"Container in pod ${exitedPod.getMetadata.getName} exited from" + + s" explicit termination request."))) + } + + test("Executors that disconnect from application errors are noted as exits caused by app.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + executorPodsWatcherArgument.getValue.eventReceived( + Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + scheduler.doRequestTotalExecutors(1) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getAllValues.asScala.last.run() + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 1, + exitCausedByApp = true, + s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + + " exit status code 1."))) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should only try to get the loss reason a number of times before giving up and" + + " removing the executor.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => + allocatorRunnable.getValue.run() + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", SlaveLost("Executor lost for unknown reasons."))) + } + + private def newSchedulerBackend(externalShuffle: Boolean): KubernetesClusterSchedulerBackend = { + new KubernetesClusterSchedulerBackend( + taskSchedulerImpl, + rpcEnv, + executorPodFactory, + if (externalShuffle) Some(shuffleManager) else None, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) + } + + private def exitPod(basePod: Pod, exitCode: Int): Pod = { + new PodBuilder(FIRST_EXECUTOR_POD) + .editStatus() + .addNewContainerStatus() + .withNewState() + .withNewTerminated() + .withExitCode(exitCode) + .endTerminated() + .endState() + .endContainerStatus() + .endStatus() + .build() + } + + private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Unit = { + when(executorPodFactory.createExecutorPod( + executorId.toString, + APP_ID, + DRIVER_URL, + sparkConf.getExecutorEnv, + driverPod, + Map.empty)).thenReturn(expectedPod) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala index e10b9508530f7..4924b67480048 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.{Pod, PodSpec, PodStatus} import org.mockito.Matchers._ @@ -22,7 +22,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.FakeTask class KubernetesTaskSchedulerImplSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala index 889758731a6c1..56c576a7dd449 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer @@ -23,7 +23,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{FakeTask, FakeTaskScheduler, HostTaskLocation, TaskLocation} class KubernetesTaskSetManagerSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala index ee671ce7d6590..ec1428117280f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile index 1178dd2428448..a52f5e1305598 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile @@ -21,4 +21,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.KubernetesSparkDependencyDownloadInitContainer" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile index 87ed7d10f3eb3..9ddc2c9bcaa0f 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile @@ -22,4 +22,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.ResourceStagingServer" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.ResourceStagingServer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile index b76e66d316c5c..a240e76a12a15 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile @@ -21,4 +21,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-shuffle:latest -f dockerfiles/shuffle-service/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService", "1" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.k8s.KubernetesExternalShuffleService", "1" ] diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java similarity index 94% rename from resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java rename to resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java index 99d982397bb6e..b18f8deea6a20 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest; +package org.apache.spark.deploy.k8s.integrationtest; /** * Primarily extracted so that a separate jar can be added as a dependency for the diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala index a9e328f4ff248..be1c3f6dc3af4 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.File import java.nio.file.Paths diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala index fe47d42485b24..7bae66cbf71f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.util.Random diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala index 967032eddccb5..62cba7f6b0742 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala similarity index 92% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala index d3372749f999e..5ae22671e41f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs -import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper +import org.apache.spark.deploy.k8s.integrationtest.PiHelper import org.apache.spark.sql.SparkSession // Equivalent to SparkPi except does not stop the Spark Context diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala similarity index 95% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index e204d0173aff8..27041207ffdce 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, FileOutputStream} import java.nio.file.Paths @@ -29,12 +29,12 @@ import org.scalatest.time.{Minutes, Seconds, Span} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} +import org.apache.spark.deploy.k8s.SSLUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackendFactory +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -64,7 +64,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf = kubernetesTestComponents.newSparkConf() .set(INIT_CONTAINER_DOCKER_IMAGE, s"spark-init:latest") .set(DRIVER_DOCKER_IMAGE, s"spark-driver:latest") - .set(KUBERNETES_DRIVER_LABELS, s"spark-app-locator=$APP_LOCATOR_LABEL") + .set(s"${KUBERNETES_DRIVER_LABEL_PREFIX}spark-app-locator", APP_LOCATOR_LABEL) kubernetesTestComponents.createNamespace() } @@ -150,6 +150,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) sparkConf.set("spark.dynamicAllocation.enabled", "true") + sparkConf.set("spark.local.dir", "/tmp") sparkConf.set("spark.shuffle.service.enabled", "true") sparkConf.set("spark.kubernetes.shuffle.labels", "app=spark-shuffle-service") sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace) @@ -421,7 +422,7 @@ private[spark] object KubernetesSuite { s"integration-tests-jars/${HELPER_JAR_FILE.getName}" val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) - val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = @@ -429,11 +430,11 @@ private[spark] object KubernetesSuite { val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/examples/src/main/python/sort.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" - val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.FileExistenceTest" - val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.GroupByTest" - val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.JavaOptionsTest" val TEST_EXISTENCE_FILE_CONTENTS = "contents" diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala similarity index 95% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 0ca1f482269db..44eb125fb8e77 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.util.UUID @@ -23,7 +23,7 @@ import org.scalatest.concurrent.Eventually import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 4008007b72fc4..ed15f66f53458 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{BufferedReader, InputStreamReader} import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala index e5e1b1f085f9f..1aa90f4ce691d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, StringWriter} import java.util.Properties @@ -25,8 +25,8 @@ import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SSLOptions -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala index bd604ab94b936..f1fd6dc19ce54 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala index 6b483769f5254..69a9df63bc4af 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import io.fabric8.kubernetes.api.model.{HTTPGetActionBuilder, Pod} import io.fabric8.kubernetes.client.KubernetesClient diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala index 1ef096be4af02..09a3118a202eb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE +package org.apache.spark.deploy.k8s.integrationtest.backend.GCE import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.config.resolveK8sMaster -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.GCE_TEST_BACKEND +import org.apache.spark.deploy.k8s.config.resolveK8sMaster +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.GCE_TEST_BACKEND private[spark] class GCETestBackend(val master: String) extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala similarity index 78% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index c5bc923dd51a6..b61daf65a4093 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend +package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE.GCETestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.GCE.GCETestBackend +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] trait IntegrationTestBackend { def name(): String diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala index 7c4b344e8f72b..bd31bde6dabf4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import java.nio.file.Paths import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.integrationtest.ProcessUtils +import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala index 461264877edc2..8e94f13360536 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] class MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala index bfded1003fc25..0807a68cd823c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest package object constants { val MINIKUBE_TEST_BACKEND = "minikube" diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala index e240fcf953f8c..0e2fced70c9f7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.docker +package org.apache.spark.deploy.k8s.integrationtest.docker import java.io.File import java.net.URI diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala index 7a3b06b1b5e58..6c7ef3c592431 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.restapis +package org.apache.spark.deploy.k8s.integrationtest.restapis import java.util.{List => JList} import javax.ws.rs._ diff --git a/sbin/build-push-docker-images.sh b/sbin/build-push-docker-images.sh new file mode 100755 index 0000000000000..efd6f96516d90 --- /dev/null +++ b/sbin/build-push-docker-images.sh @@ -0,0 +1,72 @@ +#!/usr/bin/env bash + +# 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. +# + +# This script builds and pushes docker images when run from a release of Spark +# with Kubernetes support. + +declare -A path=( [spark-driver]=dockerfiles/driver/Dockerfile \ + [spark-executor]=dockerfiles/executor/Dockerfile \ + [spark-driver-py]=dockerfiles/driver-py/Dockerfile \ + [spark-executor-py]=dockerfiles/executor-py/Dockerfile \ + [spark-init]=dockerfiles/init-container/Dockerfile \ + [spark-shuffle]=dockerfiles/shuffle-service/Dockerfile \ + [spark-resource-staging-server]=dockerfiles/resource-staging-server/Dockerfile ) + +function build { + docker build -t spark-base -f dockerfiles/spark-base/Dockerfile . + for image in "${!path[@]}"; do + docker build -t ${REPO}/$image:${TAG} -f ${path[$image]} . + done +} + + +function push { + for image in "${!path[@]}"; do + docker push ${REPO}/$image:${TAG} + done +} + +function usage { + echo "Usage: ./sbin/build-push-docker-images.sh -r -t build" + echo " ./sbin/build-push-docker-images.sh -r -t push" + echo "for example: ./sbin/build-push-docker-images.sh -r docker.io/kubespark -t v2.2.0 push" +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage + exit 0 +fi + +while getopts r:t: option +do + case "${option}" + in + r) REPO=${OPTARG};; + t) TAG=${OPTARG};; + esac +done + +if [ -z "$REPO" ] || [ -z "$TAG" ]; then + usage +else + case "${@: -1}" in + build) build;; + push) push;; + *) usage;; + esac +fi \ No newline at end of file