diff --git a/.travis.yml b/.travis.yml index 8739849a20798..b9ae28a421309 100644 --- a/.travis.yml +++ b/.travis.yml @@ -25,11 +25,22 @@ sudo: required dist: trusty -# 2. Choose language and target JDKs for parallel builds. +# 2. Choose language, target JDK and env's for parallel builds. language: java jdk: - - oraclejdk7 - oraclejdk8 +env: # Used by the install section below. + # Configure the unit test build for spark core and kubernetes modules, + # while excluding some flaky unit tests using a regex pattern. + - PHASE=test \ + PROFILES="-Pmesos -Pyarn -Phadoop-2.7 -Pkubernetes" \ + MODULES="-pl core,resource-managers/kubernetes/core -am" \ + ARGS="-Dtest=none -Dsuffixes='^org\.apache\.spark\.(?!ExternalShuffleServiceSuite|SortShuffleSuite$|rdd\.LocalCheckpointSuite$|deploy\.SparkSubmitSuite$|deploy\.StandaloneDynamicAllocationSuite$).*'" + # Configure the full build. + - PHASE=install \ + PROFILES="-Pmesos -Pyarn -Phadoop-2.7 -Pkubernetes -Pkinesis-asl -Phive -Phive-thriftserver" \ + MODULES="" \ + ARGS="-T 4 -q -DskipTests" # 3. Setup cache directory for SBT and Maven. cache: @@ -41,11 +52,12 @@ cache: notifications: email: false -# 5. Run maven install before running lint-java. +# 5. Run maven build before running lints. install: - export MAVEN_SKIP_RC=1 - - build/mvn -T 4 -q -DskipTests -Pmesos -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install + - build/mvn ${PHASE} ${PROFILES} ${MODULES} ${ARGS} -# 6. Run lint-java. +# 6. Run lints. script: - dev/lint-java + - dev/lint-scala diff --git a/README.md b/README.md index f5983239c043f..cb747225a11d4 100644 --- a/README.md +++ b/README.md @@ -1,104 +1,34 @@ -# Apache Spark +# Apache Spark On Kubernetes -Spark is a fast and general cluster computing system for Big Data. It provides -high-level APIs in Scala, Java, Python, and R, and an optimized engine that -supports general computation graphs for data analysis. It also supports a -rich set of higher-level tools including Spark SQL for SQL and DataFrames, -MLlib for machine learning, GraphX for graph processing, -and Spark Streaming for stream processing. +This repository, located at https://github.com/apache-spark-on-k8s/spark, contains a fork of Apache Spark that enables running Spark jobs natively on a Kubernetes cluster. - +## What is this? +This is a collaboratively maintained project working on [SPARK-18278](https://issues.apache.org/jira/browse/SPARK-18278). The goal is to bring native support for Spark to use Kubernetes as a cluster manager, in a fully supported way on par with the Spark Standalone, Mesos, and Apache YARN cluster managers. -## Online Documentation +## Getting Started -You can find the latest Spark documentation, including a programming -guide, on the [project web page](http://spark.apache.org/documentation.html). -This README file only contains basic setup instructions. +- [Usage guide](https://apache-spark-on-k8s.github.io/userdocs/) shows how to run the code +- [Development docs](resource-managers/kubernetes/README.md) shows how to get set up for development +- Code is primarily located in the [resource-managers/kubernetes](resource-managers/kubernetes) folder -## Building Spark +## Why does this fork exist? -Spark is built using [Apache Maven](http://maven.apache.org/). -To build Spark and its example programs, run: +Adding native integration for a new cluster manager is a large undertaking. If poorly executed, it could introduce bugs into Spark when run on other cluster managers, cause release blockers slowing down the overall Spark project, or require hotfixes which divert attention away from development towards managing additional releases. Any work this deep inside Spark needs to be done carefully to minimize the risk of those negative externalities. - build/mvn -DskipTests clean package +At the same time, an increasing number of people from various companies and organizations desire to work together to natively run Spark on Kubernetes. The group needs a code repository, communication forum, issue tracking, and continuous integration, all in order to work together effectively on an open source product. -(You do not need to do this if you downloaded a pre-built package.) +We've been asked by an Apache Spark Committer to work outside of the Apache infrastructure for a short period of time to allow this feature to be hardened and improved without creating risk for Apache Spark. The aim is to rapidly bring it to the point where it can be brought into the mainline Apache Spark repository for continued development within the Apache umbrella. If all goes well, this should be a short-lived fork rather than a long-lived one. -You can build Spark using more than one thread by using the -T option with Maven, see ["Parallel builds in Maven 3"](https://cwiki.apache.org/confluence/display/MAVEN/Parallel+builds+in+Maven+3). -More detailed documentation is available from the project site, at -["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). +## Who are we? -For general development tips, including info on developing Spark using an IDE, see -[http://spark.apache.org/developer-tools.html](the Useful Developer Tools page). +This is a collaborative effort by several folks from different companies who are interested in seeing this feature be successful. Companies active in this project include (alphabetically): -## Interactive Scala Shell - -The easiest way to start using Spark is through the Scala shell: - - ./bin/spark-shell - -Try the following command, which should return 1000: - - scala> sc.parallelize(1 to 1000).count() - -## Interactive Python Shell - -Alternatively, if you prefer Python, you can use the Python shell: - - ./bin/pyspark - -And run the following command, which should also return 1000: - - >>> sc.parallelize(range(1000)).count() - -## Example Programs - -Spark also comes with several sample programs in the `examples` directory. -To run one of them, use `./bin/run-example [params]`. For example: - - ./bin/run-example SparkPi - -will run the Pi example locally. - -You can set the MASTER environment variable when running examples to submit -examples to a cluster. This can be a mesos:// or spark:// URL, -"yarn" to run on YARN, and "local" to run -locally with one thread, or "local[N]" to run locally with N threads. You -can also use an abbreviated class name if the class is in the `examples` -package. For instance: - - MASTER=spark://host:7077 ./bin/run-example SparkPi - -Many of the example programs print usage help if no params are given. - -## Running Tests - -Testing first requires [building Spark](#building-spark). Once Spark is built, tests -can be run using: - - ./dev/run-tests - -Please see the guidance on how to -[run tests for a module, or individual tests](http://spark.apache.org/developer-tools.html#individual-tests). - -## A Note About Hadoop Versions - -Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported -storage systems. Because the protocols have changed in different versions of -Hadoop, you must build Spark against the same version that your cluster runs. - -Please refer to the build documentation at -["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version) -for detailed guidance on building for a particular distribution of Hadoop, including -building for particular Hive and Hive Thriftserver distributions. - -## Configuration - -Please refer to the [Configuration Guide](http://spark.apache.org/docs/latest/configuration.html) -in the online documentation for an overview on how to configure Spark. - -## Contributing - -Please review the [Contribution to Spark guide](http://spark.apache.org/contributing.html) -for information on how to get started contributing to the project. +- Bloomberg +- Google +- Haiwen +- Hyperpilot +- Intel +- Palantir +- Pepperdata +- Red Hat \ No newline at end of file diff --git a/assembly/pom.xml b/assembly/pom.xml index aebfd12227751..ab733e7a6aa35 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml @@ -148,6 +148,16 @@ + + kubernetes + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + hive diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 67d78d5f102fb..de83219cddf37 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 93790979d7b26..ab418e30c74f8 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml 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 new file mode 100644 index 0000000000000..49cb5243e32dc --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java @@ -0,0 +1,79 @@ +/* + * 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 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 saslEnabled, + boolean saslEncryptionEnabled) { + super(conf, secretKeyHolder, saslEnabled, saslEncryptionEnabled); + } + + public void registerDriverWithShuffleService(String host, int port) throws IOException { + 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); + } + } + + @Override + public void close() { + super.close(); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java index 42cedd9943150..e36cfd165db30 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/mesos/MesosExternalShuffleClient.java @@ -32,7 +32,7 @@ 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.mesos.RegisterDriver; +import org.apache.spark.network.shuffle.protocol.RegisterDriver; import org.apache.spark.network.util.TransportConf; /** diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index 9af6759f5d5f3..6012a84599368 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -23,7 +23,6 @@ import io.netty.buffer.Unpooled; import org.apache.spark.network.protocol.Encodable; -import org.apache.spark.network.shuffle.protocol.mesos.RegisterDriver; import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat; /** diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterDriver.java similarity index 91% rename from common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterDriver.java index d5f53ccb7f741..ac606e6539f3e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/mesos/RegisterDriver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterDriver.java @@ -15,19 +15,18 @@ * limitations under the License. */ -package org.apache.spark.network.shuffle.protocol.mesos; +package org.apache.spark.network.shuffle.protocol; import com.google.common.base.Objects; import io.netty.buffer.ByteBuf; import org.apache.spark.network.protocol.Encoders; -import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; // Needed by ScalaDoc. See SPARK-7726 import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** - * A message sent from the driver to register with the MesosExternalShuffleService. + * A message sent from the driver to register with an ExternalShuffleService. */ public class RegisterDriver extends BlockTransferMessage { private final String appId; diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53cb8dd815d81..1ec41447511ae 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 89bee8567fc74..e143da1f9790f 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 7b45b23e9c546..fa67d941cf600 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 9b84f1e0c1dfc..93ccd5ba4e8cd 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index a7b0e6f80c2b6..fd6e95c3e0a38 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -252,7 +252,7 @@ public static long parseSecondNano(String secondNano) throws IllegalArgumentExce public final int months; public final long microseconds; - public final long milliseconds() { + public long milliseconds() { return this.microseconds / MICROS_PER_MILLI; } diff --git a/conf/kubernetes-resource-staging-server.yaml b/conf/kubernetes-resource-staging-server.yaml new file mode 100644 index 0000000000000..025b9b125d9e0 --- /dev/null +++ b/conf/kubernetes-resource-staging-server.yaml @@ -0,0 +1,79 @@ +# +# 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: extensions/v1beta1 +kind: Deployment +metadata: + name: spark-resource-staging-server +spec: + replicas: 1 + template: + metadata: + labels: + resource-staging-server-instance: default + spec: + volumes: + - name: resource-staging-server-properties + configMap: + name: spark-resource-staging-server-config + containers: + - name: spark-resource-staging-server + image: kubespark/spark-resource-staging-server:v2.1.0-kubernetes-0.2.0 + resources: + requests: + cpu: 100m + memory: 256Mi + limits: + cpu: 100m + memory: 256Mi + volumeMounts: + - name: resource-staging-server-properties + mountPath: '/etc/spark-resource-staging-server' + args: + - '/etc/spark-resource-staging-server/resource-staging-server.properties' +--- +apiVersion: v1 +kind: ConfigMap +metadata: + name: spark-resource-staging-server-config +data: + resource-staging-server.properties: | + spark.kubernetes.resourceStagingServer.port=10000 + spark.ssl.kubernetes.resourceStagingServer.enabled=false +# Other possible properties are listed below, primarily for setting up TLS. The paths given by KeyStore, password, and PEM files here should correspond to +# files that are securely mounted into the resource staging server container, via e.g. secret volumes. +# spark.ssl.kubernetes.resourceStagingServer.keyStore=/mnt/secrets/resource-staging-server/keyStore.jks +# spark.ssl.kubernetes.resourceStagingServer.keyStorePassword=changeit +# spark.ssl.kubernetes.resourceStagingServer.keyPassword=changeit +# spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile=/mnt/secrets/resource-staging-server/keystore-password.txt +# spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile=/mnt/secrets/resource-staging-server/keystore-key-password.txt +# spark.ssl.kubernetes.resourceStagingServer.keyPem=/mnt/secrets/resource-staging-server/key.pem +# spark.ssl.kubernetes.resourceStagingServer.serverCertPem=/mnt/secrets/resource-staging-server/cert.pem +--- +apiVersion: v1 +kind: Service +metadata: + name: spark-resource-staging-service +spec: + type: NodePort + selector: + resource-staging-server-instance: default + ports: + - protocol: TCP + port: 10000 + targetPort: 10000 + nodePort: 31000 diff --git a/conf/kubernetes-shuffle-service.yaml b/conf/kubernetes-shuffle-service.yaml new file mode 100644 index 0000000000000..55c170b01a4f5 --- /dev/null +++ b/conf/kubernetes-shuffle-service.yaml @@ -0,0 +1,54 @@ +# +# 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: extensions/v1beta1 +kind: DaemonSet +metadata: + labels: + app: spark-shuffle-service + spark-version: 2.1.0 + name: shuffle +spec: + template: + metadata: + labels: + app: spark-shuffle-service + spark-version: 2.1.0 + spec: + volumes: + - name: temp-volume + hostPath: + path: '/var/tmp' # 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.1.0-kubernetes-0.2.0 + imagePullPolicy: IfNotPresent + volumeMounts: + - mountPath: '/tmp' + name: temp-volume + # more volumes can be mounted here. + # The spark job must be configured to use these + # mounts using the configuration: + # spark.kubernetes.shuffle.dir=,,... + resources: + requests: + cpu: "1" + limits: + cpu: "1" \ No newline at end of file diff --git a/core/pom.xml b/core/pom.xml index bbe07006109ea..9b273984ce321 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js index fe5db6aa26b65..fa0282678d1f4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage.js @@ -54,7 +54,28 @@ $(document).ajaxStart(function () { $.blockUI({message: '

Loading Executors Page...

'}); }); +function findKubernetesServiceBaseURI() { + var k8sProxyPattern = '/api/v1/proxy/namespaces/'; + var k8sProxyPatternPos = document.baseURI.indexOf(k8sProxyPattern); + if (k8sProxyPatternPos > 0) { + // Spark is running in a kubernetes cluster, and the web ui is served + // through the kubectl proxy. + var remaining = document.baseURI.substr(k8sProxyPatternPos + k8sProxyPattern.length); + var urlSlashesCount = remaining.split('/').length - 3; + var words = document.baseURI.split('/'); + var baseURI = words.slice(0, words.length - urlSlashesCount).join('/'); + return baseURI; + } + + return null; +} + function createTemplateURI(appId) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + return kubernetesBaseURI + '/static/executorspage-template.html'; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { @@ -70,6 +91,14 @@ function createTemplateURI(appId) { } function getStandAloneppId(cb) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + var appIdAndPort = kubernetesBaseURI.split('/').slice(-1)[0]; + var appId = appIdAndPort.split(':')[0]; + cb(appId); + return; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { @@ -95,6 +124,11 @@ function getStandAloneppId(cb) { } function createRESTEndPoint(appId) { + var kubernetesBaseURI = findKubernetesServiceBaseURI(); + if (kubernetesBaseURI) { + return kubernetesBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; + } + var words = document.baseURI.split('/'); var ind = words.indexOf("proxy"); if (ind > 0) { 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 c70061bc5b5bc..7c93a958fb4f9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -70,7 +70,8 @@ object SparkSubmit { private val STANDALONE = 2 private val MESOS = 4 private val LOCAL = 8 - private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val KUBERNETES = 16 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | KUBERNETES | LOCAL // Deploy modes private val CLIENT = 1 @@ -239,9 +240,10 @@ object SparkSubmit { YARN case m if m.startsWith("spark") => STANDALONE case m if m.startsWith("mesos") => MESOS + case m if m.startsWith("k8s") => KUBERNETES case m if m.startsWith("local") => LOCAL case _ => - printErrorAndExit("Master must either be yarn or start with spark, mesos, local") + printErrorAndExit("Master must either be yarn or start with spark, mesos, k8s, or local") -1 } @@ -284,6 +286,7 @@ object SparkSubmit { } val isYarnCluster = clusterManager == YARN && deployMode == CLUSTER val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER + val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files // too for packages that include Python code @@ -330,6 +333,10 @@ object SparkSubmit { // The following modes are not supported or applicable (clusterManager, deployMode) match { + case (KUBERNETES, CLIENT) => + printErrorAndExit("Client mode is currently not supported for Kubernetes.") + case (KUBERNETES, CLUSTER) if args.isR => + printErrorAndExit("Kubernetes does not currently support R applications.") case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") @@ -463,17 +470,21 @@ object SparkSubmit { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), - // Other options + OptionAssigner(args.kubernetesNamespace, KUBERNETES, ALL_DEPLOY_MODES, + sysProp = "spark.kubernetes.namespace"), + + // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), - OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, + OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, sysProp = "spark.files"), OptionAssigner(args.jars, LOCAL, CLIENT, sysProp = "spark.jars"), - OptionAssigner(args.jars, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, + sysProp = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN, CLUSTER, sysProp = "spark.driver.memory"), OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN, CLUSTER, @@ -506,8 +517,9 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" + // In Kubernetes cluster mode, the jar will be uploaded by the client separately. // For python and R files, the primary resource is already distributed as a regular file - if (!isYarnCluster && !args.isPython && !args.isR) { + if (!isYarnCluster && !isKubernetesCluster && !args.isPython && !args.isR) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) @@ -606,6 +618,24 @@ object SparkSubmit { } } + if (isKubernetesCluster) { + childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" + if (args.isPython) { + childArgs ++= Array("--primary-py-file", args.primaryResource) + childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") + if (args.pyFiles != null) { + childArgs ++= Array("--other-py-files", args.pyFiles) + } + } else { + childArgs ++= Array("--primary-java-resource", args.primaryResource) + childArgs ++= Array("--main-class", args.mainClass) + } + args.childArgs.foreach { arg => + childArgs += "--arg" + childArgs += arg + } + } + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index f1761e7c1ec92..4e297fe3b0e3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -71,6 +71,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var principal: String = null var keytab: String = null + // Kubernetes only + var kubernetesNamespace: String = null + // Standalone cluster mode only var supervise: Boolean = false var driverCores: String = null @@ -186,6 +189,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) keytab = Option(keytab).orElse(sparkProperties.get("spark.yarn.keytab")).orNull principal = Option(principal).orElse(sparkProperties.get("spark.yarn.principal")).orNull + kubernetesNamespace = Option(kubernetesNamespace) + .orElse(sparkProperties.get("spark.kubernetes.namespace")) + .orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -426,6 +432,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S case KEYTAB => keytab = value + case KUBERNETES_NAMESPACE => + kubernetesNamespace = value + case HELP => printUsageAndExit(0) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala index b30c980e95a9a..524726c2ccf92 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala @@ -19,16 +19,16 @@ package org.apache.spark.deploy.rest import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import scala.io.Source - import com.fasterxml.jackson.core.JsonProcessingException -import org.eclipse.jetty.server.{HttpConnectionFactory, Server, ServerConnector} +import org.eclipse.jetty.http.HttpVersion +import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector, SslConnectionFactory} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} import org.json4s._ import org.json4s.jackson.JsonMethods._ +import scala.io.Source -import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} +import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf, SSLOptions} import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -50,7 +50,8 @@ import org.apache.spark.util.Utils private[spark] abstract class RestSubmissionServer( val host: String, val requestedPort: Int, - val masterConf: SparkConf) extends Logging { + val masterConf: SparkConf, + val sslOptions: SSLOptions = SSLOptions()) extends Logging { protected val submitRequestServlet: SubmitRequestServlet protected val killRequestServlet: KillRequestServlet protected val statusRequestServlet: StatusRequestServlet @@ -79,19 +80,32 @@ private[spark] abstract class RestSubmissionServer( * Return a 2-tuple of the started server and the bound port. */ private def doStart(startPort: Int): (Server, Int) = { + // TODO consider using JettyUtils#startServer to do this instead val threadPool = new QueuedThreadPool threadPool.setDaemon(true) val server = new Server(threadPool) + val resolvedConnectionFactories = sslOptions + .createJettySslContextFactory() + .map(sslFactory => { + val sslConnectionFactory = new SslConnectionFactory( + sslFactory, HttpVersion.HTTP_1_1.asString()) + val rawHttpConfiguration = new HttpConfiguration() + rawHttpConfiguration.setSecureScheme("https") + rawHttpConfiguration.setSecurePort(startPort) + val rawHttpConnectionFactory = new HttpConnectionFactory(rawHttpConfiguration) + Array(sslConnectionFactory, rawHttpConnectionFactory) + }).getOrElse(Array(new HttpConnectionFactory())) + val connector = new ServerConnector( - server, - null, - // Call this full constructor to set this, which forces daemon threads: - new ScheduledExecutorScheduler("RestSubmissionServer-JettyScheduler", true), - null, - -1, - -1, - new HttpConnectionFactory()) + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("RestSubmissionServer-JettyScheduler", true), + null, + -1, + -1, + resolvedConnectionFactories: _*) connector.setHost(host) connector.setPort(startPort) server.addConnector(connector) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 92a27902c6696..f0e13aa6bf109 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -200,7 +200,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { new SecurityManager(executorConf), clientMode = true) val driver = fetcher.setupEndpointRefByURI(driverUrl) - val cfg = driver.askWithRetry[SparkAppConfig](RetrieveSparkAppConfig) + val cfg = driver.askWithRetry[SparkAppConfig](RetrieveSparkAppConfig(executorId)) val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index b766e4148e496..30df8862c3589 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -221,7 +221,7 @@ private[spark] class TaskSetManager( * Return the pending tasks list for a given host, or an empty list if * there is no map entry for that host */ - private def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { + protected def getPendingTasksForHost(host: String): ArrayBuffer[Int] = { pendingTasksForHost.getOrElse(host, ArrayBuffer()) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 0a4f19d76073e..2406999f9ee92 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -28,7 +28,7 @@ private[spark] sealed trait CoarseGrainedClusterMessage extends Serializable private[spark] object CoarseGrainedClusterMessages { - case object RetrieveSparkAppConfig extends CoarseGrainedClusterMessage + case class RetrieveSparkAppConfig(executorId: String) extends CoarseGrainedClusterMessage case class SparkAppConfig( sparkProperties: Seq[(String, String)], diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 3452487e72e88..89e59353de845 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -206,7 +206,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp removeExecutor(executorId, reason) context.reply(true) - case RetrieveSparkAppConfig => + case RetrieveSparkAppConfig(executorId) => val reply = SparkAppConfig(sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey()) context.reply(reply) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 04521c9159eac..18f7d135acdd2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -184,8 +184,14 @@ private[spark] class BlockManager( blockManagerId = if (idFromMaster != null) idFromMaster else id shuffleServerId = if (externalShuffleServiceEnabled) { - logInfo(s"external shuffle service port = $externalShuffleServicePort") - BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) + val shuffleServerHostName = if (blockManagerId.isDriver) { + blockTransferService.hostName + } else { + conf.get("spark.shuffle.service.host", blockTransferService.hostName) + } + logInfo(s"external shuffle service host = $shuffleServerHostName, " + + s"port = $externalShuffleServicePort") + BlockManagerId(executorId, shuffleServerHostName, externalShuffleServicePort) } else { blockManagerId } diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 89bfcef4d9466..97c8a38cf6143 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -8,9 +8,12 @@ aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar +automaton-1.11-8.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar +bcpkix-jdk15on-1.54.jar +bcprov-jdk15on-1.54.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.12.jar breeze_2.11-0.12.jar @@ -40,6 +43,8 @@ commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar compress-lzf-1.0.3.jar +converter-jackson-2.2.0.jar +converter-scalars-2.2.0.jar core-1.1.2.jar curator-client-2.4.0.jar curator-framework-2.4.0.jar @@ -49,6 +54,7 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar +generex-1.0.1.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -77,7 +83,11 @@ jackson-annotations-2.6.5.jar jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar jackson-databind-2.6.5.jar +jackson-dataformat-yaml-2.6.5.jar +jackson-jaxrs-base-2.6.5.jar +jackson-jaxrs-json-provider-2.6.5.jar jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.5.jar jackson-module-paranamer-2.6.5.jar jackson-module-scala_2.11-2.6.5.jar janino-3.0.0.jar @@ -96,6 +106,7 @@ jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar +jersey-media-multipart-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.7.1.jar jetty-util-6.1.26.jar @@ -111,20 +122,26 @@ jta-1.1.jar jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-3.0.3.jar +kubernetes-client-2.2.13.jar +kubernetes-model-1.0.67.jar leveldbjni-all-1.8.jar libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar +logging-interceptor-3.6.0.jar lz4-1.3.0.jar mesos-1.0.0-shaded-protobuf.jar metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar +mimepull-1.9.6.jar minlog-1.3.0.jar netty-3.8.0.Final.jar netty-all-4.0.42.Final.jar objenesis-2.1.jar +okhttp-3.6.0.jar +okio-1.11.0.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar @@ -141,6 +158,7 @@ pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.4.jar pyrolite-4.13.jar +retrofit-2.2.0.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar @@ -150,6 +168,7 @@ scalap-2.11.8.jar shapeless_2.11-2.0.0.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar snappy-0.2.jar snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar @@ -163,4 +182,5 @@ validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar +zjsonpatch-0.3.0.jar zookeeper-3.4.5.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 8df3858825e13..21ca9259ed3ff 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -9,11 +9,13 @@ aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar +automaton-1.11-8.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcpkix-jdk15on-1.54.jar +bcprov-jdk15on-1.54.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.12.jar breeze_2.11-0.12.jar @@ -42,6 +44,8 @@ commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar compress-lzf-1.0.3.jar +converter-jackson-2.2.0.jar +converter-scalars-2.2.0.jar core-1.1.2.jar curator-client-2.4.0.jar curator-framework-2.4.0.jar @@ -51,6 +55,7 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar +generex-1.0.1.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -79,7 +84,11 @@ jackson-annotations-2.6.5.jar jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar jackson-databind-2.6.5.jar +jackson-dataformat-yaml-2.6.5.jar +jackson-jaxrs-base-2.6.5.jar +jackson-jaxrs-json-provider-2.6.5.jar jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.5.jar jackson-module-paranamer-2.6.5.jar jackson-module-scala_2.11-2.6.5.jar janino-3.0.0.jar @@ -100,6 +109,7 @@ jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar +jersey-media-multipart-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar @@ -116,10 +126,13 @@ jta-1.1.jar jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-3.0.3.jar +kubernetes-client-2.2.13.jar +kubernetes-model-1.0.67.jar leveldbjni-all-1.8.jar libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar +logging-interceptor-3.6.0.jar lz4-1.3.0.jar mail-1.4.7.jar mesos-1.0.0-shaded-protobuf.jar @@ -127,11 +140,14 @@ metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar +mimepull-1.9.6.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar netty-all-4.0.42.Final.jar objenesis-2.1.jar +okhttp-3.6.0.jar +okio-1.11.0.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar @@ -148,6 +164,7 @@ pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.4.jar pyrolite-4.13.jar +retrofit-2.2.0.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar @@ -157,6 +174,7 @@ scalap-2.11.8.jar shapeless_2.11-2.0.0.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar snappy-0.2.jar snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar @@ -171,4 +189,5 @@ validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar +zjsonpatch-0.3.0.jar zookeeper-3.4.5.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 71e7fb6dd243d..f71a3cd06216c 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -9,11 +9,13 @@ aopalliance-1.0.jar aopalliance-repackaged-2.4.0-b34.jar apache-log4j-extras-1.2.17.jar arpack_combined_all-0.1.jar +automaton-1.11-8.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcpkix-jdk15on-1.54.jar +bcprov-jdk15on-1.54.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.12.jar breeze_2.11-0.12.jar @@ -42,6 +44,8 @@ commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar compress-lzf-1.0.3.jar +converter-jackson-2.2.0.jar +converter-scalars-2.2.0.jar core-1.1.2.jar curator-client-2.4.0.jar curator-framework-2.4.0.jar @@ -51,6 +55,7 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar +generex-1.0.1.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar @@ -79,7 +84,11 @@ jackson-annotations-2.6.5.jar jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar jackson-databind-2.6.5.jar +jackson-dataformat-yaml-2.6.5.jar +jackson-jaxrs-base-2.6.5.jar +jackson-jaxrs-json-provider-2.6.5.jar jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.5.jar jackson-module-paranamer-2.6.5.jar jackson-module-scala_2.11-2.6.5.jar janino-3.0.0.jar @@ -100,6 +109,7 @@ jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar +jersey-media-multipart-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar @@ -116,10 +126,13 @@ jta-1.1.jar jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-3.0.3.jar +kubernetes-client-2.2.13.jar +kubernetes-model-1.0.67.jar leveldbjni-all-1.8.jar libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar +logging-interceptor-3.6.0.jar lz4-1.3.0.jar mail-1.4.7.jar mesos-1.0.0-shaded-protobuf.jar @@ -127,11 +140,14 @@ metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar +mimepull-1.9.6.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar netty-all-4.0.42.Final.jar objenesis-2.1.jar +okhttp-3.6.0.jar +okio-1.11.0.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar @@ -148,6 +164,7 @@ pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.4.jar pyrolite-4.13.jar +retrofit-2.2.0.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar @@ -157,6 +174,7 @@ scalap-2.11.8.jar shapeless_2.11-2.0.0.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar snappy-0.2.jar snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar @@ -171,4 +189,5 @@ validation-api-1.1.0.Final.jar xbean-asm5-shaded-4.4.jar xmlenc-0.52.jar xz-1.0.jar +zjsonpatch-0.3.0.jar zookeeper-3.4.5.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index ba31391495f54..211946d583879 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -13,11 +13,13 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar +automaton-1.11-8.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcpkix-jdk15on-1.54.jar +bcprov-jdk15on-1.54.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.12.jar breeze_2.11-0.12.jar @@ -46,6 +48,8 @@ commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar compress-lzf-1.0.3.jar +converter-jackson-2.2.0.jar +converter-scalars-2.2.0.jar core-1.1.2.jar curator-client-2.6.0.jar curator-framework-2.6.0.jar @@ -55,6 +59,7 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar +generex-1.0.1.jar gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar @@ -85,8 +90,12 @@ jackson-annotations-2.6.5.jar jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar jackson-databind-2.6.5.jar +jackson-dataformat-yaml-2.6.5.jar jackson-jaxrs-1.9.13.jar +jackson-jaxrs-base-2.6.5.jar +jackson-jaxrs-json-provider-2.6.5.jar jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.5.jar jackson-module-paranamer-2.6.5.jar jackson-module-scala_2.11-2.6.5.jar jackson-xc-1.9.13.jar @@ -108,6 +117,7 @@ jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar +jersey-media-multipart-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar @@ -124,10 +134,13 @@ jta-1.1.jar jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-3.0.3.jar +kubernetes-client-2.2.13.jar +kubernetes-model-1.0.67.jar leveldbjni-all-1.8.jar libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar +logging-interceptor-3.6.0.jar lz4-1.3.0.jar mail-1.4.7.jar mesos-1.0.0-shaded-protobuf.jar @@ -135,11 +148,14 @@ metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar +mimepull-1.9.6.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar netty-all-4.0.42.Final.jar objenesis-2.1.jar +okhttp-3.6.0.jar +okio-1.11.0.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar @@ -156,6 +172,7 @@ pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.4.jar pyrolite-4.13.jar +retrofit-2.2.0.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar @@ -165,6 +182,7 @@ scalap-2.11.8.jar shapeless_2.11-2.0.0.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar snappy-0.2.jar snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar @@ -180,4 +198,5 @@ xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar xz-1.0.jar +zjsonpatch-0.3.0.jar zookeeper-3.4.6.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index b129e5a99e2ff..d0a472d3d3805 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -13,11 +13,13 @@ apacheds-kerberos-codec-2.0.0-M15.jar api-asn1-api-1.0.0-M20.jar api-util-1.0.0-M20.jar arpack_combined_all-0.1.jar +automaton-1.11-8.jar avro-1.7.7.jar avro-ipc-1.7.7.jar avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar -bcprov-jdk15on-1.51.jar +bcpkix-jdk15on-1.54.jar +bcprov-jdk15on-1.54.jar bonecp-0.8.0.RELEASE.jar breeze-macros_2.11-0.12.jar breeze_2.11-0.12.jar @@ -46,6 +48,8 @@ commons-math3-3.4.1.jar commons-net-2.2.jar commons-pool-1.5.4.jar compress-lzf-1.0.3.jar +converter-jackson-2.2.0.jar +converter-scalars-2.2.0.jar core-1.1.2.jar curator-client-2.6.0.jar curator-framework-2.6.0.jar @@ -55,6 +59,7 @@ datanucleus-core-3.2.10.jar datanucleus-rdbms-3.2.9.jar derby-10.12.1.1.jar eigenbase-properties-1.1.5.jar +generex-1.0.1.jar gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar @@ -85,8 +90,12 @@ jackson-annotations-2.6.5.jar jackson-core-2.6.5.jar jackson-core-asl-1.9.13.jar jackson-databind-2.6.5.jar +jackson-dataformat-yaml-2.6.5.jar jackson-jaxrs-1.9.13.jar +jackson-jaxrs-base-2.6.5.jar +jackson-jaxrs-json-provider-2.6.5.jar jackson-mapper-asl-1.9.13.jar +jackson-module-jaxb-annotations-2.6.5.jar jackson-module-paranamer-2.6.5.jar jackson-module-scala_2.11-2.6.5.jar jackson-xc-1.9.13.jar @@ -108,6 +117,7 @@ jersey-container-servlet-2.22.2.jar jersey-container-servlet-core-2.22.2.jar jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar +jersey-media-multipart-2.22.2.jar jersey-server-2.22.2.jar jets3t-0.9.3.jar jetty-6.1.26.jar @@ -125,10 +135,13 @@ jta-1.1.jar jtransforms-2.4.0.jar jul-to-slf4j-1.7.16.jar kryo-shaded-3.0.3.jar +kubernetes-client-2.2.13.jar +kubernetes-model-1.0.67.jar leveldbjni-all-1.8.jar libfb303-0.9.2.jar libthrift-0.9.2.jar log4j-1.2.17.jar +logging-interceptor-3.6.0.jar lz4-1.3.0.jar mail-1.4.7.jar mesos-1.0.0-shaded-protobuf.jar @@ -136,11 +149,14 @@ metrics-core-3.1.2.jar metrics-graphite-3.1.2.jar metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar +mimepull-1.9.6.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar netty-all-4.0.42.Final.jar objenesis-2.1.jar +okhttp-3.6.0.jar +okio-1.11.0.jar opencsv-2.3.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar @@ -157,6 +173,7 @@ pmml-schema-1.2.15.jar protobuf-java-2.5.0.jar py4j-0.10.4.jar pyrolite-4.13.jar +retrofit-2.2.0.jar scala-compiler-2.11.8.jar scala-library-2.11.8.jar scala-parser-combinators_2.11-1.0.4.jar @@ -166,6 +183,7 @@ scalap-2.11.8.jar shapeless_2.11-2.0.0.jar slf4j-api-1.7.16.jar slf4j-log4j12-1.7.16.jar +snakeyaml-1.15.jar snappy-0.2.jar snappy-java-1.1.2.6.jar spire-macros_2.11-0.7.4.jar @@ -181,4 +199,5 @@ xbean-asm5-shaded-4.4.jar xercesImpl-2.9.1.jar xmlenc-0.52.jar xz-1.0.jar +zjsonpatch-0.3.0.jar zookeeper-3.4.6.jar diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 6ea319e4362ab..bb5fa3da12209 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -175,6 +175,11 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/jars/* "$DISTDIR/jars/" +# Copy docker files +mkdir -p "$DISTDIR/dockerfiles" +DOCKERFILES_SRC="$SPARK_HOME/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker" +cp -R "$DOCKERFILES_SRC/." "$DISTDIR/dockerfiles/." + # Only create the yarn directory if the yarn artifacts were build. if [ -f "$SPARK_HOME"/common/network-yarn/target/scala*/spark-*-yarn-shuffle.jar ]; then mkdir "$DISTDIR"/yarn @@ -241,6 +246,7 @@ fi # Copy other things mkdir "$DISTDIR"/conf cp "$SPARK_HOME"/conf/*.template "$DISTDIR"/conf +cp "$SPARK_HOME"/conf/*.yaml "$DISTDIR"/conf cp "$SPARK_HOME/README.md" "$DISTDIR" cp -r "$SPARK_HOME/bin" "$DISTDIR" cp -r "$SPARK_HOME/python" "$DISTDIR" diff --git a/dev/scalastyle b/dev/scalastyle index f3dec833636c6..de7423913fad9 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -26,6 +26,8 @@ ERRORS=$(echo -e "q\n" \ -Pyarn \ -Phive \ -Phive-thriftserver \ + -Pkubernetes \ + -Pkubernetes-integration-tests \ scalastyle test:scalastyle \ | awk '{if($1~/error/)print}' \ ) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index b34ab51f3b996..0cc9d71d962ce 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -490,6 +490,14 @@ def __hash__(self): sbt_test_goals=["mesos/test"] ) +kubernetes = Module( + name="kubernetes", + dependencies=[], + source_file_regexes=["resource-managers/kubernetes/core"], + build_profile_flags=["-Pkubernetes"], + sbt_test_goals=["kubernetes/test"] +) + # The root module is a dummy module which is used to run all of the tests. # No other modules should directly depend on this module. root = Module( diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh index 4014f42e1983c..b1555957ecc50 100755 --- a/dev/test-dependencies.sh +++ b/dev/test-dependencies.sh @@ -29,7 +29,7 @@ export LC_ALL=C # TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution. # NOTE: These should match those in the release publishing script -HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pyarn -Phive" +HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pyarn -Phive -Pkubernetes" MVN="build/mvn" HADOOP_PROFILES=( hadoop-2.2 diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index c00d0db63cd10..3c786a6344066 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -99,6 +99,7 @@
  • Spark Standalone
  • Mesos
  • YARN
  • +
  • Kubernetes
  • diff --git a/docs/index.md b/docs/index.md index 57b9fa848f4a3..81d37aa5f63a1 100644 --- a/docs/index.md +++ b/docs/index.md @@ -113,6 +113,7 @@ options for deployment: * [Mesos](running-on-mesos.html): deploy a private cluster using [Apache Mesos](http://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) + * [Kubernetes](running-on-kubernetes.html): deploy Spark on top of Kubernetes **Other Documents:** diff --git a/docs/running-on-kubernetes-cloud.md b/docs/running-on-kubernetes-cloud.md new file mode 100644 index 0000000000000..244c64d696ab3 --- /dev/null +++ b/docs/running-on-kubernetes-cloud.md @@ -0,0 +1,24 @@ +--- +layout: global +title: Running Spark in the cloud with Kubernetes +--- + +For general information about running Spark on Kubernetes, refer to [running Spark on Kubernetes](running-on-kubernetes.md). + +A Kubernetes cluster may be brought up on different cloud providers or on premise. It is commonly provisioned through [Google Container Engine](https://cloud.google.com/container-engine/), or using [kops](https://github.com/kubernetes/kops) on AWS, or on premise using [kubeadm](https://kubernetes.io/docs/getting-started-guides/kubeadm/). + +## Running on Google Container Engine (GKE) + +* Create a GKE [container cluster](https://cloud.google.com/container-engine/docs/clusters/operations). +* Obtain kubectl and [configure](https://cloud.google.com/container-engine/docs/clusters/operations#configuring_kubectl) it appropriately. +* Find the identity of the master associated with this project. + + > kubectl cluster-info + Kubernetes master is running at https://:443 + +* Run spark-submit with the master option set to `k8s://https://:443`. The instructions for running spark-submit are provided in the [running on kubernetes](running-on-kubernetes.md) tutorial. +* Check that your driver pod, and subsequently your executor pods are launched using `kubectl get pods`. +* Read the stdout and stderr of the driver pod using `kubectl logs `, or stream the logs using `kubectl logs -f `. + +Known issues: +* If you face OAuth token expiry errors when you run spark-submit, it is likely because the token needs to be refreshed. The easiest way to fix this is to run any `kubectl` command, say, `kubectl version` and then retry your submission. diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md new file mode 100644 index 0000000000000..5e23801e15b10 --- /dev/null +++ b/docs/running-on-kubernetes.md @@ -0,0 +1,779 @@ +--- +layout: global +title: Running Spark on Kubernetes +--- + +Support for running on [Kubernetes](https://kubernetes.io/docs/whatisk8s/) is available in experimental status. The +feature set is currently limited and not well-tested. This should not be used in production environments. + +## Prerequisites + +* You must have a running Kubernetes cluster with access configured to it +using [kubectl](https://kubernetes.io/docs/user-guide/prereqs/). If you do not already have a working Kubernetes +cluster, you may setup a test cluster on your local machine using +[minikube](https://kubernetes.io/docs/getting-started-guides/minikube/). + * We recommend that minikube be updated to the most recent version (0.19.0 at the time of this documentation), as some + earlier versions may not start up the kubernetes cluster with all the necessary components. +* You must have appropriate permissions to create and list [pods](https://kubernetes.io/docs/user-guide/pods/), +[ConfigMaps](https://kubernetes.io/docs/tasks/configure-pod-container/configmap/) and +[secrets](https://kubernetes.io/docs/concepts/configuration/secret/) in your cluster. You can verify that +you can list these resources by running `kubectl get pods` `kubectl get configmap`, and `kubectl get secrets` which +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). + +## Driver & Executor Images + +Kubernetes requires users to supply images that can be deployed into containers within pods. The images are built to +be run in a container runtime environment that Kubernetes supports. Docker is a container runtime environment that is +frequently used with Kubernetes, so Spark provides some support for working with Docker to get started quickly. + +If you wish to use pre-built docker images, you may use the images published in +[kubespark](https://hub.docker.com/u/kubespark/). The images are as follows: + + + + + + + + + + + + + + + +
    ComponentImage
    Spark Driver Imagekubespark/spark-driver:v2.1.0-kubernetes-0.2.0
    Spark Executor Imagekubespark/spark-executor:v2.1.0-kubernetes-0.2.0
    Spark Initialization Imagekubespark/spark-init:v2.1.0-kubernetes-0.2.0
    + +You may also build these docker images from sources, or customize them as required. Spark distributions include the +Docker files for the base-image, driver, executor, and init-container at `dockerfiles/spark-base/Dockerfile`, `dockerfiles/driver/Dockerfile`, +`dockerfiles/executor/Dockerfile`, and `dockerfiles/init-container/Dockerfile` respectively. Use these Docker files to +build the Docker images, and then tag them with the registry that the images should be sent to. Finally, push the images +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-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 . + docker push registry-host:5000/spark-base:latest + docker push registry-host:5000/spark-driver:latest + docker push registry-host:5000/spark-executor:latest + docker push registry-host:5000/spark-init:latest + +Note that `spark-base` is the base image for the other images. It must be built first before the other images, and then afterwards the other images can be built in any order. + +## Submitting Applications to Kubernetes + +Kubernetes applications can be executed via `spark-submit`. For example, to compute the value of pi, assuming the images +are set up as described above: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://https://: \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.initcontainer.docker.image=kubespark/spark-init:v2.1.0-kubernetes-0.2.0 \ + local:///opt/spark/examples/jars/spark_examples_2.11-2.2.0.jar + +The Spark master, specified either via passing the `--master` command line argument to `spark-submit` or by setting +`spark.master` in the application's configuration, must be a URL with the format `k8s://`. Prefixing the +master string with `k8s://` will cause the Spark application to launch on the Kubernetes cluster, with the API server +being contacted at `api_server_url`. If no HTTP protocol is specified in the URL, it defaults to `https`. For example, +setting the master to `k8s://example.com:443` is equivalent to setting it to `k8s://https://example.com:443`, but to +connect without TLS on a different port, the master would be set to `k8s://http://example.com:8443`. + +If you have a Kubernetes cluster setup, one way to discover the apiserver URL is by executing `kubectl cluster-info`. + + > kubectl cluster-info + Kubernetes master is running at http://127.0.0.1:8080 + +In the above example, the specific Kubernetes cluster can be used with spark submit by specifying +`--master k8s://http://127.0.0.1:8080` as an argument to spark-submit. + +Note that applications can currently only be executed in cluster mode, where the driver and its executors are running on +the cluster. + +Finally, notice that in the above example we specify a jar with a specific URI with a scheme of `local://`. This URI is +the location of the example jar that is already in the Docker image. Using dependencies that are on your machine's local +disk is discussed below. + +## Dependency Management + +Application dependencies that are being submitted from your machine need to be sent to a **resource staging server** +that the driver and executor can then communicate with to retrieve those dependencies. A YAML file denoting a minimal +set of Kubernetes resources that runs this service is located in the file `conf/kubernetes-resource-staging-server.yaml`. +This YAML file configures a Deployment with one pod running the resource staging server configured with a ConfigMap, +and exposes the server through a Service with a fixed NodePort. Deploying a resource staging server with the included +YAML file requires you to have permissions to create Deployments, Services, and ConfigMaps. + +To run the resource staging server with default configurations, the Kubernetes resources can be created: + + kubectl create -f conf/kubernetes-resource-staging-server.yaml + +and then you can compute the value of Pi as follows: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://: \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.initcontainer.docker.image=kubespark/spark-init:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.resourceStagingServer.uri=http://:31000 \ + examples/jars/spark_examples_2.11-2.2.0.jar + +The Docker image for the resource staging server may also be built from source, in a similar manner to the driver +and executor images. The Dockerfile is provided in `dockerfiles/resource-staging-server/Dockerfile`. + +The provided YAML file specifically sets the NodePort to 31000 on the service's specification. If port 31000 is not +available on any of the nodes of your cluster, you should remove the NodePort field from the service's specification +and allow the Kubernetes cluster to determine the NodePort itself. Be sure to provide the correct port in the resource +staging server URI when submitting your application, in accordance to the NodePort chosen by the Kubernetes cluster. + +### Dependency Management Without The Resource Staging Server + +Note that this resource staging server is only required for submitting local dependencies. If your application's +dependencies are all hosted in remote locations like HDFS or http servers, they may be referred to by their appropriate +remote URIs. Also, application dependencies can be pre-mounted into custom-built Docker images. Those dependencies +can be added to the classpath by referencing them with `local://` URIs and/or setting the `SPARK_EXTRA_CLASSPATH` +environment variable in your Dockerfiles. + +### Accessing Kubernetes Clusters + +Spark-submit also supports submission through the +[local kubectl proxy](https://kubernetes.io/docs/user-guide/accessing-the-cluster/#using-kubectl-proxy). One can use the +authenticating proxy to communicate with the api server directly without passing credentials to spark-submit. + +The local proxy can be started by running: + + kubectl proxy + +If our local proxy were listening on port 8001, we would have our submission looking like the following: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://http://127.0.0.1:8001 \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.initcontainer.docker.image=kubespark/spark-init:v2.1.0-kubernetes-0.2.0 \ + local:///opt/spark/examples/jars/spark_examples_2.11-2.2.0.jar + +Communication between Spark and Kubernetes clusters is performed using the fabric8 kubernetes-client library. +The above mechanism using `kubectl proxy` can be used when we have authentication providers that the fabric8 +kubernetes-client library does not support. Authentication using X509 Client Certs and OAuth tokens +is currently supported. + +### Running PySpark + +Running PySpark on Kubernetes leverages the same spark-submit logic when launching on Yarn and Mesos. +Python files can be distributed by including, in the conf, `--py-files` + +Below is an example submission: + + +``` + bin/spark-submit \ + --deploy-mode cluster \ + --master k8s://http://127.0.0.1:8001 \ + --kubernetes-namespace default \ + --conf spark.executor.memory=500m \ + --conf spark.driver.memory=1G \ + --conf spark.driver.cores=1 \ + --conf spark.executor.cores=1 \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=spark-driver-py:latest \ + --conf spark.kubernetes.executor.docker.image=spark-executor-py:latest \ + --conf spark.kubernetes.initcontainer.docker.image=spark-init:latest \ + --py-files local:///opt/spark/examples/src/main/python/sort.py \ + local:///opt/spark/examples/src/main/python/pi.py 100 +``` + +## Dynamic Executor Scaling + +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/) +with a provisioned [hostpath](https://kubernetes.io/docs/concepts/storage/volumes/#hostpath) volume. +This shuffle service may be shared by executors belonging to different SparkJobs. Using Spark with dynamic allocation +on Kubernetes assumes that a cluster administrator has set up one or more shuffle-service daemonsets in the cluster. + +A sample configuration file is provided in `conf/kubernetes-shuffle-service.yaml` which can be customized as needed +for a particular cluster. It is important to note that `spec.template.metadata.labels` are setup appropriately for the shuffle +service because there may be multiple shuffle service instances running in a cluster. The labels give Spark applications +a way to target a particular shuffle service. + +For example, if the shuffle service we want to use is in the default namespace, and +has pods with labels `app=spark-shuffle-service` and `spark-version=2.1.0`, we can +use those tags to target that particular shuffle service at job launch time. In order to run a job with dynamic allocation enabled, +the command may then look like the following: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.GroupByTest \ + --master k8s://: \ + --kubernetes-namespace default \ + --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 \ + --conf spark.dynamicAllocation.enabled=true \ + --conf spark.shuffle.service.enabled=true \ + --conf spark.kubernetes.shuffle.namespace=default \ + --conf spark.kubernetes.shuffle.labels="app=spark-shuffle-service,spark-version=2.1.0" \ + local:///opt/spark/examples/jars/spark_examples_2.11-2.2.0.jar 10 400000 2 + +## Advanced + +### Securing the Resource Staging Server with TLS + +The default configuration of the resource staging server is not secured with TLS. It is highly recommended to configure +this to protect the secrets and jars/files being submitted through the staging server. + +The YAML file in `conf/kubernetes-resource-staging-server.yaml` includes a ConfigMap resource that holds the resource +staging server's configuration. The properties can be adjusted here to make the resource staging server listen over TLS. +Refer to the [security](security.html) page for the available settings related to TLS. The namespace for the +resource staging server is `kubernetes.resourceStagingServer`, so for example the path to the server's keyStore would +be set by `spark.ssl.kubernetes.resourceStagingServer.keyStore`. + +In addition to the settings specified by the previously linked security page, the resource staging server supports the +following additional configurations: + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.ssl.kubernetes.resourceStagingServer.keyPem(none) + Private key file encoded in PEM format that the resource staging server uses to secure connections over TLS. If this + is specified, the associated public key file must be specified in + spark.ssl.kubernetes.resourceStagingServer.serverCertPem. PEM files and a keyStore file (set by + spark.ssl.kubernetes.resourceStagingServer.keyStore) cannot both be specified at the same time. +
    spark.ssl.kubernetes.resourceStagingServer.serverCertPem(none) + Certificate file encoded in PEM format that the resource staging server uses to secure connections over TLS. If this + is specified, the associated private key file must be specified in + spark.ssl.kubernetes.resourceStagingServer.keyPem. PEM files and a keyStore file (set by + spark.ssl.kubernetes.resourceStagingServer.keyStore) cannot both be specified at the same time. +
    spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile(none) + Provides the KeyStore password through a file in the container instead of a static value. This is useful if the + keyStore's password is to be mounted into the container with a secret. +
    spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile(none) + Provides the keyStore's key password using a file in the container instead of a static value. This is useful if the + keyStore's key password is to be mounted into the container with a secret. +
    + +Note that while the properties can be set in the ConfigMap, you will still need to consider the means of mounting the +appropriate secret files into the resource staging server's container. A common mechanism that is used for this is +to use [Kubernetes secrets](https://kubernetes.io/docs/concepts/configuration/secret/) that are mounted as secret +volumes. Refer to the appropriate Kubernetes documentation for guidance and adjust the resource staging server's +specification in the provided YAML file accordingly. + +Finally, when you submit your application, you must specify either a trustStore or a PEM-encoded certificate file to +communicate with the resource staging server over TLS. The trustStore can be set with +`spark.ssl.kubernetes.resourceStagingServer.trustStore`, or a certificate file can be set with +`spark.ssl.kubernetes.resourceStagingServer.clientCertPem`. For example, our SparkPi example now looks like this: + + bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.SparkPi \ + --master k8s://https://: \ + --kubernetes-namespace default \ + --conf spark.executor.instances=5 \ + --conf spark.app.name=spark-pi \ + --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.initcontainer.docker.image=kubespark/spark-init:v2.1.0-kubernetes-0.2.0 \ + --conf spark.kubernetes.resourceStagingServer.uri=https://:31000 \ + --conf spark.ssl.kubernetes.resourceStagingServer.enabled=true \ + --conf spark.ssl.kubernetes.resourceStagingServer.clientCertPem=/home/myuser/cert.pem \ + examples/jars/spark_examples_2.11-2.2.0.jar + +### Spark Properties + +Below are some other common properties that are specific to Kubernetes. Most of the other configurations are the same +from the other deployment modes. See the [configuration page](configuration.html) for more information on those. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.kubernetes.namespacedefault + The namespace that will be used for running the driver and executor pods. When using + spark-submit in cluster mode, this can also be passed to spark-submit via the + --kubernetes-namespace command line argument. +
    spark.kubernetes.driver.docker.imagespark-driver:2.2.0 + Docker image to use for the driver. Specify this using the standard + Docker tag format. +
    spark.kubernetes.executor.docker.imagespark-executor:2.2.0 + Docker image to use for the executors. Specify this using the standard + Docker tag format. +
    spark.kubernetes.initcontainer.docker.imagespark-init:2.2.0 + Docker image to use for the init-container that is run before the driver and executor containers. Specify this using + the standard Docker tag format. The + init-container is responsible for fetching application dependencies from both remote locations like HDFS or S3, + and from the resource staging server, if applicable. +
    spark.kubernetes.shuffle.namespacedefault + Namespace in which the shuffle service pods are present. The shuffle service must be + created in the cluster prior to attempts to use it. +
    spark.kubernetes.shuffle.labels(none) + Labels that will be used to look up shuffle service pods. This should be a comma-separated list of label key-value pairs, + where each label is in the format key=value. The labels chosen must be such that + they match exactly one shuffle service pod on each node that executors are launched. +
    spark.kubernetes.allocation.batch.size5 + Number of pods to launch at once in each round of executor pod allocation. +
    spark.kubernetes.allocation.batch.delay1 + Number of seconds to wait between each round of executor pod allocation. +
    spark.kubernetes.authenticate.submission.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS when starting the driver. This file + must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide + a scheme). +
    spark.kubernetes.authenticate.submission.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server when starting the driver. This file + must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not provide + a scheme). +
    spark.kubernetes.authenticate.submission.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server when starting the driver. This + file must be located on the submitting machine's disk. Specify this as a path as opposed to a URI (i.e. do not + provide a scheme). +
    spark.kubernetes.authenticate.submission.oauthToken(none) + OAuth token to use when authenticating against the Kubernetes API server when starting the driver. Note + that unlike the other authentication options, this is expected to be the exact string value of the token to use for + the authentication. +
    spark.kubernetes.authenticate.driver.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS from the driver pod when requesting + executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server from the driver pod when requesting + executors. This file must be located on the submitting machine's disk, and will be uploaded to the driver pod. + Specify this as a path as opposed to a URI (i.e. do not provide a scheme). If this is specified, it is highly + recommended to set up TLS for the driver submission server, as this value is sensitive information that would be + passed to the driver pod in plaintext otherwise. +
    spark.kubernetes.authenticate.driver.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server from the driver pod when + requesting executors. This file must be located on the submitting machine's disk, and will be uploaded to the + driver pod. Specify this as a path as opposed to a URI (i.e. do not provide a scheme). +
    spark.kubernetes.authenticate.driver.oauthToken(none) + OAuth token to use when authenticating against the against the Kubernetes API server from the driver pod when + requesting executors. Note that unlike the other authentication options, this must be the exact string value of + the token to use for the authentication. This token value is uploaded to the driver pod. If this is specified, it is + highly recommended to set up TLS for the driver submission server, as this value is sensitive information that would + be passed to the driver pod in plaintext otherwise. +
    spark.kubernetes.authenticate.driver.serviceAccountNamedefault + Service account that is used when running the driver pod. The driver pod uses this service account when requesting + executor pods from the API server. Note that this cannot be specified alongside a CA cert file, client key file, + client cert file, and/or OAuth token. +
    spark.kubernetes.authenticate.resourceStagingServer.caCertFile(none) + Path to the CA cert file for connecting to the Kubernetes API server over TLS from the resource staging server when + it monitors objects in determining when to clean up resource bundles. +
    spark.kubernetes.authenticate.resourceStagingServer.clientKeyFile(none) + Path to the client key file for authenticating against the Kubernetes API server from the resource staging server + when it monitors objects in determining when to clean up resource bundles. The resource staging server must have + credentials that allow it to view API objects in any namespace. +
    spark.kubernetes.authenticate.resourceStagingServer.clientCertFile(none) + Path to the client cert file for authenticating against the Kubernetes API server from the resource staging server + when it monitors objects in determining when to clean up resource bundles. The resource staging server must have + credentials that allow it to view API objects in any namespace. +
    spark.kubernetes.authenticate.resourceStagingServer.oauthToken(none) + OAuth token value for authenticating against the Kubernetes API server from the resource staging server + when it monitors objects in determining when to clean up resource bundles. The resource staging server must have + credentials that allow it to view API objects in any namespace. Note that this cannot be set at the same time as + spark.kubernetes.authenticate.resourceStagingServer.oauthTokenFile. +
    spark.kubernetes.authenticate.resourceStagingServer.oauthTokenFile(none) + File containing the OAuth token to use when authenticating against the against the Kubernetes API server from the + resource staging server, when it monitors objects in determining when to clean up resource bundles. The resource + staging server must have credentials that allow it to view API objects in any namespace. Note that this cannot be + set at the same time as spark.kubernetes.authenticate.resourceStagingServer.oauthToken. +
    spark.kubernetes.authenticate.resourceStagingServer.useServiceAccountCredentialstrue + Whether or not to use a service account token and a service account CA certificate when the resource staging server + authenticates to Kubernetes. If this is set, interactions with Kubernetes will authenticate using a token located at + /var/run/secrets/kubernetes.io/serviceaccount/token and the CA certificate located at + /var/run/secrets/kubernetes.io/serviceaccount/ca.crt. Note that if + spark.kubernetes.authenticate.resourceStagingServer.oauthTokenFile is set, it takes precedence + over the usage of the service account token file. Also, if + spark.kubernetes.authenticate.resourceStagingServer.caCertFile is set, it takes precedence over using + the service account's CA certificate file. This generally should be set to true (the default value) when the + resource staging server is deployed as a Kubernetes pod, but should be set to false if the resource staging server + is deployed by other means (i.e. when running the staging server process outside of Kubernetes). The resource + staging server must have credentials that allow it to view API objects in any namespace. +
    spark.kubernetes.executor.memoryOverheadexecutorMemory * 0.10, with minimum of 384 + The amount of off-heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things + like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size + (typically 6-10%). +
    spark.kubernetes.driver.label.[labelKey](none) + Adds a label to the driver pod, with key labelKey and the value as the configuration's value. For + example, setting spark.kubernetes.driver.label.identifier to myIdentifier will result in + the driver pod having a label with key identifier and value myIdentifier. Multiple labels + can be added by setting multiple configurations with this prefix. +
    spark.kubernetes.driver.annotation.[annotationKey](none) + Adds an annotation to the driver pod, with key annotationKey and the value as the configuration's + value. For example, setting spark.kubernetes.driver.annotation.identifier to myIdentifier + will result in the driver pod having an annotation with key identifier and value + myIdentifier. Multiple annotations can be added by setting multiple configurations with this prefix. +
    spark.kubernetes.executor.label.[labelKey](none) + Adds a label to all executor pods, with key labelKey and the value as the configuration's value. For + example, setting spark.kubernetes.executor.label.identifier to myIdentifier will result in + the executor pods having a label with key identifier and value myIdentifier. Multiple + labels can be added by setting multiple configurations with this prefix. +
    spark.kubernetes.executor.annotation.[annotationKey](none) + Adds an annotation to the executor pods, with key annotationKey and the value as the configuration's + value. For example, setting spark.kubernetes.executor.annotation.identifier to myIdentifier + will result in the executor pods having an annotation with key identifier and value + 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) + Name of the driver pod. If not set, the driver pod name is set to "spark.app.name" suffixed by the current timestamp + to avoid name conflicts. +
    spark.kubernetes.submission.waitAppCompletiontrue + In cluster mode, whether to wait for the application to finish before exiting the launcher process. When changed to + false, the launcher has a "fire-and-forget" behavior when launching the Spark job. +
    spark.kubernetes.resourceStagingServer.port10000 + Port for the resource staging server to listen on when it is deployed. +
    spark.kubernetes.resourceStagingServer.uri(none) + URI of the resource staging server that Spark should use to distribute the application's local dependencies. Note + that by default, this URI must be reachable by both the submitting machine and the pods running in the cluster. If + one URI is not simultaneously reachable both by the submitter and the driver/executor pods, configure the pods to + access the staging server at a different URI by setting + spark.kubernetes.resourceStagingServer.internal.uri as discussed below. +
    spark.kubernetes.resourceStagingServer.internal.uriValue of spark.kubernetes.resourceStagingServer.uri + URI of the resource staging server to communicate with when init-containers bootstrap the driver and executor pods + with submitted local dependencies. Note that this URI must by the pods running in the cluster. This is useful to + set if the resource staging server has a separate "internal" URI that must be accessed by components running in the + cluster. +
    spark.ssl.kubernetes.resourceStagingServer.internal.trustStoreValue of spark.ssl.kubernetes.resourceStagingServer.trustStore + Location of the trustStore file to use when communicating with the resource staging server over TLS, as + init-containers bootstrap the driver and executor pods with submitted local dependencies. This can be a URI with a + scheme of local://, which denotes that the file is pre-mounted on the pod's disk. A uri without a + scheme or a scheme of file:// will result in this file being mounted from the submitting machine's + disk as a secret into the init-containers. +
    spark.ssl.kubernetes.resourceStagingServer.internal.trustStorePasswordValue of spark.ssl.kubernetes.resourceStagingServer.trustStorePassword + Password of the trustStore file that is used when communicating with the resource staging server over TLS, as + init-containers bootstrap the driver and executor pods with submitted local dependencies. +
    spark.ssl.kubernetes.resourceStagingServer.internal.trustStoreTypeValue of spark.ssl.kubernetes.resourceStagingServer.trustStoreType + Type of the trustStore file that is used when communicating with the resource staging server over TLS, when + init-containers bootstrap the driver and executor pods with submitted local dependencies. +
    spark.ssl.kubernetes.resourceStagingServer.internal.clientCertPemValue of spark.ssl.kubernetes.resourceStagingServer.clientCertPem + Location of the certificate file to use when communicating with the resource staging server over TLS, as + init-containers bootstrap the driver and executor pods with submitted local dependencies. This can be a URI with a + scheme of local://, which denotes that the file is pre-mounted on the pod's disk. A uri without a + scheme or a scheme of file:// will result in this file being mounted from the submitting machine's + disk as a secret into the init-containers. +
    spark.kubernetes.mountdependencies.jarsDownloadDir/var/spark-data/spark-jars + Location to download jars to in the driver and executors. This will be mounted as an empty directory volume + into the driver and executor containers. +
    spark.kubernetes.mountdependencies.filesDownloadDir/var/spark-data/spark-files + Location to download files to in the driver and executors. This will be mounted as an empty directory volume + into the driver and executor containers. +
    spark.kubernetes.report.interval1s + Interval between reports of the current Spark job status in cluster mode. +
    spark.kubernetes.docker.image.pullPolicyIfNotPresent + Docker image pull policy used when pulling Docker images with Kubernetes. +
    spark.kubernetes.driver.limit.cores(none) + Specify the hard cpu limit for the driver pod +
    spark.kubernetes.executor.limit.cores(none) + Specify the hard cpu limit for a single executor pod +
    spark.kubernetes.node.selector.[labelKey](none) + Adds to the node selector of the driver pod and executor pods, with key labelKey and the value as the + configuration's value. For example, setting spark.kubernetes.node.selector.identifier to myIdentifier + will result in the driver pod and executors having a node selector with key identifier and value + myIdentifier. Multiple node selector keys can be added by setting multiple configurations with this prefix. +
    + + +## Current Limitations + +Running Spark on Kubernetes is currently an experimental feature. Some restrictions on the current implementation that +should be lifted in the future include: +* Applications can only run in cluster mode. +* Only Scala and Java applications can be run. diff --git a/examples/pom.xml b/examples/pom.xml index 2fb42413aca81..f301aa595f256 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 4061c5f089c54..f00f67c145740 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cfc47ef00e2a..2fcaa114555d5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 58caf35f65a16..5378d1e1dbdf2 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index ed32fc0ec4c18..2c96f56c17534 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index a3f3907573f21..013c95c7e691e 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 9ae4461db64a2..c2634c5ca7e5d 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index f7276d0bd2197..bc0fb963e2496 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 52c88150137e3..9ed8511d9cba4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 93b49bcf615b6..69f5d81a97038 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index cdfd29e3a9208..cd2d489fa1e26 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index c6a79aa86bcf0..74a4f779c255a 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 3fa28aa81f214..622d6bf366228 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 5c828780600cd..c72fed52c00f3 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 1818bc80ea78a..dc0f29ea2ca32 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index d60a633b87699..f6cac3262c756 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 6767cc5079649..a4d43c0795abc 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,6 +76,9 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; + // Kubernetes-only options. + protected final String KUBERNETES_NAMESPACE = "--kubernetes-namespace"; + /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" @@ -115,6 +118,7 @@ class SparkSubmitOptionParser { { REPOSITORIES }, { STATUS }, { TOTAL_EXECUTOR_CORES }, + { KUBERNETES_NAMESPACE } }; /** diff --git a/mesos/pom.xml b/mesos/pom.xml index f8e43d2c43ec2..36b1d08bb4516 100644 --- a/mesos/pom.xml +++ b/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala index 859aa836a3157..cbb03c7d3b1d6 100644 --- a/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala +++ b/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosExternalShuffleService.scala @@ -29,7 +29,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler import org.apache.spark.network.shuffle.protocol.BlockTransferMessage -import org.apache.spark.network.shuffle.protocol.mesos.{RegisterDriver, ShuffleServiceHeartbeat} +import org.apache.spark.network.shuffle.protocol.RegisterDriver +import org.apache.spark.network.shuffle.protocol.mesos.ShuffleServiceHeartbeat import org.apache.spark.network.util.TransportConf import org.apache.spark.util.ThreadUtils diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 6dcb44cebb254..4b0712df44367 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 5cf3a7f3e0f5e..bbe0af191952f 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 49f12703c04df..6ec3d7e0bcf80 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -136,6 +136,8 @@ 10.12.1.1 1.8.1 1.6.0 + 2.2.0 + 1.54 9.2.16.v20160414 3.1.0 0.8.0 @@ -222,6 +224,7 @@ 64m 512m 512m + @@ -303,6 +306,60 @@ chill-java ${chill.version} + + + com.squareup.retrofit2 + retrofit + ${retrofit.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + + com.squareup.retrofit2 + converter-jackson + ${retrofit.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + + + com.squareup.retrofit2 + converter-scalars + ${retrofit.version} + + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} + + + org.bouncycastle + bcprov-jdk15on + ${bouncycastle.version} + @@ -599,6 +656,11 @@ jackson-annotations ${fasterxml.jackson.version} + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${fasterxml.jackson.version} + @@ -617,6 +679,11 @@ jackson-module-jaxb-annotations ${fasterxml.jackson.version} + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + ${fasterxml.jackson.version} + org.glassfish.jersey.core jersey-server @@ -647,6 +714,11 @@ jersey-client ${jersey.version} + + org.glassfish.jersey.media + jersey-media-multipart + ${jersey.version} + javax.ws.rs javax.ws.rs-api @@ -2052,7 +2124,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} ${extraScalaTestArgs} + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.3.1-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes_2.11 + jar + Spark Project Kubernetes + + kubernetes + 2.2.13 + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + + io.fabric8 + kubernetes-client + ${kubernetes.client.version} + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + + + + + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + ${fasterxml.jackson.version} + + + org.glassfish.jersey.containers + jersey-container-servlet + + + org.glassfish.jersey.media + jersey-media-multipart + + + com.squareup.retrofit2 + retrofit + + + com.squareup.retrofit2 + converter-jackson + + + com.squareup.retrofit2 + converter-scalars + + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + javax.ws.rs + javax.ws.rs-api + + + + com.google.guava + guava + + + + + org.bouncycastle + bcpkix-jdk15on + + + org.bouncycastle + bcprov-jdk15on + + + org.mockito + mockito-core + test + + + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + 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 new file mode 100644 index 0000000000000..55e7e38b28a08 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -0,0 +1 @@ +org.apache.spark.scheduler.cluster.kubernetes.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/kubernetes/CompressionUtils.scala new file mode 100644 index 0000000000000..a6f0ca502f6f0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala @@ -0,0 +1,96 @@ +/* + * 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 + +import java.io.{File, FileInputStream, FileOutputStream, InputStream, OutputStream} +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import com.google.common.io.Files +import org.apache.commons.compress.archivers.tar.{TarArchiveEntry, TarArchiveInputStream, TarArchiveOutputStream} +import org.apache.commons.compress.utils.CharsetNames +import org.apache.commons.io.IOUtils +import scala.collection.mutable + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] object CompressionUtils extends Logging { + // Defaults from TarArchiveOutputStream + private val BLOCK_SIZE = 10240 + private val RECORD_SIZE = 512 + private val ENCODING = CharsetNames.UTF_8 + + def writeTarGzipToStream(outputStream: OutputStream, paths: Iterable[String]): Unit = { + Utils.tryWithResource(new GZIPOutputStream(outputStream)) { gzipping => + Utils.tryWithResource(new TarArchiveOutputStream( + gzipping, + BLOCK_SIZE, + RECORD_SIZE, + ENCODING)) { tarStream => + val usedFileNames = mutable.HashSet.empty[String] + for (path <- paths) { + val file = new File(path) + if (!file.isFile) { + throw new IllegalArgumentException(s"Cannot add $path to tarball; either does" + + s" not exist or is a directory.") + } + var resolvedFileName = file.getName + val extension = Files.getFileExtension(file.getName) + val nameWithoutExtension = Files.getNameWithoutExtension(file.getName) + var deduplicationCounter = 1 + while (usedFileNames.contains(resolvedFileName)) { + val oldResolvedFileName = resolvedFileName + resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" + logWarning(s"File with name $oldResolvedFileName already exists. Trying to add" + + s" with file name $resolvedFileName instead.") + deduplicationCounter += 1 + } + usedFileNames += resolvedFileName + val tarEntry = new TarArchiveEntry(resolvedFileName) + tarEntry.setSize(file.length()); + tarStream.putArchiveEntry(tarEntry) + Utils.tryWithResource(new FileInputStream(file)) { fileInput => + IOUtils.copy(fileInput, tarStream) + } + tarStream.closeArchiveEntry() + } + } + } + } + + def unpackTarStreamToDirectory(inputStream: InputStream, outputDir: File): Seq[String] = { + val paths = mutable.Buffer.empty[String] + Utils.tryWithResource(new GZIPInputStream(inputStream)) { gzipped => + Utils.tryWithResource(new TarArchiveInputStream( + gzipped, + BLOCK_SIZE, + RECORD_SIZE, + ENCODING)) { tarInputStream => + var nextTarEntry = tarInputStream.getNextTarEntry + while (nextTarEntry != null) { + val outputFile = new File(outputDir, nextTarEntry.getName) + Utils.tryWithResource(new FileOutputStream(outputFile)) { fileOutputStream => + IOUtils.copy(tarInputStream, fileOutputStream) + } + paths += outputFile.getAbsolutePath + nextTarEntry = tarInputStream.getNextTarEntry + } + } + } + paths + } +} 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/kubernetes/ConfigurationUtils.scala new file mode 100644 index 0000000000000..1a008c236d00f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala @@ -0,0 +1,82 @@ +/* + * 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 + +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( + maybeKeyValues: Option[String], + configKey: String, + keyValueType: String): Map[String, String] = { + + maybeKeyValues.map(keyValues => { + keyValues.split(",").map(_.trim).filterNot(_.isEmpty).map(keyValue => { + keyValue.split("=", 2).toSeq match { + case Seq(k, v) => + (k, v) + case _ => + throw new SparkException(s"Custom $keyValueType set by $configKey must be a" + + s" comma-separated list of key-value pairs, with format =." + + s" Got value: $keyValue. All values: $keyValues") + } + }).toMap + }).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, + configType: String): Map[String, String] = { + val fromPrefix = sparkConf.getAllWithPrefix(prefix) + fromPrefix.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") + } + fromPrefix.toMap + } +} 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/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala new file mode 100644 index 0000000000000..265b8f197a102 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala @@ -0,0 +1,66 @@ +/* + * 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 + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} + +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait InitContainerResourceStagingServerSecretPlugin { + + /** + * Configure the init-container to mount the secret files that allow it to retrieve dependencies + * from a resource staging server. + */ + def mountResourceStagingServerSecretIntoInitContainer( + initContainer: Container): Container + + /** + * Configure the pod to attach a Secret volume which hosts secret files allowing the + * init-container to retrieve dependencies from the resource staging server. + */ + def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod +} + +private[spark] class InitContainerResourceStagingServerSecretPluginImpl( + initContainerSecretName: String, + initContainerSecretMountPath: String) + extends InitContainerResourceStagingServerSecretPlugin { + + override def mountResourceStagingServerSecretIntoInitContainer( + initContainer: Container): Container = { + new ContainerBuilder(initContainer) + .addNewVolumeMount() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withMountPath(initContainerSecretMountPath) + .endVolumeMount() + .build() + } + + override def addResourceStagingServerSecretVolumeToPod(basePod: Pod): Pod = { + new PodBuilder(basePod) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(initContainerSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + } +} 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/kubernetes/KubernetesCredentials.scala new file mode 100644 index 0000000000000..aba94e6969529 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala @@ -0,0 +1,23 @@ +/* + * 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 + +case class KubernetesCredentials( + oauthTokenBase64: Option[String], + caCertDataBase64: Option[String], + clientKeyDataBase64: Option[String], + clientCertDataBase64: 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/kubernetes/KubernetesExternalShuffleService.scala new file mode 100644 index 0000000000000..c61f4f1d44acf --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala @@ -0,0 +1,189 @@ +/* + * 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 + +import java.io.File +import java.nio.ByteBuffer + +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{Config, KubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.commons.io.IOUtils +import scala.collection.JavaConverters._ +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.internal.Logging +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler +import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterDriver} +import org.apache.spark.network.util.TransportConf + +/** + * An RPC endpoint that receives registration requests from Spark drivers running on Kubernetes. + * It detects driver termination and calls the cleanup callback to [[ExternalShuffleService]]. + */ +private[spark] class KubernetesShuffleBlockHandler ( + transportConf: TransportConf, + kubernetesClient: KubernetesClient) + extends ExternalShuffleBlockHandler(transportConf, null) with Logging { + + private val INIT_AND_STOP_LOCK = new Object + private val CONNECTED_APPS_LOCK = new Object + private val connectedApps = mutable.Set.empty[String] + private var shuffleWatch: Option[Watch] = None + + def start(): Unit = INIT_AND_STOP_LOCK.synchronized { + shuffleWatch = startShuffleWatcher() + } + + override def close(): Unit = { + try { + super.close() + } finally { + INIT_AND_STOP_LOCK.synchronized { + shuffleWatch.foreach(IOUtils.closeQuietly) + shuffleWatch = None + IOUtils.closeQuietly(kubernetesClient) + } + } + } + + protected override def handleMessage( + message: BlockTransferMessage, + client: TransportClient, + callback: RpcResponseCallback): Unit = { + message match { + case RegisterDriverParam(appId) => + val address = client.getSocketAddress + logInfo(s"Received registration request from app $appId (remote address $address).") + CONNECTED_APPS_LOCK.synchronized { + if (connectedApps.contains(appId)) { + logWarning(s"Received a registration request from app $appId, but it was already " + + s"registered") + } + connectedApps += appId + } + callback.onSuccess(ByteBuffer.allocate(0)) + case _ => super.handleMessage(message, client, callback) + } + } + + private def startShuffleWatcher(): Option[Watch] = { + try { + Some(kubernetesClient + .pods() + .withLabels(Map(SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE).asJava) + .watch(new Watcher[Pod] { + override def eventReceived(action: Watcher.Action, p: Pod): Unit = { + action match { + case Action.DELETED | Action.ERROR => + val labels = p.getMetadata.getLabels + if (labels.containsKey(SPARK_APP_ID_LABEL)) { + val appId = labels.get(SPARK_APP_ID_LABEL) + CONNECTED_APPS_LOCK.synchronized { + if (connectedApps.contains(appId)) { + connectedApps -= appId + applicationRemoved(appId, true) + } + } + } + case Action.ADDED | Action.MODIFIED => + } + } + + override def onClose(e: KubernetesClientException): Unit = {} + })) + } catch { + case throwable: Throwable => + logWarning(s"Shuffle service cannot access Kubernetes. " + + s"Orphaned file cleanup is disabled.", throwable) + None + } + } + + /** An extractor object for matching [[RegisterDriver]] message. */ + private object RegisterDriverParam { + def unapply(r: RegisterDriver): Option[(String)] = + Some(r.getAppId) + } +} + +/** + * A wrapper of [[ExternalShuffleService]] that provides an additional endpoint for drivers + * to associate with. This allows the shuffle service to detect when a driver is terminated + * and can clean up the associated shuffle files. + */ +private[spark] class KubernetesExternalShuffleService( + conf: SparkConf, + securityManager: SecurityManager) + extends ExternalShuffleService(conf, securityManager) { + + private var shuffleBlockHandlers: mutable.Buffer[KubernetesShuffleBlockHandler] = _ + protected override def newShuffleBlockHandler( + tConf: TransportConf): ExternalShuffleBlockHandler = { + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( + conf.get(KUBERNETES_SHUFFLE_APISERVER_URI), + None, + APISERVER_AUTH_SHUFFLE_SERVICE_CONF_PREFIX, + conf, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)) + .filter( _ => conf.get(KUBERNETES_SHUFFLE_USE_SERVICE_ACCOUNT_CREDENTIALS)), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)) + .filter( _ => conf.get(KUBERNETES_SHUFFLE_USE_SERVICE_ACCOUNT_CREDENTIALS))) + val newBlockHandler = new KubernetesShuffleBlockHandler(tConf, kubernetesClient) + try { + newBlockHandler.start() + // TODO: figure out a better way of doing this. + // This is necessary because the constructor is not called + // when this class is initialized through ExternalShuffleService. + if (shuffleBlockHandlers == null) { + shuffleBlockHandlers = mutable.Buffer.empty[KubernetesShuffleBlockHandler] + } + shuffleBlockHandlers += newBlockHandler + newBlockHandler + } catch { + case e: Throwable => + logError("Failed to create Kubernetes shuffle block handler.", e) + newBlockHandler.close() + throw e + } + } + + override def stop(): Unit = { + try { + super.stop() + } finally { + if (shuffleBlockHandlers != null) { + shuffleBlockHandlers.foreach(_.close()) + } + } + } +} + +private[spark] object KubernetesExternalShuffleService extends Logging { + def main(args: Array[String]): Unit = { + ExternalShuffleService.main(args, + (conf: SparkConf, sm: SecurityManager) => new KubernetesExternalShuffleService(conf, sm)) + } +} + + 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/kubernetes/OptionRequirements.scala new file mode 100644 index 0000000000000..eda43de0a9a5b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala @@ -0,0 +1,40 @@ +/* + * 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 + +private[spark] object OptionRequirements { + + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], opt2: Option[_], errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } + } + + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { + opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + } +} 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/kubernetes/PodWithDetachedInitContainer.scala new file mode 100644 index 0000000000000..36b1b07dc6bc4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala @@ -0,0 +1,24 @@ +/* + * 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 + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] case class PodWithDetachedInitContainer( + pod: Pod, + initContainer: Container, + mainContainer: Container) 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/kubernetes/SparkKubernetesClientFactory.scala new file mode 100644 index 0000000000000..d2729a2db2fa0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala @@ -0,0 +1,103 @@ +/* + * 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 + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient, KubernetesClient} +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.util.ThreadUtils + +/** + * Spark-opinionated builder for Kubernetes clients. It uses a prefix plus common suffixes to + * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL + * options for different components. + */ +private[spark] object SparkKubernetesClientFactory { + + def createKubernetesClient( + master: String, + namespace: Option[String], + kubernetesAuthConfPrefix: String, + sparkConf: SparkConf, + maybeServiceAccountToken: Option[File], + maybeServiceAccountCaCert: Option[File]): KubernetesClient = { + val oauthTokenFileConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_FILE_CONF_SUFFIX" + val oauthTokenConf = s"$kubernetesAuthConfPrefix.$OAUTH_TOKEN_CONF_SUFFIX" + val oauthTokenFile = sparkConf.getOption(oauthTokenFileConf) + .map(new File(_)) + .orElse(maybeServiceAccountToken) + val oauthTokenValue = sparkConf.getOption(oauthTokenConf) + OptionRequirements.requireNandDefined( + oauthTokenFile, + oauthTokenValue, + s"Cannot specify OAuth token through both a file $oauthTokenFileConf and a" + + s" value $oauthTokenConf.") + + val caCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CA_CERT_FILE_CONF_SUFFIX") + .orElse(maybeServiceAccountCaCert.map(_.getAbsolutePath)) + val clientKeyFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") + val clientCertFile = sparkConf + .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") + val dispatcher = new Dispatcher( + ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) + val config = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withWebsocketPingInterval(0) + .withOption(oauthTokenValue) { + (token, configBuilder) => configBuilder.withOauthToken(token) + }.withOption(oauthTokenFile) { + (file, configBuilder) => + configBuilder.withOauthToken(Files.toString(file, Charsets.UTF_8)) + }.withOption(caCertFile) { + (file, configBuilder) => configBuilder.withCaCertFile(file) + }.withOption(clientKeyFile) { + (file, configBuilder) => configBuilder.withClientKeyFile(file) + }.withOption(clientCertFile) { + (file, configBuilder) => configBuilder.withClientCertFile(file) + }.withOption(namespace) { + (ns, configBuilder) => configBuilder.withNamespace(ns) + }.build() + val baseHttpClient = HttpClientUtils.createHttpClient(config) + val httpClientWithCustomDispatcher = baseHttpClient.newBuilder() + .dispatcher(dispatcher) + .build() + new DefaultKubernetesClient(httpClientWithCustomDispatcher, config) + } + + private implicit class OptionConfigurableConfigBuilder(configBuilder: ConfigBuilder) { + + def withOption[T] + (option: Option[T]) + (configurator: ((T, ConfigBuilder) => ConfigBuilder)): OptionConfigurableConfigBuilder = { + new OptionConfigurableConfigBuilder(option.map { opt => + configurator(opt, configBuilder) + }.getOrElse(configBuilder)) + } + + def build(): Config = configBuilder.build() + } +} 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/kubernetes/SparkPodInitContainerBootstrap.scala new file mode 100644 index 0000000000000..2df7ac7a204c4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala @@ -0,0 +1,112 @@ +/* + * 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 + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ + +/** + * This is separated out from the init-container steps API because this component can be reused to + * set up the init-container for executors as well. + */ +private[spark] trait SparkPodInitContainerBootstrap { + /** + * Bootstraps an init-container that downloads dependencies to be used by a main container. + * Note that this primarily assumes that the init-container's configuration is being provided + * by a ConfigMap that was installed by some other component; that is, the implementation + * here makes no assumptions about how the init-container is specifically configured. For + * example, this class is unaware if the init-container is fetching remote dependencies or if + * it is fetching dependencies from a resource staging server. Additionally, the container itself + * is not actually attached to the pod, but the init container is returned so it can be attached + * by InitContainerUtil after the caller has decided to make any changes to it. + */ + def bootstrapInitContainerAndVolumes( + originalPodWithUnattachedInitContainer: PodWithDetachedInitContainer) + : PodWithDetachedInitContainer +} + +private[spark] class SparkPodInitContainerBootstrapImpl( + initContainerImage: String, + dockerImagePullPolicy: String, + jarsDownloadPath: String, + filesDownloadPath: String, + downloadTimeoutMinutes: Long, + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends SparkPodInitContainerBootstrap { + + override def bootstrapInitContainerAndVolumes( + podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = { + val sharedVolumeMounts = Seq[VolumeMount]( + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withMountPath(jarsDownloadPath) + .build(), + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withMountPath(filesDownloadPath) + .build()) + + val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer) + .withName(s"spark-init") + .withImage(initContainerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addNewVolumeMount() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR) + .endVolumeMount() + .addToVolumeMounts(sharedVolumeMounts: _*) + .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) + .build() + val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withNewConfigMap() + .withName(initContainerConfigMapName) + .addNewItem() + .withKey(initContainerConfigMapKey) + .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .endSpec() + .build() + val mainContainerWithMountedFiles = new ContainerBuilder( + podWithDetachedInitContainer.mainContainer) + .addToVolumeMounts(sharedVolumeMounts: _*) + .addNewEnv() + .withName(ENV_MOUNTED_FILES_DIR) + .withValue(filesDownloadPath) + .endEnv() + .build() + PodWithDetachedInitContainer( + podWithBasicVolumes, + initContainer, + mainContainerWithMountedFiles) + } + +} 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/kubernetes/config.scala new file mode 100644 index 0000000000000..f9c4c9c6a1e18 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -0,0 +1,530 @@ +/* + * 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 + +import java.util.concurrent.TimeUnit + +import org.apache.spark.{SPARK_VERSION => sparkVersion} +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.network.util.ByteUnit + +package object config extends Logging { + + private[spark] val KUBERNETES_NAMESPACE = + ConfigBuilder("spark.kubernetes.namespace") + .doc("The namespace that will be used for running the driver and executor pods. When using" + + " spark-submit in cluster mode, this can also be passed to spark-submit via the" + + " --kubernetes-namespace command line argument.") + .stringConf + .createWithDefault("default") + + private[spark] val DRIVER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.driver.docker.image") + .doc("Docker image to use for the driver. Specify this using the standard Docker tag format.") + .stringConf + .createWithDefault(s"spark-driver:$sparkVersion") + + private[spark] val EXECUTOR_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.executor.docker.image") + .doc("Docker image to use for the executors. Specify this using the standard Docker tag" + + " format.") + .stringConf + .createWithDefault(s"spark-executor:$sparkVersion") + + private[spark] val DOCKER_IMAGE_PULL_POLICY = + ConfigBuilder("spark.kubernetes.docker.image.pullPolicy") + .doc("Docker image pull policy when pulling any docker image in Kubernetes integration") + .stringConf + .createWithDefault("IfNotPresent") + + private[spark] val APISERVER_AUTH_SUBMISSION_CONF_PREFIX = + "spark.kubernetes.authenticate.submission" + private[spark] val APISERVER_AUTH_DRIVER_CONF_PREFIX = + "spark.kubernetes.authenticate.driver" + private[spark] val APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX = + "spark.kubernetes.authenticate.driver.mounted" + private[spark] val APISERVER_AUTH_RESOURCE_STAGING_SERVER_CONF_PREFIX = + "spark.kubernetes.authenticate.resourceStagingServer" + private[spark] val APISERVER_AUTH_SHUFFLE_SERVICE_CONF_PREFIX = + "spark.kubernetes.authenticate.shuffleService" + private[spark] val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" + private[spark] val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" + private[spark] val CLIENT_KEY_FILE_CONF_SUFFIX = "clientKeyFile" + private[spark] val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" + private[spark] val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" + + private[spark] val RESOURCE_STAGING_SERVER_USE_SERVICE_ACCOUNT_CREDENTIALS = + ConfigBuilder( + s"$APISERVER_AUTH_RESOURCE_STAGING_SERVER_CONF_PREFIX.useServiceAccountCredentials") + .doc("Use a service account token and CA certificate in the resource staging server to" + + " watch the API server's objects.") + .booleanConf + .createWithDefault(true) + + private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME = + ConfigBuilder(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") + .doc("Service account that is used when running the driver pod. The driver pod uses" + + " this service account when requesting executor pods from the API server. If specific" + + " credentials are given for the driver pod to use, the driver will favor" + + " using those credentials instead.") + .stringConf + .createOptional + + private[spark] val SPARK_SHUFFLE_SERVICE_HOST = + ConfigBuilder("spark.shuffle.service.host") + .doc("Host for Spark Shuffle Service") + .internal() + .stringConf + .createOptional + + // Note that while we set a default for this when we start up the + // scheduler, the specific default value is dynamically determined + // based on the executor memory. + private[spark] val KUBERNETES_EXECUTOR_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.executor.memoryOverhead") + .doc("The amount of off-heap memory (in megabytes) to be allocated per executor. This" + + " is memory that accounts for things like VM overheads, interned strings, other native" + + " overheads, etc. This tends to grow with the executor size. (typically 6-10%).") + .bytesConf(ByteUnit.MiB) + .createOptional + + private[spark] val KUBERNETES_DRIVER_MEMORY_OVERHEAD = + ConfigBuilder("spark.kubernetes.driver.memoryOverhead") + .doc("The amount of off-heap memory (in megabytes) to be allocated for the driver and the" + + " driver submission server. This is memory that accounts for things like VM overheads," + + " interned strings, other native overheads, etc. This tends to grow with the driver's" + + " memory size (typically 6-10%).") + .bytesConf(ByteUnit.MiB) + .createOptional + + 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_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_POD_NAME = + ConfigBuilder("spark.kubernetes.driver.pod.name") + .doc("Name of the driver pod.") + .stringConf + .createOptional + + private[spark] val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.executor.podNamePrefix") + .doc("Prefix to use in front of the executor pod names.") + .internal() + .stringConf + .createWithDefault("spark") + + private[spark] val KUBERNETES_SHUFFLE_NAMESPACE = + ConfigBuilder("spark.kubernetes.shuffle.namespace") + .doc("Namespace of the shuffle service") + .stringConf + .createWithDefault("default") + + private[spark] val KUBERNETES_SHUFFLE_SVC_IP = + ConfigBuilder("spark.kubernetes.shuffle.ip") + .doc("This setting is for debugging only. Setting this " + + "allows overriding the IP that the executor thinks its colocated " + + "shuffle service is on") + .stringConf + .createOptional + + private[spark] val KUBERNETES_SHUFFLE_LABELS = + ConfigBuilder("spark.kubernetes.shuffle.labels") + .doc("Labels to identify the shuffle service") + .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.") + .stringConf + .createWithDefault(KUBERNETES_MASTER_INTERNAL_URL) + + private[spark] val KUBERNETES_SHUFFLE_USE_SERVICE_ACCOUNT_CREDENTIALS = + ConfigBuilder(s"$APISERVER_AUTH_SHUFFLE_SERVICE_CONF_PREFIX.useServiceAccountCredentials") + .doc("Whether or not to use service account credentials when contacting the API server from" + + " the shuffle service.") + .booleanConf + .createWithDefault(true) + + private[spark] val KUBERNETES_ALLOCATION_BATCH_SIZE = + ConfigBuilder("spark.kubernetes.allocation.batch.size") + .doc("Number of pods to launch at once in each round of dynamic allocation. ") + .intConf + .createWithDefault(5) + + private[spark] val KUBERNETES_ALLOCATION_BATCH_DELAY = + ConfigBuilder("spark.kubernetes.allocation.batch.delay") + .doc("Number of seconds to wait between each round of executor allocation. ") + .longConf + .createWithDefault(1) + + private[spark] val WAIT_FOR_APP_COMPLETION = + ConfigBuilder("spark.kubernetes.submission.waitAppCompletion") + .doc("In cluster mode, whether to wait for the application to finish before exiting the" + + " launcher process.") + .booleanConf + .createWithDefault(true) + + private[spark] val REPORT_INTERVAL = + ConfigBuilder("spark.kubernetes.report.interval") + .doc("Interval between reports of the current app status in cluster mode.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") + + // Spark resource staging server. + private[spark] val RESOURCE_STAGING_SERVER_API_SERVER_URL = + ConfigBuilder("spark.kubernetes.resourceStagingServer.apiServer.url") + .doc("URL for the Kubernetes API server. The resource staging server monitors the API" + + " server to check when pods no longer are using mounted resources. Note that this isn't" + + " to be used in Spark applications, as the API server URL should be set via spark.master.") + .stringConf + .createWithDefault(KUBERNETES_MASTER_INTERNAL_URL) + + private[spark] val RESOURCE_STAGING_SERVER_API_SERVER_CA_CERT_FILE = + ConfigBuilder("spark.kubernetes.resourceStagingServer.apiServer.caCertFile") + .doc("CA certificate for the resource staging server to use when contacting the Kubernetes" + + " API server over TLS.") + .stringConf + .createOptional + + private[spark] val RESOURCE_STAGING_SERVER_PORT = + ConfigBuilder("spark.kubernetes.resourceStagingServer.port") + .doc("Port for the Kubernetes resource staging server to listen on.") + .intConf + .createWithDefault(10000) + + private[spark] val RESOURCE_STAGING_SERVER_INITIAL_ACCESS_EXPIRATION_TIMEOUT = + ConfigBuilder("spark.kubernetes.resourceStagingServer.initialAccessExpirationTimeout") + .doc("The resource staging server will wait for any resource bundle to be accessed for a" + + " first time for this period. If this timeout expires before the resources are accessed" + + " the first time, the resources are cleaned up under the assumption that the dependents" + + " of the given resource bundle failed to launch at all.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("30m") + + private[spark] val RESOURCE_STAGING_SERVER_KEY_PEM = + ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyPem") + .doc("Key PEM file to use when having the Kubernetes dependency server listen on TLS.") + .stringConf + .createOptional + + private[spark] val RESOURCE_STAGING_SERVER_SSL_NAMESPACE = "kubernetes.resourceStagingServer" + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE = + "kubernetes.resourceStagingServer.internal" + private[spark] val RESOURCE_STAGING_SERVER_CERT_PEM = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.serverCertPem") + .doc("Certificate PEM file to use when having the resource staging server" + + " listen on TLS.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.clientCertPem") + .doc("Certificate PEM file to use when the client contacts the resource staging server." + + " This must strictly be a path to a file on the submitting machine's disk.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE.clientCertPem") + .doc("Certificate PEM file to use when the init-container contacts the resource staging" + + " server. If this is not provided, it defaults to the value of" + + " spark.ssl.kubernetes.resourceStagingServer.clientCertPem. This can be a URI with" + + " a scheme of local:// which denotes that the file is pre-mounted on the init-container's" + + " disk. A uri without a scheme or a scheme of file:// will result in this file being" + + " mounted from the submitting machine's disk as a secret into the pods.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.keyStorePasswordFile") + .doc("File containing the keystore password for the Kubernetes resource staging server.") + .stringConf + .createOptional + + private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.keyPasswordFile") + .doc("File containing the key password for the Kubernetes resource staging server.") + .stringConf + .createOptional + + private[spark] val RESOURCE_STAGING_SERVER_SSL_ENABLED = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.enabled") + .doc("Whether or not to use SSL when communicating with the resource staging server.") + .booleanConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE.enabled") + .doc("Whether or not to use SSL when communicating with the resource staging server from" + + " the init-container. If this is not provided, defaults to" + + " the value of spark.ssl.kubernetes.resourceStagingServer.enabled") + .booleanConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStore") + .doc("File containing the trustStore to communicate with the Kubernetes dependency server." + + " This must strictly be a path on the submitting machine's disk.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE.trustStore") + .doc("File containing the trustStore to communicate with the Kubernetes dependency server" + + " from the init-container. If this is not provided, defaults to the value of" + + " spark.ssl.kubernetes.resourceStagingServer.trustStore. This can be a URI with a scheme" + + " of local:// indicating that the trustStore is pre-mounted on the init-container's" + + " disk. If no scheme, or a scheme of file:// is provided, this file is mounted from the" + + " submitting machine's disk as a Kubernetes secret into the pods.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStorePassword") + .doc("Password for the trustStore for communicating to the dependency server.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE.trustStorePassword") + .doc("Password for the trustStore for communicating to the dependency server from the" + + " init-container. If this is not provided, defaults to" + + " spark.ssl.kubernetes.resourceStagingServer.trustStorePassword.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStoreType") + .doc("Type of trustStore for communicating with the dependency server.") + .stringConf + .createOptional + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE = + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_INTERNAL_SSL_NAMESPACE.trustStoreType") + .doc("Type of trustStore for communicating with the dependency server from the" + + " init-container. If this is not provided, defaults to" + + " spark.ssl.kubernetes.resourceStagingServer.trustStoreType") + .stringConf + .createOptional + + // Driver and Init-Container parameters + private[spark] val RESOURCE_STAGING_SERVER_URI = + ConfigBuilder("spark.kubernetes.resourceStagingServer.uri") + .doc("Base URI for the Spark resource staging server.") + .stringConf + .createOptional + + private[spark] val RESOURCE_STAGING_SERVER_INTERNAL_URI = + ConfigBuilder("spark.kubernetes.resourceStagingServer.internal.uri") + .doc("Base URI for the Spark resource staging server when the init-containers access it for" + + " downloading resources. If this is not provided, it defaults to the value provided in" + + " spark.kubernetes.resourceStagingServer.uri, the URI that the submission client uses to" + + " upload the resources from outside the cluster.") + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER = + ConfigBuilder("spark.kubernetes.initcontainer.downloadJarsResourceIdentifier") + .doc("Identifier for the jars tarball that was uploaded to the staging service.") + .internal() + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION = + ConfigBuilder("spark.kubernetes.initcontainer.downloadJarsSecretLocation") + .doc("Location of the application secret to use when the init-container contacts the" + + " resource staging server to download jars.") + .internal() + .stringConf + .createWithDefault(s"$INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH/" + + s"$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY") + + private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER = + ConfigBuilder("spark.kubernetes.initcontainer.downloadFilesResourceIdentifier") + .doc("Identifier for the files tarball that was uploaded to the staging service.") + .internal() + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION = + ConfigBuilder("spark.kubernetes.initcontainer.downloadFilesSecretLocation") + .doc("Location of the application secret to use when the init-container contacts the" + + " resource staging server to download files.") + .internal() + .stringConf + .createWithDefault( + s"$INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY") + + private[spark] val INIT_CONTAINER_REMOTE_JARS = + ConfigBuilder("spark.kubernetes.initcontainer.remoteJars") + .doc("Comma-separated list of jar URIs to download in the init-container. This is" + + " calculated from spark.jars.") + .internal() + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_REMOTE_FILES = + ConfigBuilder("spark.kubernetes.initcontainer.remoteFiles") + .doc("Comma-separated list of file URIs to download in the init-container. This is" + + " calculated from spark.files.") + .internal() + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_DOCKER_IMAGE = + ConfigBuilder("spark.kubernetes.initcontainer.docker.image") + .doc("Image for the driver and executor's init-container that downloads dependencies.") + .stringConf + .createWithDefault(s"spark-init:$sparkVersion") + + private[spark] val INIT_CONTAINER_JARS_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountdependencies.jarsDownloadDir") + .doc("Location to download jars to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pod.") + .stringConf + .createWithDefault("/var/spark-data/spark-jars") + + private[spark] val INIT_CONTAINER_FILES_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountdependencies.filesDownloadDir") + .doc("Location to download files to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pods.") + .stringConf + .createWithDefault("/var/spark-data/spark-files") + + private[spark] val INIT_CONTAINER_MOUNT_TIMEOUT = + ConfigBuilder("spark.kubernetes.mountdependencies.mountTimeout") + .doc("Timeout before aborting the attempt to download and unpack local dependencies from" + + " remote locations and the resource staging server when initializing the driver and" + + " executor pods.") + .timeConf(TimeUnit.MINUTES) + .createWithDefault(5) + + private[spark] val EXECUTOR_INIT_CONTAINER_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.initcontainer.executor.configmapname") + .doc("Name of the config map to use in the init-container that retrieves submitted files" + + " for the executor.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY = + ConfigBuilder("spark.kubernetes.initcontainer.executor.configmapkey") + .doc("Key for the entry in the init container config map for submitted files that" + + " corresponds to the properties for this init-container.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_SECRET = + ConfigBuilder("spark.kubernetes.initcontainer.executor.stagingServerSecret.name") + .doc("Name of the secret to mount into the init-container that retrieves submitted files.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR = + ConfigBuilder("spark.kubernetes.initcontainer.executor.stagingServerSecret.mountDir") + .doc("Directory to mount the resource staging server secrets into for the executor" + + " init-containers. This must be exactly the same as the directory that the submission" + + " client mounted the secret into because the config map's properties specify the" + + " secret location as to be the same between the driver init-container and the executor" + + " init-container. Thus the submission client will always set this and the driver will" + + " never rely on a constant or convention, in order to protect against cases where the" + + " submission client has a different version from the driver itself, and hence might" + + " have different constants loaded in constants.scala.") + .internal() + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_LIMIT_CORES = + ConfigBuilder("spark.kubernetes.driver.limit.cores") + .doc("Specify the hard cpu limit for the driver pod") + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED = + ConfigBuilder("spark.kubernetes.driver.hdfslocality.clusterNodeNameDNSLookup.enabled") + .doc("Whether or not HDFS locality support code should look up DNS for full hostnames of" + + " cluster nodes. In some K8s clusters, notably GKE, cluster node names are short" + + " hostnames, and so comparing them against HDFS datanode hostnames always fail. To fix," + + " enable this flag. This is disabled by default because DNS lookup can be expensive." + + " The driver can slow down and fail to respond to executor heartbeats in time." + + " If enabling this flag, make sure your DNS server has enough capacity" + + " for the workload.") + .internal() + .booleanConf + .createWithDefault(false) + + private[spark] val KUBERNETES_EXECUTOR_LIMIT_CORES = + ConfigBuilder("spark.kubernetes.executor.limit.cores") + .doc("Specify the hard cpu limit for a single executor pod") + .stringConf + .createOptional + + private[spark] val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + + private[spark] def resolveK8sMaster(rawMasterString: String): String = { + if (!rawMasterString.startsWith("k8s://")) { + throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") + } + val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + if (masterWithoutK8sPrefix.startsWith("http://") + || masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + s" URL is $resolvedURL") + resolvedURL + } + } +} 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/kubernetes/constants.scala new file mode 100644 index 0000000000000..92f051b2ac298 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -0,0 +1,100 @@ +/* + * 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 + +package object constants { + // Labels + private[spark] val SPARK_DRIVER_LABEL = "spark-driver" + private[spark] val SPARK_APP_ID_LABEL = "spark-app-selector" + private[spark] val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" + private[spark] val SPARK_ROLE_LABEL = "spark-role" + private[spark] val SPARK_POD_DRIVER_ROLE = "driver" + private[spark] val SPARK_POD_EXECUTOR_ROLE = "executor" + private[spark] val SPARK_APP_NAME_ANNOTATION = "spark-app-name" + + // Credentials secrets + private[spark] val DRIVER_CREDENTIALS_SECRETS_BASE_DIR = + "/mnt/secrets/spark-kubernetes-credentials" + private[spark] val DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME = "ca-cert" + private[spark] val DRIVER_CREDENTIALS_CA_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME = "client-key" + private[spark] val DRIVER_CREDENTIALS_CLIENT_KEY_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME = "client-cert" + private[spark] val DRIVER_CREDENTIALS_CLIENT_CERT_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME" + private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME = "oauth-token" + private[spark] val DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH = + s"$DRIVER_CREDENTIALS_SECRETS_BASE_DIR/$DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME" + 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 EXECUTOR_PORT_NAME = "executor" + + // Environment Variables + private[spark] val ENV_EXECUTOR_PORT = "SPARK_EXECUTOR_PORT" + private[spark] val ENV_DRIVER_URL = "SPARK_DRIVER_URL" + private[spark] val ENV_EXECUTOR_CORES = "SPARK_EXECUTOR_CORES" + private[spark] val ENV_EXECUTOR_MEMORY = "SPARK_EXECUTOR_MEMORY" + private[spark] val ENV_APPLICATION_ID = "SPARK_APPLICATION_ID" + private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" + private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" + private[spark] val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" + private[spark] val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" + private[spark] val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_EXECUTOR_EXTRA_CLASSPATH" + private[spark] val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" + private[spark] val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" + private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" + private[spark] val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" + private[spark] val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR" + private[spark] val ENV_PYSPARK_FILES = "PYSPARK_FILES" + private[spark] val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" + + // Bootstrapping dependencies with the init-container + private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" + private[spark] val INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH = + "/mnt/secrets/spark-init" + private[spark] val INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY = + "downloadSubmittedJarsSecret" + private[spark] val INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY = + "downloadSubmittedFilesSecret" + private[spark] val INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY = "trustStore" + private[spark] val INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY = "ssl-certificate" + private[spark] val INIT_CONTAINER_CONFIG_MAP_KEY = "download-submitted-files" + private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume" + private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_PATH = + s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" + private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" + private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" + + // Miscellaneous + private[spark] val ANNOTATION_EXECUTOR_NODE_AFFINITY = "scheduler.alpha.kubernetes.io/affinity" + private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + 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 = 384L +} 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/kubernetes/submit/Client.scala new file mode 100644 index 0000000000000..2fa9b416330e5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala @@ -0,0 +1,200 @@ +/* + * 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 + +import java.util.{Collections, UUID} + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, OwnerReferenceBuilder, PodBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.mutable + +import org.apache.spark.SparkConf +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.kubernetes.SparkKubernetesClientFactory +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] case class ClientArguments( + mainAppResource: MainAppResource, + otherPyFiles: Seq[String], + mainClass: String, + driverArgs: Array[String]) + +private[spark] object ClientArguments { + def fromCommandLineArgs(args: Array[String]): ClientArguments = { + var mainAppResource: Option[MainAppResource] = None + var otherPyFiles = Seq.empty[String] + var mainClass: Option[String] = None + val driverArgs = mutable.Buffer.empty[String] + args.sliding(2, 2).toList.collect { + case Array("--primary-py-file", mainPyFile: String) => + mainAppResource = Some(PythonMainAppResource(mainPyFile)) + case Array("--primary-java-resource", primaryJavaResource: String) => + mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) + case Array("--main-class", clazz: String) => + mainClass = Some(clazz) + case Array("--other-py-files", pyFiles: String) => + otherPyFiles = pyFiles.split(",") + case Array("--arg", arg: String) => + driverArgs += arg + case other => + val invalid = other.mkString(" ") + throw new RuntimeException(s"Unknown arguments: $invalid") + } + require(mainAppResource.isDefined, + "Main app resource must be defined by either --primary-py-file or --primary-java-resource.") + require(mainClass.isDefined, "Main class must be specified via --main-class") + ClientArguments( + mainAppResource.get, + otherPyFiles, + mainClass.get, + driverArgs.toArray) + } +} + +private[spark] class Client( + submissionSteps: Seq[DriverConfigurationStep], + submissionSparkConf: SparkConf, + kubernetesClient: KubernetesClient, + waitForAppCompletion: Boolean, + appName: String, + loggingPodStatusWatcher: LoggingPodStatusWatcher) extends Logging { + + private val driverJavaOptions = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + + /** + * Run command that initalizes a DriverSpec that will be updated after each + * DriverConfigurationStep in the sequence that is passed in. The final KubernetesDriverSpec + * will be used to build the Driver Container, Driver Pod, and Kubernetes Resources + */ + def run(): Unit = { + var currentDriverSpec = KubernetesDriverSpec.initialSpec(submissionSparkConf) + // submissionSteps contain steps necessary to take, to resolve varying + // client arguments that are passed in, created by orchestrator + for (nextStep <- submissionSteps) { + currentDriverSpec = nextStep.configureDriver(currentDriverSpec) + } + val resolvedDriverJavaOpts = currentDriverSpec + .driverSparkConf + // We don't need this anymore since we just set the JVM options on the environment + .remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + .getAll + .map { + case (confKey, confValue) => s"-D$confKey=$confValue" + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + val resolvedDriverContainer = new ContainerBuilder(currentDriverSpec.driverContainer) + .addNewEnv() + .withName(ENV_DRIVER_JAVA_OPTS) + .withValue(resolvedDriverJavaOpts) + .endEnv() + .build() + val resolvedDriverPod = new PodBuilder(currentDriverSpec.driverPod) + .editSpec() + .addToContainers(resolvedDriverContainer) + .endSpec() + .build() + Utils.tryWithResource( + kubernetesClient + .pods() + .withName(resolvedDriverPod.getMetadata.getName) + .watch(loggingPodStatusWatcher)) { _ => + val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) + try { + if (currentDriverSpec.otherKubernetesResources.nonEmpty) { + val driverPodOwnerReference = new OwnerReferenceBuilder() + .withName(createdDriverPod.getMetadata.getName) + .withApiVersion(createdDriverPod.getApiVersion) + .withUid(createdDriverPod.getMetadata.getUid) + .withKind(createdDriverPod.getKind) + .withController(true) + .build() + currentDriverSpec.otherKubernetesResources.foreach { resource => + val originalMetadata = resource.getMetadata + originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) + } + val otherKubernetesResources = currentDriverSpec.otherKubernetesResources + kubernetesClient.resourceList(otherKubernetesResources: _*).createOrReplace() + } + } catch { + case e: Throwable => + kubernetesClient.pods().delete(createdDriverPod) + throw e + } + if (waitForAppCompletion) { + logInfo(s"Waiting for application $appName to finish...") + loggingPodStatusWatcher.awaitCompletion() + logInfo(s"Application $appName finished.") + } else { + logInfo(s"Deployed Spark application $appName into Kubernetes.") + } + } + } +} + +private[spark] object Client { + def run(sparkConf: SparkConf, clientArguments: ClientArguments): Unit = { + val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" + val launchTime = System.currentTimeMillis() + val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) + val appName = sparkConf.getOption("spark.app.name").getOrElse("spark") + val master = resolveK8sMaster(sparkConf.get("spark.master")) + val loggingInterval = Option(sparkConf.get(REPORT_INTERVAL)).filter( _ => waitForAppCompletion) + val loggingPodStatusWatcher = new LoggingPodStatusWatcherImpl( + kubernetesAppId, loggingInterval) + val configurationStepsOrchestrator = new DriverConfigurationStepsOrchestrator( + namespace, + kubernetesAppId, + launchTime, + clientArguments.mainAppResource, + appName, + clientArguments.mainClass, + clientArguments.driverArgs, + clientArguments.otherPyFiles, + sparkConf) + Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( + master, + Some(namespace), + APISERVER_AUTH_SUBMISSION_CONF_PREFIX, + sparkConf, + None, + None)) { kubernetesClient => + new Client( + configurationStepsOrchestrator.getAllConfigurationSteps(), + sparkConf, + kubernetesClient, + waitForAppCompletion, + appName, + loggingPodStatusWatcher).run() + } + } + + /** + * Entry point from SparkSubmit in spark-core + * + * @param args Array of strings that have interchanging values that will be + * parsed by ClientArguments with the identifiers that precede the values + */ + def main(args: Array[String]): Unit = { + val parsedArguments = ClientArguments.fromCommandLineArgs(args) + val sparkConf = new SparkConf() + run(sparkConf, parsedArguments) + } +} 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/kubernetes/submit/ContainerNameEqualityPredicate.scala new file mode 100644 index 0000000000000..434919208ba2e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala @@ -0,0 +1,29 @@ +/* + * 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 + +import java.lang.Boolean + +import io.fabric8.kubernetes.api.builder.Predicate +import io.fabric8.kubernetes.api.model.ContainerBuilder + +private[spark] class ContainerNameEqualityPredicate(containerName: String) + extends Predicate[ContainerBuilder] { + override def apply(item: ContainerBuilder): Boolean = { + item.getName == containerName + } +} 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/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala new file mode 100644 index 0000000000000..82abe55ac6989 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -0,0 +1,138 @@ +/* + * 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 + +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, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.util.Utils + +/** + * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. + */ +private[spark] class DriverConfigurationStepsOrchestrator( + namespace: String, + kubernetesAppId: String, + launchTime: Long, + mainAppResource: MainAppResource, + appName: String, + mainClass: String, + appArgs: Array[String], + additionalPythonFiles: Seq[String], + submissionSparkConf: SparkConf) { + + // The resource name prefix is derived from the application name, making it easy to connect the + // names of the Kubernetes resources from e.g. kubectl or the Kubernetes dashboard to the + // application the user submitted. However, we can't use the application name in the label, as + // label values are considerably restrictive, e.g. must be no longer than 63 characters in + // length. So we generate a separate identifier for the app ID itself, and bookkeeping that + // requires finding "all pods for this application" should use the kubernetesAppId. + private val kubernetesResourceNamePrefix = + s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + private val jarsDownloadPath = submissionSparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = submissionSparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) + private val dockerImagePullPolicy = submissionSparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val initContainerConfigMapName = s"$kubernetesResourceNamePrefix-init-config" + + def getAllConfigurationSteps(): Seq[DriverConfigurationStep] = { + val additionalMainAppJar = mainAppResource match { + case JavaMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => + Option(resource) + case _ => Option.empty + } + val additionalMainAppPythonFile = mainAppResource match { + case PythonMainAppResource(resource) if resource != SparkLauncher.NO_RESOURCE => + Option(resource) + case _ => Option.empty + } + val sparkJars = submissionSparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ + additionalMainAppJar.toSeq + val sparkFiles = submissionSparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ + additionalMainAppPythonFile.toSeq ++ + additionalPythonFiles + val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + 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" + + s" operations.") + val allDriverLabels = driverCustomLabels ++ Map( + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val initialSubmissionStep = new BaseDriverConfigurationStep( + kubernetesAppId, + kubernetesResourceNamePrefix, + allDriverLabels, + dockerImagePullPolicy, + appName, + mainClass, + appArgs, + submissionSparkConf) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, kubernetesResourceNamePrefix) + val pythonStep = mainAppResource match { + case PythonMainAppResource(mainPyResource) => + Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) + case _ => Option.empty[DriverConfigurationStep] + } + val initContainerBootstrapStep = if ((sparkJars ++ sparkFiles).exists { uri => + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") != "local" + }) { + val initContainerConfigurationStepsOrchestrator = + new InitContainerConfigurationStepsOrchestrator( + namespace, + kubernetesResourceNamePrefix, + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + dockerImagePullPolicy, + allDriverLabels, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY, + submissionSparkConf) + val initContainerConfigurationSteps = + initContainerConfigurationStepsOrchestrator.getAllConfigurationSteps() + Some(new InitContainerBootstrapStep(initContainerConfigurationSteps, + initContainerConfigMapName, + INIT_CONTAINER_CONFIG_MAP_KEY)) + } else { + Option.empty[DriverConfigurationStep] + } + val dependencyResolutionStep = new DependencyResolutionStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath) + Seq( + initialSubmissionStep, + kubernetesCredentialsStep, + dependencyResolutionStep) ++ + initContainerBootstrapStep.toSeq ++ + pythonStep.toSeq + } +} 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/kubernetes/submit/InitContainerUtil.scala new file mode 100644 index 0000000000000..837ec0e8c867e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala @@ -0,0 +1,49 @@ +/* + * 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 + +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._ + +private[spark] object InitContainerUtil { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) + + def appendInitContainer(originalPodSpec: Pod, initContainer: Container): Pod = { + val resolvedInitContainers = originalPodSpec + .getMetadata + .getAnnotations + .asScala + .get(INIT_CONTAINER_ANNOTATION) + .map { existingInitContainerAnnotation => + val existingInitContainers = OBJECT_MAPPER.readValue( + existingInitContainerAnnotation, classOf[List[Container]]) + existingInitContainers ++ Seq(initContainer) + }.getOrElse(Seq(initContainer)) + val resolvedSerializedInitContainers = OBJECT_MAPPER.writeValueAsString(resolvedInitContainers) + new PodBuilder(originalPodSpec) + .editMetadata() + .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) + .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) + .endMetadata() + .build() + } +} 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/kubernetes/submit/KubernetesFileUtils.scala new file mode 100644 index 0000000000000..ec591923f1472 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala @@ -0,0 +1,96 @@ +/* + * 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 + +import java.io.File + +import org.apache.spark.util.Utils + +private[spark] object KubernetesFileUtils { + + private def filterUriStringsByScheme( + uris: Iterable[String], schemeFilter: (String => Boolean)): Iterable[String] = { + uris.filter(uri => schemeFilter(Option(Utils.resolveURI(uri).getScheme).getOrElse("file"))) + } + + def getNonSubmitterLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ != "file") + } + + def getOnlyContainerLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ == "local") + } + + def getNonContainerLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ != "local") + } + + def getOnlySubmitterLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ == "file") + } + + def isUriLocalFile(uri: String): Boolean = { + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") == "file" + } + + def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") + } + + /** + * For the collection of uris, resolves any files as follows: + * - Files with scheme file:// are resolved to the given download path + * - Files with scheme local:// resolve to just the path of the URI + * - Otherwise, the URI is returned as-is. + */ + def resolveSubmittedUris(fileUris: Iterable[String], fileDownloadPath: String) + : Iterable[String] = { + fileUris.map { uri => + val fileUri = Utils.resolveURI(uri) + val fileScheme = Option(fileUri.getScheme).getOrElse("file") + fileScheme match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + case "local" => + fileUri.getPath + case _ => uri + } + } + } + + /** + * If any file uri has any scheme other than local:// it is mapped as if the file + * was downloaded to the file download path. Otherwise, it is mapped to the path + * part of the URI. + */ + def resolveFilePaths(fileUris: Iterable[String], fileDownloadPath: String): Iterable[String] = { + fileUris.map { uri => + resolveFilePath(uri, fileDownloadPath) + } + } + + def resolveFilePath(uri: String, fileDownloadPath: String): String = { + val fileUri = Utils.resolveURI(uri) + if (Option(fileUri.getScheme).getOrElse("file") == "local") { + fileUri.getPath + } else { + val fileName = new File(fileUri.getPath).getName + s"$fileDownloadPath/$fileName" + } + } +} 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/kubernetes/submit/LoggingPodStatusWatcher.scala new file mode 100644 index 0000000000000..4a8a7308b9fe4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala @@ -0,0 +1,178 @@ +/* + * 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 + +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod} +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.util.ThreadUtils + +private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { + def awaitCompletion(): Unit +} + +/** + * A monitor for the running Kubernetes pod of a Spark application. Status logging occurs on + * every state change and also at an interval for liveness. + * + * @param appId + * @param maybeLoggingInterval ms between each state request. If provided, must be a positive + * number. + */ +private[kubernetes] class LoggingPodStatusWatcherImpl( + appId: String, maybeLoggingInterval: Option[Long]) + extends LoggingPodStatusWatcher with Logging { + + private val podCompletedFuture = new CountDownLatch(1) + // start timer for periodic logging + private val scheduler = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("logging-pod-status-watcher") + private val logRunnable: Runnable = new Runnable { + override def run() = logShortStatus() + } + + private var pod = Option.empty[Pod] + + private def phase: String = pod.map(_.getStatus.getPhase).getOrElse("unknown") + + def start(): Unit = { + maybeLoggingInterval.foreach { interval => + require(interval > 0, s"Logging interval must be a positive time value, got: $interval ms.") + scheduler.scheduleAtFixedRate(logRunnable, 0, interval, TimeUnit.MILLISECONDS) + } + } + + override def eventReceived(action: Action, pod: Pod): Unit = { + this.pod = Option(pod) + action match { + case Action.DELETED => + closeWatch() + + case Action.ERROR => + closeWatch() + + case _ => + logLongStatus() + if (hasCompleted()) { + closeWatch() + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + logDebug(s"Stopping watching application $appId with last-observed phase $phase") + closeWatch() + } + + private def logShortStatus() = { + logInfo(s"Application status for $appId (phase: $phase)") + } + + private def logLongStatus() = { + logInfo("State changed, new state: " + pod.map(formatPodState(_)).getOrElse("unknown")) + } + + private def hasCompleted(): Boolean = { + phase == "Succeeded" || phase == "Failed" + } + + private def closeWatch(): Unit = { + podCompletedFuture.countDown() + scheduler.shutdown() + } + + private def formatPodState(pod: Pod): String = { + // TODO include specific container state + val details = Seq[(String, String)]( + // pod metadata + ("pod name", pod.getMetadata.getName()), + ("namespace", pod.getMetadata.getNamespace()), + ("labels", pod.getMetadata.getLabels().asScala.mkString(", ")), + ("pod uid", pod.getMetadata.getUid), + ("creation time", pod.getMetadata.getCreationTimestamp()), + + // spec details + ("service account name", pod.getSpec.getServiceAccountName()), + ("volumes", pod.getSpec.getVolumes().asScala.map(_.getName).mkString(", ")), + ("node name", pod.getSpec.getNodeName()), + + // status + ("start time", pod.getStatus.getStartTime), + ("container images", + pod.getStatus.getContainerStatuses() + .asScala + .map(_.getImage) + .mkString(", ")), + ("phase", pod.getStatus.getPhase()), + ("status", pod.getStatus.getContainerStatuses().toString) + ) + formatPairsBundle(details) + } + + private def formatPairsBundle(pairs: Seq[(String, String)]) = { + // Use more loggable format if value is null or empty + pairs.map { + case (k, v) => s"\n\t $k: ${Option(v).filter(_.nonEmpty).getOrElse("N/A")}" + }.mkString("") + } + + override def awaitCompletion(): Unit = { + podCompletedFuture.await() + logInfo(pod.map { p => + s"Container final statuses:\n\n${containersDescription(p)}" + }.getOrElse("No containers were found in the driver pod.")) + } + + private def containersDescription(p: Pod): String = { + p.getStatus.getContainerStatuses.asScala.map { status => + Seq( + ("Container name", status.getName), + ("Container image", status.getImage)) ++ + containerStatusDescription(status) + }.map(formatPairsBundle).mkString("\n\n") + } + + private def containerStatusDescription( + containerStatus: ContainerStatus): Seq[(String, String)] = { + val state = containerStatus.getState + Option(state.getRunning) + .orElse(Option(state.getTerminated)) + .orElse(Option(state.getWaiting)) + .map { + case running: ContainerStateRunning => + Seq( + ("Container state", "Running"), + ("Container started at", running.getStartedAt)) + case waiting: ContainerStateWaiting => + Seq( + ("Container state", "Waiting"), + ("Pending reason", waiting.getReason)) + case terminated: ContainerStateTerminated => + Seq( + ("Container state", "Terminated"), + ("Exit code", terminated.getExitCode.toString)) + case unknown => + throw new SparkException(s"Unexpected container status type ${unknown.getClass}.") + }.getOrElse(Seq(("Container state", "N/A"))) + } +} 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/kubernetes/submit/MainAppResource.scala new file mode 100644 index 0000000000000..436d531a850ff --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala @@ -0,0 +1,23 @@ +/* + * 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 + +private[spark] sealed trait MainAppResource + +private[spark] case class PythonMainAppResource(primaryPyFile: String) extends MainAppResource + +private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource 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/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala new file mode 100644 index 0000000000000..8103272c27518 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala @@ -0,0 +1,48 @@ +/* + * 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 + +import java.io.StringWriter +import java.util.Properties + +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder} + +/** + * Creates a config map from a map object, with a single given key + * and writing the map in a {@link java.util.Properties} format. + */ +private[spark] object PropertiesConfigMapFromScalaMapBuilder { + + def buildConfigMap( + configMapName: String, + configMapKey: String, + config: Map[String, String]): ConfigMap = { + val properties = new Properties() + config.foreach { case (key, value) => properties.setProperty(key, value) } + val propertiesWriter = new StringWriter() + properties.store(propertiesWriter, + s"Java properties built from Kubernetes config map with name: $configMapName" + + " and config map key: $configMapKey") + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData(configMapKey, propertiesWriter.toString) + .build() + } +} 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/kubernetes/submit/SubmittedDependencyUploaderImpl.scala new file mode 100644 index 0000000000000..83d7a28f5ca10 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala @@ -0,0 +1,109 @@ +/* + * 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 + +import java.io.{File, FileOutputStream} +import javax.ws.rs.core.MediaType + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +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.util.Utils + +private[spark] trait SubmittedDependencyUploader { + /** + * Upload submitter-local jars to the resource staging server. + * @return The resource ID and secret to use to retrieve these jars. + */ + def uploadJars(): SubmittedResourceIdAndSecret + + /** + * Upload submitter-local files to the resource staging server. + * @return The resource ID and secret to use to retrieve these files. + */ + def uploadFiles(): SubmittedResourceIdAndSecret +} + +/** + * Default implementation of a SubmittedDependencyManager that is backed by a + * Resource Staging Service. + */ +private[spark] class SubmittedDependencyUploaderImpl( + podLabels: Map[String, String], + podNamespace: String, + stagingServerUri: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], + stagingServiceSslOptions: SSLOptions, + retrofitClientFactory: RetrofitClientFactory) extends SubmittedDependencyUploader { + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + + private def localUriStringsToFiles(uris: Seq[String]): Iterable[File] = { + KubernetesFileUtils.getOnlySubmitterLocalFiles(uris) + .map(Utils.resolveURI) + .map(uri => new File(uri.getPath)) + } + private def localJars: Iterable[File] = localUriStringsToFiles(sparkJars) + private def localFiles: Iterable[File] = localUriStringsToFiles(sparkFiles) + + override def uploadJars(): SubmittedResourceIdAndSecret = doUpload(localJars, "uploaded-jars") + override def uploadFiles(): SubmittedResourceIdAndSecret = doUpload(localFiles, "uploaded-files") + + private def doUpload(files: Iterable[File], fileNamePrefix: String) + : SubmittedResourceIdAndSecret = { + val filesDir = Utils.createTempDir(namePrefix = fileNamePrefix) + val filesTgz = new File(filesDir, s"$fileNamePrefix.tgz") + Utils.tryWithResource(new FileOutputStream(filesTgz)) { filesOutputStream => + CompressionUtils.writeTarGzipToStream(filesOutputStream, files.map(_.getAbsolutePath)) + } + val stagedResourcesOwner = StagedResourcesOwner( + ownerNamespace = podNamespace, + ownerLabels = podLabels, + ownerType = StagedResourcesOwnerType.Pod) + + val stagedResourcesOwnerString = OBJECT_MAPPER.writeValueAsString(stagedResourcesOwner) + val stagedResourcesOwnerBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), stagedResourcesOwnerString) + val filesRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.MULTIPART_FORM_DATA), filesTgz) + + val service = retrofitClientFactory.createRetrofitClient( + stagingServerUri, + classOf[ResourceStagingServiceRetrofit], + stagingServiceSslOptions) + val uploadResponse = service.uploadResources( + resources = filesRequestBody, resourcesOwner = stagedResourcesOwnerBody) + getTypedResponseResult(uploadResponse) + } + + private def getTypedResponseResult[T](call: Call[T]): T = { + val response = call.execute() + if (response.code() < 200 || response.code() >= 300) { + throw new SparkException("Unexpected response from dependency server when uploading" + + s" dependencies: ${response.code()}. Error body: " + + Option(response.errorBody()).map(_.string()).getOrElse("N/A")) + } + response.body() + } +} 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/kubernetes/submit/SubmittedResources.scala new file mode 100644 index 0000000000000..225972c1057f2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala @@ -0,0 +1,32 @@ +/* + * 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 + +case class SubmittedResourceIdAndSecret(resourceId: String, resourceSecret: String) + +case class SubmittedResources( + jarsResourceIdAndSecret: SubmittedResourceIdAndSecret, + filesResourceIdAndSecret: SubmittedResourceIdAndSecret) { + def ids(): SubmittedResourceIds = SubmittedResourceIds( + jarsResourceIdAndSecret.resourceId, filesResourceIdAndSecret.resourceId) + def secrets(): SubmittedResourceSecrets = SubmittedResourceSecrets( + jarsResourceIdAndSecret.resourceSecret, filesResourceIdAndSecret.resourceSecret) +} + +case class SubmittedResourceIds(jarsResourceId: String, filesResourceId: String) + +case class SubmittedResourceSecrets(jarsResourceSecret: String, filesResourceSecret: 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/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala new file mode 100644 index 0000000000000..b3f509b44054e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala @@ -0,0 +1,134 @@ +/* + * 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 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._ + +/** + * Represents the initial setup required for the driver. + */ +private[spark] class BaseDriverConfigurationStep( + kubernetesAppId: String, + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + dockerImagePullPolicy: String, + appName: String, + mainClass: String, + appArgs: Array[String], + submissionSparkConf: SparkConf) extends DriverConfigurationStep { + + private val kubernetesDriverPodName = submissionSparkConf.get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(s"$kubernetesResourceNamePrefix-driver") + private val driverExtraClasspath = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_CLASS_PATH) + // CPU settings + private val driverCpuCores = submissionSparkConf.getOption("spark.driver.cores").getOrElse("1") + private val driverLimitCores = submissionSparkConf.get(KUBERNETES_DRIVER_LIMIT_CORES) + + // Memory settings + private val driverMemoryMb = submissionSparkConf.get( + org.apache.spark.internal.config.DRIVER_MEMORY) + private val memoryOverheadMb = submissionSparkConf + .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb + private val driverDockerImage = submissionSparkConf.get(DRIVER_DOCKER_IMAGE) + + override def configureDriver( + driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() + } + val driverCustomAnnotations = ConfigurationUtils + .combinePrefixedKeyValuePairsWithDeprecatedConf( + 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" + + s" Spark bookkeeping operations.") + val allDriverAnnotations = driverCustomAnnotations ++ Map(SPARK_APP_NAME_ANNOTATION -> appName) + val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, KUBERNETES_NODE_SELECTOR_PREFIX, "node selector") + val driverCpuQuantity = new QuantityBuilder(false) + .withAmount(driverCpuCores) + .build() + val driverMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${driverMemoryMb}M") + .build() + val driverMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryWithOverhead}M") + .build() + val maybeCpuLimitQuantity = driverLimitCores.map { limitCores => + ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) + } + val driverContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .addToEnv(driverExtraClasspathEnv.toSeq: _*) + .addNewEnv() + .withName(ENV_DRIVER_MEMORY) + .withValue(driverContainerMemoryWithOverhead + "m") + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_MAIN_CLASS) + .withValue(mainClass) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() + .withNewResources() + .addToRequests("cpu", driverCpuQuantity) + .addToRequests("memory", driverMemoryQuantity) + .addToLimits("memory", driverMemoryLimitQuantity) + .addToLimits(maybeCpuLimitQuantity.toMap.asJava) + .endResources() + .build() + val baseDriverPod = new PodBuilder(driverSpec.driverPod) + .editOrNewMetadata() + .withName(kubernetesDriverPodName) + .addToLabels(driverLabels.asJava) + .addToAnnotations(allDriverAnnotations.asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("Never") + .withNodeSelector(nodeSelector.asJava) + .endSpec() + .build() + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) + .set("spark.app.id", kubernetesAppId) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, kubernetesResourceNamePrefix) + driverSpec.copy( + driverPod = baseDriverPod, + driverSparkConf = resolvedSparkConf, + driverContainer = driverContainer) + } +} 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/kubernetes/submit/submitsteps/DependencyResolutionStep.scala new file mode 100644 index 0000000000000..dddc62410d6c9 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala @@ -0,0 +1,66 @@ +/* + * 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.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.util.Utils + +/** + * Step that configures the classpath, spark.jars, and spark.files for the driver given that + * the init-container will download files to the download paths and that the user may provide + * files with local:// schemes. Note that this is separate from the init-container bootstrap + * step because jars with local schemes can still be provided even if the init-container is + * not being used, and those URIs still need to be resolved. + */ +private[spark] class DependencyResolutionStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val resolvedSparkJars = KubernetesFileUtils.resolveSubmittedUris(sparkJars, jarsDownloadPath) + val resolvedSparkFiles = KubernetesFileUtils.resolveSubmittedUris(sparkFiles, filesDownloadPath) + val sparkConfResolvedSparkDependencies = driverSpec.driverSparkConf.clone() + if (resolvedSparkJars.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.jars", resolvedSparkJars.mkString(",")) + } + if (resolvedSparkFiles.nonEmpty) { + sparkConfResolvedSparkDependencies.set("spark.files", resolvedSparkFiles.mkString(",")) + } + val resolvedClasspath = KubernetesFileUtils.resolveFilePaths(sparkJars, jarsDownloadPath) + val driverContainerWithResolvedClasspath = if (resolvedClasspath.nonEmpty) { + new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_MOUNTED_CLASSPATH) + .withValue(resolvedClasspath.mkString(File.pathSeparator)) + .endEnv() + .build() + } else { + driverSpec.driverContainer + } + driverSpec.copy( + driverContainer = driverContainerWithResolvedClasspath, + driverSparkConf = sparkConfResolvedSparkDependencies) + } +} 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/kubernetes/submit/submitsteps/DriverConfigurationStep.scala new file mode 100644 index 0000000000000..8070e32371f94 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala @@ -0,0 +1,28 @@ +/* + * 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 + +/** + * Represents a step in preparing the Kubernetes driver. + */ +private[spark] trait DriverConfigurationStep { + + /** + * Apply some transformation to the previous state of the driver to add a new feature to it. + */ + def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec +} 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/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala new file mode 100644 index 0000000000000..70a108edc8678 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -0,0 +1,223 @@ +/* + * 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.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret, SecretBuilder} +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._ + +/** + * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials + * to request executors. + */ +private[spark] class DriverKubernetesCredentialsStep( + submissionSparkConf: SparkConf, + kubernetesResourceNamePrefix: String) extends DriverConfigurationStep { + + private val maybeMountedOAuthTokenFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX") + private val maybeMountedClientKeyFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX") + private val maybeMountedClientCertFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX") + private val maybeMountedCaCertFile = submissionSparkConf.getOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX") + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val driverSparkConf = driverSpec.driverSparkConf.clone() + val oauthTokenBase64 = submissionSparkConf + .getOption(s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX") + .map { token => + BaseEncoding.base64().encode(token.getBytes(StandardCharsets.UTF_8)) + } + val caCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + s"Driver CA cert file provided at %s does not exist or is not a file.") + val clientKeyDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + s"Driver client key file provided at %s does not exist or is not a file.") + val clientCertDataBase64 = safeFileConfToBase64( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + s"Driver client cert file provided at %s does not exist or is not a file.") + val driverSparkConfWithCredentialsLocations = setDriverPodKubernetesCredentialLocations( + driverSparkConf, + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + val kubernetesCredentialsSecret = createCredentialsSecret( + oauthTokenBase64, + caCertDataBase64, + clientKeyDataBase64, + clientCertDataBase64) + val driverPodWithMountedKubernetesCredentials = kubernetesCredentialsSecret.map { secret => + new PodBuilder(driverSpec.driverPod) + .editOrNewSpec() + .addNewVolume() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withNewSecret().withSecretName(secret.getMetadata.getName).endSecret() + .endVolume() + .endSpec() + .build() + }.getOrElse(driverSpec.driverPod) + val driverContainerWithMountedSecretVolume = kubernetesCredentialsSecret.map { secret => + new ContainerBuilder(driverSpec.driverContainer) + .addNewVolumeMount() + .withName(DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + .withMountPath(DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + .endVolumeMount() + .build() + }.getOrElse(driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithMountedKubernetesCredentials, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ kubernetesCredentialsSecret.toSeq, + driverSparkConf = driverSparkConfWithCredentialsLocations, + driverContainer = driverContainerWithMountedSecretVolume) + } + + private def createCredentialsSecret( + driverOAuthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): Option[Secret] = { + val allSecretData = + resolveSecretData( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME) ++ + resolveSecretData( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME) ++ + resolveSecretData( + maybeMountedOAuthTokenFile, + driverOAuthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME) + if (allSecretData.isEmpty) { + None + } else { + Some(new SecretBuilder() + .withNewMetadata() + .withName(s"$kubernetesResourceNamePrefix-kubernetes-credentials") + .endMetadata() + .withData(allSecretData.asJava) + .build()) + } + } + + private def setDriverPodKubernetesCredentialLocations( + driverSparkConf: SparkConf, + driverOauthTokenBase64: Option[String], + driverCaCertDataBase64: Option[String], + driverClientKeyDataBase64: Option[String], + driverClientCertDataBase64: Option[String]): SparkConf = { + val resolvedMountedOAuthTokenFile = resolveSecretLocation( + maybeMountedOAuthTokenFile, + driverOauthTokenBase64, + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH) + val resolvedMountedClientKeyFile = resolveSecretLocation( + maybeMountedClientKeyFile, + driverClientKeyDataBase64, + DRIVER_CREDENTIALS_CLIENT_KEY_PATH) + val resolvedMountedClientCertFile = resolveSecretLocation( + maybeMountedClientCertFile, + driverClientCertDataBase64, + DRIVER_CREDENTIALS_CLIENT_CERT_PATH) + val resolvedMountedCaCertFile = resolveSecretLocation( + maybeMountedCaCertFile, + driverCaCertDataBase64, + DRIVER_CREDENTIALS_CA_CERT_PATH) + val sparkConfWithCredentialLocations = driverSparkConf + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + resolvedMountedCaCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + resolvedMountedClientKeyFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + resolvedMountedClientCertFile) + .setOption( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + resolvedMountedOAuthTokenFile) + // Redact all OAuth token values + sparkConfWithCredentialLocations + .getAll + .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)).map(_._1) + .foreach { + sparkConfWithCredentialLocations.set(_, "") + } + sparkConfWithCredentialLocations + } + + private def safeFileConfToBase64( + conf: String, + fileNotFoundFormatString: String): Option[String] = { + submissionSparkConf.getOption(conf) + .map(new File(_)) + .map { file => + require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath)) + BaseEncoding.base64().encode(Files.toByteArray(file)) + } + } + + private def resolveSecretLocation( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + mountedCanonicalLocation: String): Option[String] = { + mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => { + mountedCanonicalLocation + })) + } + + private def resolveSecretData( + mountedUserSpecified: Option[String], + valueMountedFromSubmitter: Option[String], + secretName: String): Map[String, String] = { + mountedUserSpecified.map { _ => Map.empty[String, String]} + .getOrElse { + valueMountedFromSubmitter.map { valueBase64 => + Map(secretName -> valueBase64) + }.getOrElse(Map.empty[String, String]) + } + } + + private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = { + new OptionSettableSparkConf(sparkConf) + } +} + +private class OptionSettableSparkConf(sparkConf: SparkConf) { + def setOption(configEntry: String, option: Option[String]): SparkConf = { + option.map( opt => { + sparkConf.set(configEntry, opt) + }).getOrElse(sparkConf) + } +} 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/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala new file mode 100644 index 0000000000000..29cad18c484c0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.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.deploy.kubernetes.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} + +/** + * Configures the init-container that bootstraps dependencies into the driver pod. + */ +private[spark] class InitContainerBootstrapStep( + initContainerConfigurationSteps: Seq[InitContainerConfigurationStep], + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + var currentInitContainerSpec = InitContainerSpec( + initContainerProperties = Map.empty[String, String], + additionalDriverSparkConf = Map.empty[String, String], + initContainer = new ContainerBuilder().build(), + driverContainer = driverSpec.driverContainer, + podToInitialize = driverSpec.driverPod, + initContainerDependentResources = Seq.empty[HasMetadata]) + for (nextStep <- initContainerConfigurationSteps) { + currentInitContainerSpec = nextStep.configureInitContainer(currentInitContainerSpec) + } + val configMap = PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( + initContainerConfigMapName, + initContainerConfigMapKey, + currentInitContainerSpec.initContainerProperties) + val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone() + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, initContainerConfigMapName) + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, initContainerConfigMapKey) + .setAll(currentInitContainerSpec.additionalDriverSparkConf) + val resolvedDriverPod = InitContainerUtil.appendInitContainer( + currentInitContainerSpec.podToInitialize, currentInitContainerSpec.initContainer) + driverSpec.copy( + driverPod = resolvedDriverPod, + driverContainer = currentInitContainerSpec.driverContainer, + driverSparkConf = resolvedDriverSparkConf, + otherKubernetesResources = + driverSpec.otherKubernetesResources ++ + currentInitContainerSpec.initContainerDependentResources ++ + Seq(configMap)) + } +} 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/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala new file mode 100644 index 0000000000000..3ec4b6c4df10f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala @@ -0,0 +1,47 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder} + +import org.apache.spark.SparkConf + +/** + * Represents the components and characteristics of a Spark driver. The driver can be considered + * as being comprised of the driver pod itself, any other Kubernetes resources that the driver + * pod depends on, and the SparkConf that should be supplied to the Spark application. The driver + * container should be operated on via the specific field of this case class as opposed to trying + * to edit the container directly on the pod. The driver container should be attached at the + * end of executing all submission steps. + */ +private[spark] case class KubernetesDriverSpec( + driverPod: Pod, + driverContainer: Container, + otherKubernetesResources: Seq[HasMetadata], + driverSparkConf: SparkConf) + +private[spark] object KubernetesDriverSpec { + def initialSpec(initialSparkConf: SparkConf): KubernetesDriverSpec = { + KubernetesDriverSpec( + // Set new metadata and a new spec so that submission steps can use + // PodBuilder#editMetadata() and/or PodBuilder#editSpec() safely. + new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(), + new ContainerBuilder().build(), + Seq.empty[HasMetadata], + initialSparkConf.clone()) + } +} 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/kubernetes/submit/submitsteps/PythonStep.scala new file mode 100644 index 0000000000000..024d643ddf9fd --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.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.kubernetes.submit.submitsteps + +import io.fabric8.kubernetes.api.model.ContainerBuilder + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + +private[spark] class PythonStep( + primaryPyFile: String, + otherPyFiles: Seq[String], + filesDownloadPath: String) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val resolvedOtherPyFilesString = if (otherPyFiles.isEmpty) { + "null" + } else { + KubernetesFileUtils.resolveFilePaths(otherPyFiles, filesDownloadPath).mkString(",") + } + val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath)) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(resolvedOtherPyFilesString) + .endEnv() + driverSpec.copy(driverContainer = withPythonPrimaryFileContainer.build()) + } +} 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/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..60bf27beacaaf --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -0,0 +1,63 @@ +/* + * 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.initcontainer + +import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + +private[spark] class BaseInitContainerConfigurationStep( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + configMapName: String, + configMapKey: String, + podAndInitContainerBootstrap: SparkPodInitContainerBootstrap) + extends InitContainerConfigurationStep { + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars) + val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles) + val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(",")) + } else { + Map.empty[String, String] + } + val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) { + Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(",")) + } else { + Map.empty[String, String] + } + val baseInitContainerConfig = Map[String, String]( + INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath, + INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++ + remoteJarsConf ++ + remoteFilesConf + val bootstrappedPodAndInitContainer = + podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + initContainerSpec.podToInitialize, + initContainerSpec.initContainer, + initContainerSpec.driverContainer)) + initContainerSpec.copy( + initContainer = bootstrappedPodAndInitContainer.initContainer, + driverContainer = bootstrappedPodAndInitContainer.mainContainer, + podToInitialize = bootstrappedPodAndInitContainer.pod, + initContainerProperties = baseInitContainerConfig) + } +} 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/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..7b7622c3d4f8b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala @@ -0,0 +1,25 @@ +/* + * 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.initcontainer + +/** + * Represents a step in preparing the init-container for the driver and executors. + */ +private[spark] trait InitContainerConfigurationStep { + + def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec +} 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/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala new file mode 100644 index 0000000000000..e4ea5235af18f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -0,0 +1,131 @@ +/* + * 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.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.util.Utils + +/** + * Returns the complete ordered list of steps required to configure the init-container. + */ +private[spark] class InitContainerConfigurationStepsOrchestrator( + namespace: String, + kubernetesResourceNamePrefix: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + dockerImagePullPolicy: String, + driverLabels: Map[String, String], + initContainerConfigMapName: String, + initContainerConfigMapKey: String, + submissionSparkConf: SparkConf) { + + private val submittedResourcesSecretName = s"$kubernetesResourceNamePrefix-init-secret" + private val resourceStagingServerUri = submissionSparkConf.get(RESOURCE_STAGING_SERVER_URI) + private val resourceStagingServerInternalUri = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI) + private val initContainerImage = submissionSparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) + private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + private val maybeResourceStagingServerInternalTrustStore = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE)) + private val maybeResourceStagingServerInternalTrustStorePassword = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD)) + private val maybeResourceStagingServerInternalTrustStoreType = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE)) + private val maybeResourceStagingServerInternalClientCert = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)) + private val resourceStagingServerInternalSslEnabled = + submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) + .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) + .getOrElse(false) + OptionRequirements.requireNandDefined( + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStore, + "Cannot provide both a certificate file and a trustStore file for init-containers to" + + " use for contacting the resource staging server over TLS.") + + require(maybeResourceStagingServerInternalTrustStore.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "TrustStore URI used for contacting the resource staging server from init containers must" + + " have no scheme, or scheme file://, or scheme local://.") + + require(maybeResourceStagingServerInternalClientCert.forall { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match { + case "file" | "local" => true + case _ => false + } + }, "Client cert file URI used for contacting the resource staging server from init containers" + + " must have no scheme, or scheme file://, or scheme local://.") + + def getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = { + val initContainerBootstrap = new SparkPodInitContainerBootstrapImpl( + initContainerImage, + dockerImagePullPolicy, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + initContainerConfigMapName, + initContainerConfigMapKey) + val baseInitContainerStep = new BaseInitContainerConfigurationStep( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + initContainerConfigMapName, + initContainerConfigMapKey, + initContainerBootstrap) + val submittedResourcesInitContainerStep = resourceStagingServerUri.map { + stagingServerUri => + val mountSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl( + submittedResourcesSecretName, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) + val submittedDependencyUploader = new SubmittedDependencyUploaderImpl( + driverLabels, + namespace, + stagingServerUri, + sparkJars, + sparkFiles, + new ResourceStagingServerSslOptionsProviderImpl(submissionSparkConf).getSslOptions, + RetrofitClientFactoryImpl) + new SubmittedResourcesInitContainerConfigurationStep( + submittedResourcesSecretName, + resourceStagingServerInternalUri.getOrElse(stagingServerUri), + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, + resourceStagingServerInternalSslEnabled, + maybeResourceStagingServerInternalTrustStore, + maybeResourceStagingServerInternalClientCert, + maybeResourceStagingServerInternalTrustStorePassword, + maybeResourceStagingServerInternalTrustStoreType, + submittedDependencyUploader, + mountSecretPlugin) + } + Seq(baseInitContainerStep) ++ submittedResourcesInitContainerStep.toSeq + } +} 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/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala new file mode 100644 index 0000000000000..5b5ac3c1f17c2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala @@ -0,0 +1,41 @@ +/* + * 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.initcontainer + +import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} + +/** + * Represents a given configuration of the init-container, informing the main + * InitContainerBootstrapStep of how the driver should be configured. This includes: + *

    + * - What properties should be set on the init-container, + * - What Spark properties should be set on the driver's SparkConf given this init-container, + * - The spec of the init container itself, + * - The spec of the main container so that it can be modified to share volumes with the + * init-container + * - The spec of the pod EXCEPT for the addition of the given init-container (e.g. volumes + * the init-container needs or modifications to a main container that shares data with the + * init-container), + * - Any Kubernetes resources that need to be created for the init-container's function. + */ +private[spark] case class InitContainerSpec( + initContainerProperties: Map[String, String], + additionalDriverSparkConf: Map[String, String], + initContainer: Container, + driverContainer: Container, + podToInitialize: Pod, + initContainerDependentResources: Seq[HasMetadata]) 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/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala new file mode 100644 index 0000000000000..7aa27a1de6811 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala @@ -0,0 +1,146 @@ +/* + * 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.initcontainer + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +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.internal.config.OptionalConfigEntry +import org.apache.spark.util.Utils + +private[spark] class SubmittedResourcesInitContainerConfigurationStep( + submittedResourcesSecretName: String, + internalResourceStagingServerUri: String, + initContainerSecretMountPath: String, + resourceStagingServerSslEnabled: Boolean, + maybeInternalTrustStoreUri: Option[String], + maybeInternalClientCertUri: Option[String], + maybeInternalTrustStorePassword: Option[String], + maybeInternalTrustStoreType: Option[String], + submittedDependencyUploader: SubmittedDependencyUploader, + submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin) + extends InitContainerConfigurationStep { + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val jarsIdAndSecret = submittedDependencyUploader.uploadJars() + val filesIdAndSecret = submittedDependencyUploader.uploadFiles() + + val submittedResourcesInitContainerProperties = Map[String, String]( + RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsIdAndSecret.resourceId, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesIdAndSecret.resourceId, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++ + resolveSecretPath( + maybeInternalTrustStoreUri, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, + "TrustStore URI") ++ + resolveSecretPath( + maybeInternalClientCertUri, + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM, + "Client certificate URI") ++ + maybeInternalTrustStorePassword.map { password => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) + }.toMap ++ + maybeInternalTrustStoreType.map { storeType => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) + }.toMap + val initContainerSecret = createResourceStagingServerSecret( + jarsIdAndSecret.resourceSecret, filesIdAndSecret.resourceSecret) + val additionalDriverSparkConf = + Map( + EXECUTOR_INIT_CONTAINER_SECRET.key -> initContainerSecret.getMetadata.getName, + EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> initContainerSecretMountPath) + val initContainerWithSecretVolumeMount = submittedResourcesSecretPlugin + .mountResourceStagingServerSecretIntoInitContainer(initContainerSpec.initContainer) + val podWithSecretVolume = submittedResourcesSecretPlugin + .addResourceStagingServerSecretVolumeToPod(initContainerSpec.podToInitialize) + initContainerSpec.copy( + initContainer = initContainerWithSecretVolumeMount, + podToInitialize = podWithSecretVolume, + initContainerDependentResources = + initContainerSpec.initContainerDependentResources ++ Seq(initContainerSecret), + initContainerProperties = + initContainerSpec.initContainerProperties ++ submittedResourcesInitContainerProperties, + additionalDriverSparkConf = additionalDriverSparkConf) + } + + private def createResourceStagingServerSecret( + jarsResourceSecret: String, filesResourceSecret: String): Secret = { + val trustStoreBase64 = convertFileToBase64IfSubmitterLocal( + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, maybeInternalTrustStoreUri) + val clientCertBase64 = convertFileToBase64IfSubmitterLocal( + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, maybeInternalClientCertUri) + val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) + val filesSecretBase64 = BaseEncoding.base64().encode( + filesResourceSecret.getBytes(Charsets.UTF_8)) + val secretData = Map( + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> jarsSecretBase64, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> filesSecretBase64) ++ + trustStoreBase64 ++ + clientCertBase64 + val kubernetesSecret = new SecretBuilder() + .withNewMetadata() + .withName(submittedResourcesSecretName) + .endMetadata() + .addToData(secretData.asJava) + .build() + kubernetesSecret + } + + private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String]) + : Map[String, String] = { + secretUri.filter { trustStore => + Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file" + }.map { uri => + val file = new File(Utils.resolveURI(uri).getPath) + require(file.isFile, "Dependency server trustStore provided at" + + file.getAbsolutePath + " does not exist or is not a file.") + (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file))) + }.toMap + } + + private def resolveSecretPath( + maybeUri: Option[String], + secretKey: String, + configEntry: OptionalConfigEntry[String], + uriType: String): Map[String, String] = { + maybeUri.map(Utils.resolveURI).map { uri => + val resolvedPath = Option(uri.getScheme).getOrElse("file") match { + case "file" => s"$initContainerSecretMountPath/$secretKey" + case "local" => uri.getPath + case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" + + s" local://, file://, or empty.") + } + (configEntry.key, resolvedPath) + }.toMap + } +} 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/kubernetes/FileFetcher.scala new file mode 100644 index 0000000000000..d050e0a41a15a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala @@ -0,0 +1,24 @@ +/* + * 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.rest.kubernetes + +import java.io.File + +// Extracted for testing so that unit tests don't have to depend on Utils.fetchFile +private[spark] trait FileFetcher { + def fetchFile(uri: String, targetDir: File): Unit +} 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/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala new file mode 100644 index 0000000000000..0e274678ad6f0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala @@ -0,0 +1,225 @@ +/* + * 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.rest.kubernetes + +import java.io.File +import java.util.concurrent.TimeUnit + +import com.google.common.base.Charsets +import com.google.common.io.Files +import com.google.common.util.concurrent.SettableFuture +import okhttp3.ResponseBody +import retrofit2.{Call, Callback, Response} +import scala.concurrent.{ExecutionContext, Future} +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.internal.Logging +import org.apache.spark.util.{ThreadUtils, Utils} + +private trait WaitableCallback[T] extends Callback[T] { + private val complete = SettableFuture.create[Boolean] + + override final def onFailure(call: Call[T], t: Throwable): Unit = complete.setException(t) + + override final def onResponse(call: Call[T], response: Response[T]): Unit = { + require(response.code() >= 200 && response.code() < 300, Option(response.errorBody()) + .map(_.string()) + .getOrElse(s"Error executing HTTP request, but error body was not provided.")) + handleResponse(response.body()) + complete.set(true) + } + + protected def handleResponse(body: T): Unit + + final def waitForCompletion(time: Long, timeUnit: TimeUnit): Unit = { + complete.get(time, timeUnit) + } +} + +private class DownloadTarGzCallback(downloadDir: File) extends WaitableCallback[ResponseBody] { + + override def handleResponse(responseBody: ResponseBody): Unit = { + Utils.tryWithResource(responseBody.byteStream()) { responseStream => + CompressionUtils.unpackTarStreamToDirectory(responseStream, downloadDir) + } + } +} +/** + * Process that fetches files from a resource staging server and/or arbitrary remote locations. + * + * The init-container can handle fetching files from any of those sources, but not all of the + * sources need to be specified. This allows for composing multiple instances of this container + * with different configurations for different download sources, or using the same container to + * download everything at once. + */ +private[spark] class KubernetesSparkDependencyDownloadInitContainer( + sparkConf: SparkConf, + retrofitClientFactory: RetrofitClientFactory, + fileFetcher: FileFetcher, + resourceStagingServerSslOptions: SSLOptions) extends Logging { + + private implicit val downloadExecutor = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("download-executor")) + private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) + + private val maybeDownloadJarsResourceIdentifier = sparkConf + .get(INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER) + private val downloadJarsSecretLocation = new File( + sparkConf.get(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION)) + private val maybeDownloadFilesResourceIdentifier = sparkConf + .get(INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER) + private val downloadFilesSecretLocation = new File( + sparkConf.get(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION)) + + private val jarsDownloadDir = new File( + sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)) + private val filesDownloadDir = new File( + sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION)) + + private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS) + private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES) + + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + + def run(): Unit = { + val resourceStagingServerJarsDownload = Future[Unit] { + downloadResourcesFromStagingServer( + maybeDownloadJarsResourceIdentifier, + downloadJarsSecretLocation, + jarsDownloadDir, + "Starting to download jars from resource staging server...", + "Finished downloading jars from resource staging server.", + s"Application jars download secret provided at" + + s" ${downloadJarsSecretLocation.getAbsolutePath} does not exist or is not a file.", + s"Application jars download directory provided at" + + s" ${jarsDownloadDir.getAbsolutePath} does not exist or is not a directory.") + } + val resourceStagingServerFilesDownload = Future[Unit] { + downloadResourcesFromStagingServer( + maybeDownloadFilesResourceIdentifier, + downloadFilesSecretLocation, + filesDownloadDir, + "Starting to download files from resource staging server...", + "Finished downloading files from resource staging server.", + s"Application files download secret provided at" + + s" ${downloadFilesSecretLocation.getAbsolutePath} does not exist or is not a file.", + s"Application files download directory provided at" + + s" ${filesDownloadDir.getAbsolutePath} does not exist or is not" + + s" a directory.") + } + val remoteJarsDownload = Future[Unit] { + downloadFiles(remoteJars, + jarsDownloadDir, + s"Remote jars download directory specified at $jarsDownloadDir does not exist" + + s" or is not a directory.") + } + val remoteFilesDownload = Future[Unit] { + downloadFiles(remoteFiles, + filesDownloadDir, + s"Remote files download directory specified at $filesDownloadDir does not exist" + + s" or is not a directory.") + } + waitForFutures( + resourceStagingServerJarsDownload, + resourceStagingServerFilesDownload, + remoteJarsDownload, + remoteFilesDownload) + } + + private def downloadResourcesFromStagingServer( + maybeResourceId: Option[String], + resourceSecretLocation: File, + resourceDownloadDir: File, + downloadStartMessage: String, + downloadFinishedMessage: String, + errMessageOnSecretNotAFile: String, + errMessageOnDownloadDirNotADirectory: String): Unit = { + maybeResourceStagingServerUri.foreach { resourceStagingServerUri => + maybeResourceId.foreach { resourceId => + require(resourceSecretLocation.isFile, errMessageOnSecretNotAFile) + require(resourceDownloadDir.isDirectory, errMessageOnDownloadDirNotADirectory) + val service = retrofitClientFactory.createRetrofitClient( + resourceStagingServerUri, + classOf[ResourceStagingServiceRetrofit], + resourceStagingServerSslOptions) + val resourceSecret = Files.toString(resourceSecretLocation, Charsets.UTF_8) + val downloadResourceCallback = new DownloadTarGzCallback(resourceDownloadDir) + logInfo(downloadStartMessage) + service.downloadResources(resourceId, resourceSecret).enqueue(downloadResourceCallback) + downloadResourceCallback.waitForCompletion(downloadTimeoutMinutes, TimeUnit.MINUTES) + logInfo(downloadFinishedMessage) + } + } + } + + private def downloadFiles( + filesCommaSeparated: Option[String], + downloadDir: File, + errMessageOnDestinationNotADirectory: String): Unit = { + if (filesCommaSeparated.isDefined) { + require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory) + } + filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file => + fileFetcher.fetchFile(file, downloadDir) + } + } + + private def waitForFutures(futures: Future[_]*) { + futures.foreach { + ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES)) + } + } +} + +private class FileFetcherImpl(sparkConf: SparkConf, securityManager: SparkSecurityManager) + extends FileFetcher { + def fetchFile(uri: String, targetDir: File): Unit = { + Utils.fetchFile( + url = uri, + targetDir = targetDir, + conf = sparkConf, + securityMgr = securityManager, + hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf), + timestamp = System.currentTimeMillis(), + useCache = false) + } +} + +object KubernetesSparkDependencyDownloadInitContainer extends Logging { + def main(args: Array[String]): Unit = { + logInfo("Starting init-container to download Spark application dependencies.") + val sparkConf = if (args.nonEmpty) { + SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) + } else { + new SparkConf(true) + } + val securityManager = new SparkSecurityManager(sparkConf) + val resourceStagingServerSslOptions = + new ResourceStagingServerSslOptionsProviderImpl(sparkConf).getSslOptions + val fileFetcher = new FileFetcherImpl(sparkConf, securityManager) + new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + RetrofitClientFactoryImpl, + fileFetcher, + resourceStagingServerSslOptions).run() + logInfo("Finished downloading application dependencies.") + } +} 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/kubernetes/PemsToKeyStoreConverter.scala new file mode 100644 index 0000000000000..17f90118e150d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala @@ -0,0 +1,139 @@ +/* + * 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.rest.kubernetes + +import java.io.{File, FileInputStream, FileOutputStream, InputStreamReader} +import java.security.{KeyStore, PrivateKey} +import java.security.cert.Certificate +import java.util.UUID + +import com.google.common.base.Charsets +import org.bouncycastle.asn1.pkcs.PrivateKeyInfo +import org.bouncycastle.cert.X509CertificateHolder +import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter +import org.bouncycastle.openssl.{PEMKeyPair, PEMParser} +import org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter +import scala.collection.mutable + +import org.apache.spark.SparkException +import org.apache.spark.util.Utils + +private[spark] object PemsToKeyStoreConverter { + + /** + * Loads the given key-cert pair into a temporary keystore file. Returns the File pointing + * to where the keyStore was written to disk. + */ + def convertPemsToTempKeyStoreFile( + keyPemFile: File, + certPemFile: File, + keyAlias: String, + keyStorePassword: String, + keyPassword: String, + keyStoreType: Option[String]): File = { + require(keyPemFile.isFile, s"Key PEM file provided at ${keyPemFile.getAbsolutePath}" + + " does not exist or is not a file.") + require(certPemFile.isFile, s"Cert PEM file provided at ${certPemFile.getAbsolutePath}" + + " does not exist or is not a file.") + val privateKey = parsePrivateKeyFromPemFile(keyPemFile) + val certificates = parseCertificatesFromPemFile(certPemFile) + val resolvedKeyStoreType = keyStoreType.getOrElse(KeyStore.getDefaultType) + val keyStore = KeyStore.getInstance(resolvedKeyStoreType) + keyStore.load(null, null) + keyStore.setKeyEntry( + keyAlias, + privateKey, + keyPassword.toCharArray, + certificates) + val keyStoreDir = Utils.createTempDir("temp-keystores") + val keyStoreFile = new File(keyStoreDir, s"keystore-${UUID.randomUUID()}.$resolvedKeyStoreType") + Utils.tryWithResource(new FileOutputStream(keyStoreFile)) { storeStream => + keyStore.store(storeStream, keyStorePassword.toCharArray) + } + keyStoreFile + } + + def convertCertPemToTrustStore( + certPemFile: File, + trustStoreType: Option[String]): KeyStore = { + require(certPemFile.isFile, s"Cert PEM file provided at ${certPemFile.getAbsolutePath}" + + " does not exist or is not a file.") + val trustStore = KeyStore.getInstance(trustStoreType.getOrElse(KeyStore.getDefaultType)) + trustStore.load(null, null) + parseCertificatesFromPemFile(certPemFile).zipWithIndex.foreach { case (cert, index) => + trustStore.setCertificateEntry(s"certificate-$index", cert) + } + trustStore + } + + def convertCertPemToTempTrustStoreFile( + certPemFile: File, + trustStorePassword: String, + trustStoreType: Option[String]): File = { + val trustStore = convertCertPemToTrustStore(certPemFile, trustStoreType) + val tempTrustStoreDir = Utils.createTempDir(namePrefix = "temp-trustStore") + val tempTrustStoreFile = new File(tempTrustStoreDir, + s"trustStore.${trustStoreType.getOrElse(KeyStore.getDefaultType)}") + Utils.tryWithResource(new FileOutputStream(tempTrustStoreFile)) { + trustStore.store(_, trustStorePassword.toCharArray) + } + tempTrustStoreFile + } + + private def withPemParsedFromFile[T](pemFile: File)(f: (PEMParser => T)): T = { + Utils.tryWithResource(new FileInputStream(pemFile)) { pemStream => + Utils.tryWithResource(new InputStreamReader(pemStream, Charsets.UTF_8)) { pemReader => + Utils.tryWithResource(new PEMParser(pemReader))(f) + } + } + } + + private def parsePrivateKeyFromPemFile(keyPemFile: File): PrivateKey = { + withPemParsedFromFile(keyPemFile) { keyPemParser => + val converter = new JcaPEMKeyConverter + keyPemParser.readObject() match { + case privateKey: PrivateKeyInfo => + converter.getPrivateKey(privateKey) + case keyPair: PEMKeyPair => + converter.getPrivateKey(keyPair.getPrivateKeyInfo) + case _ => + throw new SparkException(s"Key file provided at ${keyPemFile.getAbsolutePath}" + + s" is not a key pair or private key PEM file.") + } + } + } + + private def parseCertificatesFromPemFile(certPemFile: File): Array[Certificate] = { + withPemParsedFromFile(certPemFile) { certPemParser => + val certificates = mutable.Buffer[Certificate]() + var pemObject = certPemParser.readObject() + while (pemObject != null) { + pemObject match { + case certificate: X509CertificateHolder => + val converter = new JcaX509CertificateConverter + certificates += converter.getCertificate(certificate) + case _ => + } + pemObject = certPemParser.readObject() + } + if (certificates.isEmpty) { + throw new SparkException(s"No certificates found in ${certPemFile.getAbsolutePath}") + } + certificates.toArray + } + } +} 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/kubernetes/ResourceStagingServer.scala new file mode 100644 index 0000000000000..0b97317eba8b1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala @@ -0,0 +1,142 @@ +/* + * 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.rest.kubernetes + +import java.io.File + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import io.fabric8.kubernetes.client.Config +import org.eclipse.jetty.http.HttpVersion +import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector, SslConnectionFactory} +import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} +import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler} +import org.glassfish.jersey.media.multipart.MultiPartFeature +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.internal.Logging +import org.apache.spark.util.{SystemClock, ThreadUtils, Utils} + +private[spark] class ResourceStagingServer( + port: Int, + serviceInstance: ResourceStagingService, + sslOptionsProvider: ResourceStagingServerSslOptionsProvider) extends Logging { + + private var jettyServer: Option[Server] = None + + def start(): Unit = synchronized { + val threadPool = new QueuedThreadPool + val contextHandler = new ServletContextHandler() + val jsonProvider = new JacksonJaxbJsonProvider() + jsonProvider.setMapper(new ObjectMapper().registerModule(new DefaultScalaModule)) + val resourceConfig = new ResourceConfig().registerInstances( + serviceInstance, + jsonProvider, + new MultiPartFeature) + val servletHolder = new ServletHolder("main", new ServletContainer(resourceConfig)) + contextHandler.setContextPath("/api/") + contextHandler.addServlet(servletHolder, "/*") + threadPool.setDaemon(true) + val resolvedConnectionFactories = sslOptionsProvider.getSslOptions + .createJettySslContextFactory() + .map(sslFactory => { + val sslConnectionFactory = new SslConnectionFactory( + sslFactory, HttpVersion.HTTP_1_1.asString()) + val rawHttpConfiguration = new HttpConfiguration() + rawHttpConfiguration.setSecureScheme("https") + rawHttpConfiguration.setSecurePort(port) + val rawHttpConnectionFactory = new HttpConnectionFactory(rawHttpConfiguration) + Array(sslConnectionFactory, rawHttpConnectionFactory) + }).getOrElse(Array(new HttpConnectionFactory())) + val server = new Server(threadPool) + val connector = new ServerConnector( + server, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("DependencyServer-Executor", true), + null, + -1, + -1, + resolvedConnectionFactories: _*) + connector.setPort(port) + server.addConnector(connector) + server.setHandler(contextHandler) + server.start() + jettyServer = Some(server) + logInfo(s"Resource staging server started on port $port.") + } + + def join(): Unit = jettyServer.foreach(_.join()) + + def stop(): Unit = synchronized { + jettyServer.foreach(_.stop()) + jettyServer = None + } +} + +object ResourceStagingServer { + def main(args: Array[String]): Unit = { + val sparkConf = if (args.nonEmpty) { + SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) + } else { + new SparkConf(true) + } + val apiServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_API_SERVER_URL) + val initialAccessExpirationMs = sparkConf.get( + RESOURCE_STAGING_SERVER_INITIAL_ACCESS_EXPIRATION_TIMEOUT) + val dependenciesRootDir = Utils.createTempDir(namePrefix = "local-application-dependencies") + val useServiceAccountCredentials = sparkConf.get( + RESOURCE_STAGING_SERVER_USE_SERVICE_ACCOUNT_CREDENTIALS) + // Namespace doesn't matter because we list resources from various namespaces + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( + apiServerUri, + None, + APISERVER_AUTH_RESOURCE_STAGING_SERVER_CONF_PREFIX, + sparkConf, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)) + .filter( _ => useServiceAccountCredentials), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)) + .filter( _ => useServiceAccountCredentials)) + + val stagedResourcesStore = new StagedResourcesStoreImpl(dependenciesRootDir) + val stagedResourcesCleaner = new StagedResourcesCleanerImpl( + stagedResourcesStore, + kubernetesClient, + ThreadUtils.newDaemonSingleThreadScheduledExecutor("resource-expiration"), + new SystemClock(), + initialAccessExpirationMs) + stagedResourcesCleaner.start() + val serviceInstance = new ResourceStagingServiceImpl( + stagedResourcesStore, stagedResourcesCleaner) + val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) + val server = new ResourceStagingServer( + port = sparkConf.get(RESOURCE_STAGING_SERVER_PORT), + serviceInstance = serviceInstance, + sslOptionsProvider = sslOptionsProvider) + server.start() + try { + server.join() + } finally { + server.stop() + } + } +} 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/kubernetes/ResourceStagingServerSslOptionsProvider.scala new file mode 100644 index 0000000000000..cb1e65421c013 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala @@ -0,0 +1,163 @@ +/* + * 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.rest.kubernetes + +import java.io.File +import java.security.SecureRandom + +import com.google.common.base.Charsets +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.internal.Logging + +private[spark] trait ResourceStagingServerSslOptionsProvider { + def getSslOptions: SSLOptions +} + +private[spark] class ResourceStagingServerSslOptionsProviderImpl(sparkConf: SparkConf) + extends ResourceStagingServerSslOptionsProvider with Logging { + + private val SECURE_RANDOM = new SecureRandom() + + def getSslOptions: SSLOptions = { + val baseSslOptions = new SecurityManager(sparkConf) + .getSSLOptions(RESOURCE_STAGING_SERVER_SSL_NAMESPACE) + val maybeKeyPem = sparkConf.get(RESOURCE_STAGING_SERVER_KEY_PEM) + val maybeServerCertPem = sparkConf.get(RESOURCE_STAGING_SERVER_CERT_PEM) + val maybeKeyStorePasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE) + val maybeKeyPasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE) + val maybeClientCertPem = sparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM) + + logSslConfigurations( + baseSslOptions, + maybeKeyPem, + maybeServerCertPem, + maybeKeyStorePasswordFile, + maybeKeyPasswordFile, + maybeClientCertPem) + + OptionRequirements.requireNandDefined( + baseSslOptions.keyStore, + maybeKeyPem, + "Shouldn't provide both key PEM and keyStore files for TLS.") + OptionRequirements.requireNandDefined( + baseSslOptions.keyStore, + maybeServerCertPem, + "Shouldn't provide both certificate PEM and keyStore files for TLS.") + OptionRequirements.requireNandDefined( + baseSslOptions.keyStorePassword, + maybeKeyStorePasswordFile, + "Shouldn't provide both the keyStore password value and the keyStore password file.") + OptionRequirements.requireNandDefined( + baseSslOptions.keyPassword, + maybeKeyPasswordFile, + "Shouldn't provide both a keyStore key password value and a keyStore key password file.") + OptionRequirements.requireBothOrNeitherDefined( + maybeKeyPem, + maybeServerCertPem, + "When providing a certificate PEM file, the key PEM file must also be provided.", + "When providing a key PEM file, the certificate PEM file must also be provided.") + OptionRequirements.requireNandDefined(baseSslOptions.trustStore, maybeClientCertPem, + "Shouldn't provide both the trustStore and a client certificate PEM file.") + + val resolvedKeyStorePassword = baseSslOptions.keyStorePassword + .orElse(maybeKeyStorePasswordFile.map { keyStorePasswordFile => + safeFileToString(keyStorePasswordFile, "KeyStore password file") + }) + .orElse(maybeKeyPem.map { _ => randomPassword()}) + val resolvedKeyStoreKeyPassword = baseSslOptions.keyPassword + .orElse(maybeKeyPasswordFile.map { keyPasswordFile => + safeFileToString(keyPasswordFile, "KeyStore key password file") + }) + .orElse(maybeKeyPem.map { _ => randomPassword()}) + val resolvedKeyStore = baseSslOptions.keyStore.orElse { + for { + keyPem <- maybeKeyPem + certPem <- maybeServerCertPem + keyStorePassword <- resolvedKeyStorePassword + keyPassword <- resolvedKeyStoreKeyPassword + } yield { + val keyPemFile = new File(keyPem) + val certPemFile = new File(certPem) + PemsToKeyStoreConverter.convertPemsToTempKeyStoreFile( + keyPemFile, + certPemFile, + "key", + keyStorePassword, + keyPassword, + baseSslOptions.keyStoreType) + } + } + val resolvedTrustStorePassword = baseSslOptions.trustStorePassword + .orElse(maybeClientCertPem.map( _ => "defaultTrustStorePassword")) + val resolvedTrustStore = baseSslOptions.trustStore.orElse { + for { + clientCertPem <- maybeClientCertPem + trustStorePassword <- resolvedTrustStorePassword + } yield { + val certPemFile = new File(clientCertPem) + PemsToKeyStoreConverter.convertCertPemToTempTrustStoreFile( + certPemFile, + trustStorePassword, + baseSslOptions.trustStoreType) + } + } + baseSslOptions.copy( + keyStore = resolvedKeyStore, + keyStorePassword = resolvedKeyStorePassword, + keyPassword = resolvedKeyStoreKeyPassword, + trustStore = resolvedTrustStore) + } + + private def logSslConfigurations( + baseSslOptions: SSLOptions, + maybeKeyPem: Option[String], + maybeServerCertPem: Option[String], + maybeKeyStorePasswordFile: Option[String], + maybeKeyPasswordFile: Option[String], + maybeClientCertPem: Option[String]) = { + logDebug("The following SSL configurations were provided for the resource staging server:") + logDebug(s"KeyStore File: ${baseSslOptions.keyStore.map(_.getAbsolutePath).getOrElse("N/A")}") + logDebug("KeyStore Password: " + + baseSslOptions.keyStorePassword.map(_ => "").getOrElse("N/A")) + logDebug(s"KeyStore Password File: ${maybeKeyStorePasswordFile.getOrElse("N/A")}") + logDebug("Key Password: " + + baseSslOptions.keyPassword.map(_ => "").getOrElse("N/A")) + logDebug(s"Key Password File: ${maybeKeyPasswordFile.getOrElse("N/A")}") + logDebug(s"KeyStore Type: ${baseSslOptions.keyStoreType.getOrElse("N/A")}") + logDebug(s"Key PEM: ${maybeKeyPem.getOrElse("N/A")}") + logDebug(s"Server-side certificate PEM: ${maybeServerCertPem.getOrElse("N/A")}") + logDebug(s"Client-side certificate PEM: ${maybeClientCertPem.getOrElse("N/A")}") + } + + private def safeFileToString(filePath: String, fileType: String): String = { + val file = new File(filePath) + if (!file.isFile) { + throw new SparkException(s"$fileType provided at ${file.getAbsolutePath} does not exist or" + + s" is not a file.") + } + Files.toString(file, Charsets.UTF_8) + } + + private def randomPassword(): String = { + RandomStringUtils.random(1024, 0, Integer.MAX_VALUE, false, false, null, SECURE_RANDOM) + } +} 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/kubernetes/ResourceStagingService.scala new file mode 100644 index 0000000000000..b9d283a99ade9 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala @@ -0,0 +1,92 @@ +/* + * 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.rest.kubernetes + +import java.io.InputStream +import javax.ws.rs.{Consumes, GET, HeaderParam, Path, PathParam, POST, Produces} +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 + +/** + * Service that receives application data that can be retrieved later on. This is primarily used + * in the context of Spark, but the concept is generic enough to be used for arbitrary applications. + * The use case is to have a place for Kubernetes application submitters to bootstrap dynamic, + * heavyweight application data for pods. Application submitters may have data stored on their + * local disks that they want to provide to the pods they create through the API server. ConfigMaps + * are one way to provide this data, but the data in ConfigMaps are stored in etcd which cannot + * maintain data in the hundreds of megabytes in size. + *

    + * The general use case is for an application submitter to ship the dependencies to the server via + * {@link uploadResources}; the application submitter will then receive a unique secure token. + * The application submitter then ought to convert the token into a secret, and use this secret in + * a pod that fetches the uploaded dependencies via {@link downloadResources}. An application can + * provide multiple resource bundles simply by hitting the upload endpoint multiple times and + * downloading each bundle with the appropriate secret. + */ +@Path("/v0") +private[spark] trait ResourceStagingService { + + /** + * Register a resource with the dependency service, so that pods with the given labels can + * retrieve them when they run. + * + * @param resources Application resources to upload, compacted together in tar + gzip format. + * The tarball should contain the files laid out in a flat hierarchy, without + * any directories. We take a stream here to avoid holding these entirely in + * memory. + * @param resourcesOwner A description of the "owner" of a resource. A resource owner is a + * Kubernetes API object in a given namespace, with a specific set of + * labels. When there are no resources of the owner's type in the given + * namespace with the given labels, the resources are cleaned up. The owner + * bundle also includes any Kubernetes credentials that are required for + * resource staging server to watch the object's state over time. + * @return A unique token that should be provided when retrieving these dependencies later. + */ + @POST + @Consumes(Array(MediaType.MULTIPART_FORM_DATA, MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN)) + @Produces(Array(MediaType.APPLICATION_JSON)) + @Path("/resources") + def uploadResources( + @FormDataParam("resources") resources: InputStream, + @FormDataParam("resourcesOwner") resourcesOwner: StagedResourcesOwner) + : SubmittedResourceIdAndSecret + + /** + * Download an application's resources. The resources are provided as a stream, where the stream's + * underlying data matches the stream that was uploaded in uploadResources. + */ + @GET + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) + @Path("/resources/{resourceId}") + def downloadResources( + @PathParam("resourceId") resourceId: String, + @HeaderParam("Authorization") resourceSecret: String): StreamingOutput + + /** + * Health check. + */ + @GET + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.TEXT_PLAIN)) + @Path("/ping") + def ping(): String +} 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/kubernetes/ResourceStagingServiceImpl.scala new file mode 100644 index 0000000000000..7bc21c21619e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.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.deploy.rest.kubernetes + +import java.io.{File, FileOutputStream, InputStream, OutputStream} +import java.security.SecureRandom +import java.util.UUID +import javax.ws.rs.{NotAuthorizedException, NotFoundException} +import javax.ws.rs.core.StreamingOutput + +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.internal.Logging +import org.apache.spark.util.Utils + +private[spark] class ResourceStagingServiceImpl( + stagedResourcesStore: StagedResourcesStore, + stagedResourcesCleaner: StagedResourcesCleaner) + extends ResourceStagingService with Logging { + + override def uploadResources( + resources: InputStream, + resourcesOwner: StagedResourcesOwner): SubmittedResourceIdAndSecret = { + val stagedResources = stagedResourcesStore.addResources( + resourcesOwner.ownerNamespace, resources) + stagedResourcesCleaner.registerResourceForCleaning( + stagedResources.resourceId, resourcesOwner) + SubmittedResourceIdAndSecret(stagedResources.resourceId, stagedResources.resourceSecret) + } + + override def downloadResources(resourceId: String, resourceSecret: String): StreamingOutput = { + val resource = stagedResourcesStore.getResources(resourceId) + .getOrElse(throw new NotFoundException(s"No resource bundle found with id $resourceId")) + if (!resource.resourceSecret.equals(resourceSecret)) { + throw new NotAuthorizedException(s"Unauthorized to download resource with id $resourceId") + } + stagedResourcesCleaner.markResourceAsUsed(resourceId) + new StreamingOutput { + override def write(outputStream: OutputStream) = { + Files.copy(resource.resourcesFile, outputStream) + } + } + } + + override def ping(): String = "pong" +} 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/kubernetes/ResourceStagingServiceRetrofit.scala new file mode 100644 index 0000000000000..5fbf0f9c43970 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.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.rest.kubernetes + +import okhttp3.{RequestBody, ResponseBody} +import retrofit2.Call +import retrofit2.http.{Multipart, Path, Streaming} + +import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret + +/** + * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on + * how to use this service, see the aforementioned JAX-RS based interface. + */ +private[spark] trait ResourceStagingServiceRetrofit { + + @Multipart + @retrofit2.http.POST("api/v0/resources/") + def uploadResources( + @retrofit2.http.Part("resources") resources: RequestBody, + @retrofit2.http.Part("resourcesOwner") resourcesOwner: RequestBody) + : Call[SubmittedResourceIdAndSecret] + + @Streaming + @retrofit2.http.GET("api/v0/resources/{resourceId}") + def downloadResources( + @Path("resourceId") resourceId: String, + @retrofit2.http.Header("Authorization") resourceSecret: String): Call[ResponseBody] + + @retrofit2.http.GET("/api/ping") + def ping(): String +} 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/kubernetes/RetrofitClientFactory.scala new file mode 100644 index 0000000000000..5046cb479054c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala @@ -0,0 +1,107 @@ +/* + * 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.rest.kubernetes + +import java.io.FileInputStream +import java.net.{InetSocketAddress, URI} +import java.security.{KeyStore, SecureRandom} +import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import io.fabric8.kubernetes.client.Config +import okhttp3.{Dispatcher, OkHttpClient} +import retrofit2.Retrofit +import retrofit2.converter.jackson.JacksonConverterFactory +import retrofit2.converter.scalars.ScalarsConverterFactory + +import org.apache.spark.SSLOptions +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ThreadUtils, Utils} + +private[spark] trait RetrofitClientFactory { + def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T +} + +private[spark] object RetrofitClientFactoryImpl extends RetrofitClientFactory with Logging { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + private val SECURE_RANDOM = new SecureRandom() + + def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T = { + val dispatcher = new Dispatcher(ThreadUtils.newDaemonCachedThreadPool(s"http-client-$baseUrl")) + val serviceUri = URI.create(baseUrl) + val maybeAllProxy = Option.apply(System.getProperty(Config.KUBERNETES_ALL_PROXY)) + val serviceUriScheme = serviceUri.getScheme + val maybeHttpProxy = (if (serviceUriScheme.equalsIgnoreCase("https")) { + Option.apply(System.getProperty(Config.KUBERNETES_HTTPS_PROXY)) + } else if (serviceUriScheme.equalsIgnoreCase("http")) { + Option.apply(System.getProperty(Config.KUBERNETES_HTTP_PROXY)) + } else { + maybeAllProxy + }).map(uriStringToProxy) + val maybeNoProxy = Option.apply(System.getProperty(Config.KUBERNETES_NO_PROXY)) + .map(_.split(",")) + .toSeq + .flatten + val resolvedProxy = maybeNoProxy.find(_ == serviceUri.getHost) + .map( _ => java.net.Proxy.NO_PROXY) + .orElse(maybeHttpProxy) + .getOrElse(java.net.Proxy.NO_PROXY) + val okHttpClientBuilder = new OkHttpClient.Builder() + .dispatcher(dispatcher) + .proxy(resolvedProxy) + logDebug(s"Proxying to $baseUrl through address ${resolvedProxy.address()} with proxy of" + + s" type ${resolvedProxy.`type`()}") + sslOptions.trustStore.foreach { trustStoreFile => + require(trustStoreFile.isFile, s"TrustStore provided at ${trustStoreFile.getAbsolutePath}" + + " does not exist, or is not a file.") + val trustStoreType = sslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType) + val trustStore = KeyStore.getInstance(trustStoreType) + val trustStorePassword = sslOptions.trustStorePassword.map(_.toCharArray).orNull + Utils.tryWithResource(new FileInputStream(trustStoreFile)) { + trustStore.load(_, trustStorePassword) + } + val trustManagerFactory = TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm) + trustManagerFactory.init(trustStore) + val trustManagers = trustManagerFactory.getTrustManagers + val sslContext = SSLContext.getInstance("TLSv1.2") + sslContext.init(null, trustManagers, SECURE_RANDOM) + okHttpClientBuilder.sslSocketFactory(sslContext.getSocketFactory, + trustManagers(0).asInstanceOf[X509TrustManager]) + } + val resolvedBaseUrl = if (!baseUrl.endsWith("/")) { + s"$baseUrl/" + } else { + baseUrl + } + new Retrofit.Builder() + .baseUrl(resolvedBaseUrl) + .addConverterFactory(ScalarsConverterFactory.create()) + .addConverterFactory(JacksonConverterFactory.create(OBJECT_MAPPER)) + .client(okHttpClientBuilder.build()) + .build() + .create(serviceType) + } + + private def uriStringToProxy(uriString: String): java.net.Proxy = { + val uriObject = URI.create(uriString) + new java.net.Proxy(java.net.Proxy.Type.HTTP, + new InetSocketAddress(uriObject.getHost, uriObject.getPort)) + } +} 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/kubernetes/SparkConfPropertiesParser.scala new file mode 100644 index 0000000000000..9e2b8a780df29 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.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.rest.kubernetes + +import java.io.{File, FileInputStream} +import java.util.Properties + +import com.google.common.collect.Maps +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider} +import org.apache.spark.util.Utils + +private[spark] object SparkConfPropertiesParser { + + def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = { + val sparkConf = new SparkConf(true) + if (!propertiesFile.isFile) { + throw new IllegalArgumentException(s"Server properties file given at" + + s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.") + } + val properties = new Properties + Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load) + val propertiesMap = Maps.fromProperties(properties) + val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap)) + propertiesMap.asScala.keys.foreach { key => + configReader.get(key).foreach(sparkConf.set(key, _)) + } + sparkConf + } +} 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/kubernetes/StagedResources.scala new file mode 100644 index 0000000000000..81f394800f803 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala @@ -0,0 +1,24 @@ +/* + * 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.rest.kubernetes + +import java.io.File + +case class StagedResources( + resourceId: String, + resourceSecret: String, + resourcesFile: 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/kubernetes/StagedResourcesCleaner.scala new file mode 100644 index 0000000000000..5d9db728483fa --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala @@ -0,0 +1,150 @@ +/* + * 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.rest.kubernetes + +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.util.Clock + +private[spark] trait StagedResourcesCleaner { + + def start(): Unit + + def registerResourceForCleaning( + resourceId: String, stagedResourceOwner: StagedResourcesOwner): Unit + + def markResourceAsUsed(resourceId: String): Unit +} + +private class StagedResourcesCleanerImpl( + stagedResourcesStore: StagedResourcesStore, + kubernetesClient: KubernetesClient, + cleanupExecutorService: ScheduledExecutorService, + clock: Clock, + initialAccessExpirationMs: Long) + extends StagedResourcesCleaner { + + private val CLEANUP_INTERVAL_MS = 30000 + private val RESOURCE_LOCK = new Object() + private val activeResources = mutable.Map.empty[String, MonitoredResource] + private val unusedResources = mutable.Map.empty[String, UnusedMonitoredResource] + + override def start(): Unit = { + cleanupExecutorService.scheduleAtFixedRate( + new CleanupRunnable(), + CLEANUP_INTERVAL_MS, + CLEANUP_INTERVAL_MS, + TimeUnit.MILLISECONDS) + } + + override def registerResourceForCleaning( + resourceId: String, stagedResourceOwner: StagedResourcesOwner): Unit = { + RESOURCE_LOCK.synchronized { + unusedResources(resourceId) = UnusedMonitoredResource( + clock.getTimeMillis() + initialAccessExpirationMs, + MonitoredResource(resourceId, stagedResourceOwner)) + + } + } + + override def markResourceAsUsed(resourceId: String): Unit = RESOURCE_LOCK.synchronized { + val resource = unusedResources.remove(resourceId) + resource.foreach { res => + activeResources(resourceId) = res.resource + } + } + + private class CleanupRunnable extends Runnable with Logging { + + override def run(): Unit = { + // Make a copy so we can iterate through this while modifying + val activeResourcesCopy = RESOURCE_LOCK.synchronized { + Map.apply(activeResources.toSeq: _*) + } + for ((resourceId, resource) <- activeResourcesCopy) { + val namespace = kubernetesClient.namespaces() + .withName(resource.stagedResourceOwner.ownerNamespace) + .get() + if (namespace == null) { + logInfo(s"Resource files with id $resourceId is being removed. The owner's namespace" + + s" ${resource.stagedResourceOwner.ownerNamespace} was not found.") + stagedResourcesStore.removeResources(resourceId) + RESOURCE_LOCK.synchronized { + activeResources.remove(resourceId) + } + } else { + val metadataOperation = resource.stagedResourceOwner.ownerType match { + case StagedResourcesOwnerType.Pod => + kubernetesClient.pods().inNamespace(resource.stagedResourceOwner.ownerNamespace) + case _ => + throw new SparkException(s"Unsupported resource owner type for cleanup:" + + s" ${resource.stagedResourceOwner.ownerType}") + } + if (metadataOperation + .withLabels(resource.stagedResourceOwner.ownerLabels.asJava) + .list() + .getItems + .isEmpty) { + logInfo(s"Resource files with id $resourceId is being removed. Owners of the" + + s" resource with namespace: ${resource.stagedResourceOwner.ownerNamespace}," + + s" type: ${resource.stagedResourceOwner.ownerType}, and labels:" + + s" ${resource.stagedResourceOwner.ownerLabels} was not found on the API server.") + stagedResourcesStore.removeResources(resourceId) + RESOURCE_LOCK.synchronized { + activeResources.remove(resourceId) + } + } + } + } + + // Make a copy so we can iterate through this while modifying + val unusedResourcesCopy = RESOURCE_LOCK.synchronized { + Map.apply(unusedResources.toSeq: _*) + } + + for ((resourceId, resource) <- unusedResourcesCopy) { + if (resource.expiresAt < clock.getTimeMillis()) { + RESOURCE_LOCK.synchronized { + // Check for existence again here (via foreach) because in between the time we starting + // iterating over the unused resources copy, we might have already marked the resource + // as active in-between, and likely shouldn't remove the resources in such a case. + unusedResources.remove(resourceId).foreach { _ => + logInfo(s"Resources with id $resourceId was not accessed after being added to" + + s" the staging server at least $initialAccessExpirationMs ms ago. The resource" + + s" will be deleted.") + stagedResourcesStore.removeResources(resourceId) + } + } + } + } + } + } + + private case class MonitoredResource( + resourceId: String, + stagedResourceOwner: StagedResourcesOwner) + + private case class UnusedMonitoredResource(expiresAt: Long, resource: MonitoredResource) +} + + 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/kubernetes/StagedResourcesOwner.scala new file mode 100644 index 0000000000000..4061bc36764d7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala @@ -0,0 +1,34 @@ +/* + * 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.rest.kubernetes + +import com.fasterxml.jackson.core.`type`.TypeReference +import com.fasterxml.jackson.module.scala.JsonScalaEnumeration + +object StagedResourcesOwnerType extends Enumeration { + type OwnerType = Value + // In more generic scenarios, we might want to be watching Deployments, etc. + val Pod = Value +} + +class StagedResourcesOwnerTypeReference extends TypeReference[StagedResourcesOwnerType.type] + +case class StagedResourcesOwner( + ownerNamespace: String, + ownerLabels: Map[String, String], + @JsonScalaEnumeration(classOf[StagedResourcesOwnerTypeReference]) + ownerType: StagedResourcesOwnerType.OwnerType) 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/kubernetes/StagedResourcesStore.scala new file mode 100644 index 0000000000000..0c0d428e035dc --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala @@ -0,0 +1,108 @@ +/* + * 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.rest.kubernetes + +import java.io.{File, FileOutputStream, InputStream, IOException} +import java.security.SecureRandom +import java.util.UUID + +import com.google.common.io.{BaseEncoding, ByteStreams} +import org.apache.commons.io.FileUtils +import scala.collection.concurrent.TrieMap + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + + +private[spark] trait StagedResourcesStore { + + /** + * Store the given stream on disk and return its resource ID and secret. + */ + def addResources( + podNamespace: String, + resources: InputStream): StagedResources + + /** + * Retrieve a resource bundle with the given id. Returns empty if no resources match this id. + */ + def getResources(resourceId: String): Option[StagedResources] + + def removeResources(resourceId: String): Unit +} + +private[spark] class StagedResourcesStoreImpl(dependenciesRootDir: File) + extends StagedResourcesStore with Logging { + + private val SECURE_RANDOM = new SecureRandom() + private val stagedResources = TrieMap.empty[String, StagedResources] + + override def addResources( + podNamespace: String, + resources: InputStream): StagedResources = { + val resourceId = UUID.randomUUID().toString + val secretBytes = new Array[Byte](1024) + SECURE_RANDOM.nextBytes(secretBytes) + val resourceSecret = resourceId + "-" + BaseEncoding.base64().encode(secretBytes) + + val namespaceDir = new File(dependenciesRootDir, podNamespace) + val resourcesDir = new File(namespaceDir, resourceId) + try { + if (!resourcesDir.exists()) { + if (!resourcesDir.mkdirs()) { + throw new SparkException("Failed to create dependencies directory for application" + + s" at ${resourcesDir.getAbsolutePath}") + } + } + // TODO encrypt the written data with the secret. + val resourcesFile = new File(resourcesDir, "resources.data") + Utils.tryWithResource(new FileOutputStream(resourcesFile)) { + ByteStreams.copy(resources, _) + } + val resourceBundle = StagedResources(resourceId, resourceSecret, resourcesFile) + stagedResources(resourceId) = resourceBundle + resourceBundle + } catch { + case e: Throwable => + if (!resourcesDir.delete()) { + logWarning(s"Failed to delete application directory $resourcesDir.") + } + stagedResources.remove(resourceId) + throw e + } + } + + override def getResources(resourceId: String): Option[StagedResources] = { + stagedResources.get(resourceId) + } + + override def removeResources(resourceId: String): Unit = { + stagedResources.remove(resourceId) + .map(_.resourcesFile.getParentFile) + .foreach { resourcesDirectory => + try { + FileUtils.deleteDirectory(resourcesDirectory) + } catch { + case e: IOException => + logWarning(s"Failed to delete resources directory" + + s" at ${resourcesDirectory.getAbsolutePath}", e) + } + } + } +} + 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/kubernetes/KubernetesClusterManager.scala new file mode 100644 index 0000000000000..fa0ecca3b4ee6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -0,0 +1,101 @@ +/* + * 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.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.internal.Logging +import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { + + override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + val scheduler = new KubernetesTaskSchedulerImpl(sc) + sc.taskScheduler = scheduler + scheduler + } + + override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) + : SchedulerBackend = { + val sparkConf = sc.getConf + val maybeConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP) + val maybeConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY) + + val maybeExecutorInitContainerSecretName = + sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) + val maybeExecutorInitContainerSecretMount = + sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) + val executorInitContainerSecretVolumePlugin = for { + initContainerSecretName <- maybeExecutorInitContainerSecretName + initContainerSecretMountPath <- maybeExecutorInitContainerSecretMount + } yield { + new InitContainerResourceStagingServerSecretPluginImpl( + initContainerSecretName, + initContainerSecretMountPath) + } + // Only set up the bootstrap if they've provided both the config map key and the config map + // name. Note that we generally expect both to have been set from spark-submit V2, but for + // testing developers may simply run the driver JVM locally, but the config map won't be set + // then. + val bootStrap = for { + configMap <- maybeConfigMap + configMapKey <- maybeConfigMapKey + } yield { + new SparkPodInitContainerBootstrapImpl( + sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE), + sparkConf.get(DOCKER_IMAGE_PULL_POLICY), + sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION), + sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION), + sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), + configMap, + configMapKey) + } + if (maybeConfigMap.isEmpty) { + logWarning("The executor's init-container config map was not specified. Executors will" + + " therefore not attempt to fetch remote or submitted dependencies.") + } + if (maybeConfigMapKey.isEmpty) { + 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)), + APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX, + sparkConf, + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), + Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + new KubernetesClusterSchedulerBackend( + sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], + sc, + bootStrap, + executorInitContainerSecretVolumePlugin, + kubernetesClient) + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} 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 new file mode 100644 index 0000000000000..c993bff8df962 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,802 @@ +/* + * 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 scala.collection.{concurrent, mutable} +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} + +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 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 +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], + 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 executorMemoryMb = 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 memoryOverheadMb = conf + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + private val executorMemoryWithOverhead = executorMemoryMb + memoryOverheadMb + + 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(), + sc.env.securityManager.isSaslEncryptionEnabled()) + } + + 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"${executorMemoryMb}M") + .build() + val executorMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryWithOverhead}M") + .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 requiredEnv = 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/*")) + .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() + ) + 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(requiredEnv.asJava) + .addToEnv(executorExtraClasspathEnv.toSeq: _*) + .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 (executorPodWithInitContainer, initBootstrappedExecutorContainer) = + executorInitContainerBootstrap.map { bootstrap => + val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + withMaybeShuffleConfigPod, + new ContainerBuilder().build(), + withMaybeShuffleConfigExecutorContainer)) + + 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((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) + + 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(executorId) => + 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()) + 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/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala new file mode 100644 index 0000000000000..a5e126480b83d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala @@ -0,0 +1,27 @@ +/* + * 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 org.apache.spark.SparkContext +import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} + +private[spark] class KubernetesTaskSchedulerImpl(sc: SparkContext) extends TaskSchedulerImpl(sc) { + + override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = { + new KubernetesTaskSetManager(this, taskSet, maxTaskFailures) + } +} 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/kubernetes/KubernetesTaskSetManager.scala new file mode 100644 index 0000000000000..17710fada2876 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala @@ -0,0 +1,94 @@ +/* + * 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.net.InetAddress + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} + +private[spark] class KubernetesTaskSetManager( + sched: TaskSchedulerImpl, + taskSet: TaskSet, + maxTaskFailures: Int, + inetAddressUtil: InetAddressUtil = new InetAddressUtil) + extends TaskSetManager(sched, taskSet, maxTaskFailures) { + + private val conf = sched.sc.conf + + /** + * Overrides the lookup to use not only the executor pod IP, but also the cluster node + * name and host IP address that the pod is running on. The base class may have populated + * the lookup target map with HDFS datanode locations if this task set reads HDFS data. + * Those datanode locations are based on cluster node names or host IP addresses. Using + * only executor pod IPs may not match them. + */ + override def getPendingTasksForHost(executorIP: String): ArrayBuffer[Int] = { + val pendingTasksExecutorIP = super.getPendingTasksForHost(executorIP) + if (pendingTasksExecutorIP.nonEmpty) { + pendingTasksExecutorIP + } else { + val backend = sched.backend.asInstanceOf[KubernetesClusterSchedulerBackend] + val pod = backend.getExecutorPodByIP(executorIP) + if (pod.nonEmpty) { + val clusterNodeName = pod.get.getSpec.getNodeName + val pendingTasksClusterNodeName = super.getPendingTasksForHost(clusterNodeName) + if (pendingTasksClusterNodeName.nonEmpty) { + logDebug(s"Got preferred task list $pendingTasksClusterNodeName for executor host " + + s"$executorIP using cluster node name $clusterNodeName") + pendingTasksClusterNodeName + } else { + val clusterNodeIP = pod.get.getStatus.getHostIP + val pendingTasksClusterNodeIP = super.getPendingTasksForHost(clusterNodeIP) + if (pendingTasksClusterNodeIP.nonEmpty) { + logDebug(s"Got preferred task list $pendingTasksClusterNodeIP for executor host " + + s"$executorIP using cluster node IP $clusterNodeIP") + pendingTasksClusterNodeIP + } else { + if (conf.get(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED)) { + val clusterNodeFullName = inetAddressUtil.getFullHostName(clusterNodeIP) + val pendingTasksClusterNodeFullName = super.getPendingTasksForHost( + clusterNodeFullName) + if (pendingTasksClusterNodeFullName.nonEmpty) { + logDebug(s"Got preferred task list $pendingTasksClusterNodeFullName " + + s"for executor host $executorIP using cluster node full name " + + s"$clusterNodeFullName") + } + pendingTasksClusterNodeFullName + } else { + pendingTasksExecutorIP // Empty + } + } + } + } else { + pendingTasksExecutorIP // Empty + } + } + } +} + +// To support mocks in unit tests. +private[kubernetes] class InetAddressUtil { + + // NOTE: This does issue a network call to DNS. Caching is done internally by the InetAddress + // class for both hits and misses. + def getFullHostName(ipAddress: String): String = { + InetAddress.getByName(ipAddress).getCanonicalHostName + } +} 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 new file mode 100644 index 0000000000000..15e02664589eb --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala @@ -0,0 +1,93 @@ +/* + * 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/resources/log4j.properties b/resource-managers/kubernetes/core/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..ad95fadb7c0c0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from a few verbose libraries. +log4j.logger.com.sun.jersey=WARN +log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.mortbay=WARN +log4j.logger.org.spark_project.jetty=WARN 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/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala new file mode 100644 index 0000000000000..597bcdb416fc0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -0,0 +1,59 @@ +/* + * 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 + +import io.fabric8.kubernetes.api.model._ +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.SparkFunSuite + +class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ + private val INIT_CONTAINER_SECRET_NAME = "init-secret" + private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret" + + private val initContainerSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl( + INIT_CONTAINER_SECRET_NAME, + INIT_CONTAINER_SECRET_MOUNT) + + test("Volume Mount into InitContainer") { + val returnedCont = initContainerSecretPlugin.mountResourceStagingServerSecretIntoInitContainer( + new ContainerBuilder().withName("init-container").build()) + assert(returnedCont.getName === "init-container") + assert(returnedCont.getVolumeMounts.asScala.map( + vm => (vm.getName, vm.getMountPath)) === + List((INIT_CONTAINER_SECRET_VOLUME_NAME, INIT_CONTAINER_SECRET_MOUNT))) + } + + test("Add Volume with Secret to Pod") { + val returnedPod = initContainerSecretPlugin.addResourceStagingServerSecretVolumeToPod( + basePod().build) + assert(returnedPod.getMetadata.getName === "spark-pod") + val volume = returnedPod.getSpec.getVolumes.asScala.head + assert(volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME) + assert(volume.getSecret.getSecretName === INIT_CONTAINER_SECRET_NAME) + } + private def basePod(): PodBuilder = { + new PodBuilder() + .withNewMetadata() + .withName("spark-pod") + .endMetadata() + .withNewSpec() + .endSpec() + } +} 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/kubernetes/SSLUtils.scala new file mode 100644 index 0000000000000..8de0f56f007dc --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala @@ -0,0 +1,117 @@ +/* + * 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 + +import java.io.{File, FileOutputStream, OutputStreamWriter} +import java.math.BigInteger +import java.nio.file.Files +import java.security.{KeyPair, KeyPairGenerator, KeyStore, SecureRandom} +import java.security.cert.X509Certificate +import java.util.{Calendar, Random} +import javax.security.auth.x500.X500Principal + +import com.google.common.base.Charsets +import org.bouncycastle.asn1.x509.{Extension, GeneralName, GeneralNames} +import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3CertificateBuilder} +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.util.Utils + +private[spark] object SSLUtils { + + def generateKeyStoreTrustStorePair( + ipAddress: String, + keyStorePassword: String, + keyPassword: String, + trustStorePassword: String): KeyStoreAndTrustStore = { + val keyPairGenerator = KeyPairGenerator.getInstance("RSA") + keyPairGenerator.initialize(512) + val keyPair = keyPairGenerator.generateKeyPair() + val certificate = generateCertificate(ipAddress, keyPair) + val keyStore = KeyStore.getInstance("JKS") + keyStore.load(null, null) + keyStore.setKeyEntry("key", keyPair.getPrivate, + keyPassword.toCharArray, Array(certificate)) + val tempDir = Files.createTempDirectory("temp-ssl-stores").toFile + tempDir.deleteOnExit() + val keyStoreFile = new File(tempDir, "keyStore.jks") + Utils.tryWithResource(new FileOutputStream(keyStoreFile)) { + keyStore.store(_, keyStorePassword.toCharArray) + } + val trustStore = KeyStore.getInstance("JKS") + trustStore.load(null, null) + trustStore.setCertificateEntry("key", certificate) + val trustStoreFile = new File(tempDir, "trustStore.jks") + Utils.tryWithResource(new FileOutputStream(trustStoreFile)) { + trustStore.store(_, trustStorePassword.toCharArray) + } + KeyStoreAndTrustStore(keyStoreFile, trustStoreFile) + } + + def generateKeyCertPemPair(ipAddress: String): KeyAndCertPem = { + val keyPairGenerator = KeyPairGenerator.getInstance("RSA") + keyPairGenerator.initialize(512) + val keyPair = keyPairGenerator.generateKeyPair() + val certificate = generateCertificate(ipAddress, keyPair) + val tempDir = Files.createTempDirectory("temp-ssl-pems").toFile + tempDir.deleteOnExit() + val keyPemFile = new File(tempDir, "key.pem") + val certPemFile = new File(tempDir, "cert.pem") + Utils.tryWithResource(new FileOutputStream(keyPemFile)) { keyPemStream => + Utils.tryWithResource( + new OutputStreamWriter(keyPemStream, Charsets.UTF_8)) { streamWriter => + Utils.tryWithResource( + new JcaPEMWriter(streamWriter)) { pemWriter => + pemWriter.writeObject(keyPair.getPrivate) + } + } + } + Utils.tryWithResource(new FileOutputStream(certPemFile)) { keyPemStream => + Utils.tryWithResource( + new OutputStreamWriter(keyPemStream, Charsets.UTF_8)) { streamWriter => + Utils.tryWithResource( + new JcaPEMWriter(streamWriter)) { pemWriter => + pemWriter.writeObject(certificate) + } + } + } + KeyAndCertPem(keyPemFile, certPemFile) + } + + private def generateCertificate(ipAddress: String, keyPair: KeyPair): X509Certificate = { + val selfPrincipal = new X500Principal(s"cn=$ipAddress") + val currentDate = Calendar.getInstance + val validForOneHundredYears = Calendar.getInstance + validForOneHundredYears.add(Calendar.YEAR, 100) + val certificateBuilder = new JcaX509v3CertificateBuilder( + selfPrincipal, + new BigInteger(4096, new Random()), + currentDate.getTime, + validForOneHundredYears.getTime, + selfPrincipal, + keyPair.getPublic) + certificateBuilder.addExtension(Extension.subjectAlternativeName, false, + new GeneralNames(new GeneralName(GeneralName.iPAddress, ipAddress))) + val signer = new JcaContentSignerBuilder("SHA1WithRSA") + .setSecureRandom(new SecureRandom()) + .build(keyPair.getPrivate) + val bcCertificate = certificateBuilder.build(signer) + new JcaX509CertificateConverter().getCertificate(bcCertificate) + } +} 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/kubernetes/SparkPodInitContainerBootstrapSuite.scala new file mode 100644 index 0000000000000..d5f25983f5080 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala @@ -0,0 +1,104 @@ +/* + * 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 + +import io.fabric8.kubernetes.api.model._ +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.SparkFunSuite + +class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { + private val INIT_CONTAINER_IMAGE = "spark-init:latest" + private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" + private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" + private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val DOWNLOAD_TIMEOUT_MINUTES = 5 + private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" + private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" + private val MAIN_CONTAINER_NAME = "spark-main" + + private val sparkPodInit = new SparkPodInitContainerBootstrapImpl( + INIT_CONTAINER_IMAGE, + DOCKER_IMAGE_PULL_POLICY, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOWNLOAD_TIMEOUT_MINUTES, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY) + private val expectedSharedVolumeMap = Map( + JARS_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME, + FILES_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + + test("InitContainer: Volume mounts, args, and builder specs") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val initContainer: Container = returnedPodWithCont.initContainer + val volumes = initContainer.getVolumeMounts.asScala + assert(volumes.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap + ++ Map("/etc/spark-init" -> "spark-init-properties")) + assert(initContainer.getName === "spark-init") + assert(initContainer.getImage === INIT_CONTAINER_IMAGE) + assert(initContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY) + assert(initContainer.getArgs.asScala.head === INIT_CONTAINER_PROPERTIES_FILE_PATH) + } + test("Main: Volume mounts and env") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val mainContainer: Container = returnedPodWithCont.mainContainer + assert(mainContainer.getName === MAIN_CONTAINER_NAME) + val volumeMounts = mainContainer.getVolumeMounts.asScala + assert(volumeMounts.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap) + assert(mainContainer.getEnv.asScala.map(e => (e.getName, e.getValue)).toMap === + Map(ENV_MOUNTED_FILES_DIR -> FILES_DOWNLOAD_PATH)) + } + test("Pod: Volume Mounts") { + val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + pod = basePod().build(), + initContainer = new Container(), + mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build())) + val returnedPod = returnedPodWithCont.pod + assert(returnedPod.getMetadata.getName === "spark-pod") + val volumes = returnedPod.getSpec.getVolumes.asScala.toList + assert(volumes.head.getName === INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + assert(volumes.head.getConfigMap.getName === INIT_CONTAINER_CONFIG_MAP_NAME) + assert(volumes.head.getConfigMap.getItems.asScala.map( + i => (i.getKey, i.getPath)) === + List((INIT_CONTAINER_CONFIG_MAP_KEY, INIT_CONTAINER_PROPERTIES_FILE_NAME))) + assert(volumes(1).getName === INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + assert(volumes(1).getEmptyDir === new EmptyDirVolumeSource()) + assert(volumes(2).getName === INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + assert(volumes(2).getEmptyDir === new EmptyDirVolumeSource()) + } + + private def basePod(): PodBuilder = { + new PodBuilder() + .withNewMetadata() + .withName("spark-pod") + .endMetadata() + .withNewSpec() + .endSpec() + } +} 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/kubernetes/submit/ClientSuite.scala new file mode 100644 index 0000000000000..965ee75c248b8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala @@ -0,0 +1,226 @@ +/* + * 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 + +import com.google.common.collect.Iterables +import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch} +import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource} +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Mockito.{doReturn, verify, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +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} + +class ClientSuite extends SparkFunSuite with BeforeAndAfter { + + private val DRIVER_POD_UID = "pod-id" + private val DRIVER_POD_API_VERSION = "v1" + private val DRIVER_POD_KIND = "pod" + + private type ResourceList = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, Boolean] + private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: Pods = _ + + @Mock + private var namedPods: PodResource[Pod, DoneablePod] = _ + + @Mock + private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _ + + @Mock + private var resourceList: ResourceList = _ + + private val submissionSteps = Seq(FirstTestConfigurationStep, SecondTestConfigurationStep) + private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ + private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + + createdPodArgumentCaptor = ArgumentCaptor.forClass(classOf[Pod]) + createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + when(podOperations.create(createdPodArgumentCaptor.capture())).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock): Pod = { + new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) + .editMetadata() + .withUid(DRIVER_POD_UID) + .endMetadata() + .withApiVersion(DRIVER_POD_API_VERSION) + .withKind(DRIVER_POD_KIND) + .build() + } + }) + when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods) + when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch]) + doReturn(resourceList) + .when(kubernetesClient) + .resourceList(createdResourcesArgumentCaptor.capture()) + } + + test("The client should configure the pod with the submission steps.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + assert(createdPod.getMetadata.getName === FirstTestConfigurationStep.podName) + assert(createdPod.getMetadata.getLabels.asScala === + Map(FirstTestConfigurationStep.labelKey -> FirstTestConfigurationStep.labelValue)) + assert(createdPod.getMetadata.getAnnotations.asScala === + Map(SecondTestConfigurationStep.annotationKey -> + SecondTestConfigurationStep.annotationValue)) + assert(createdPod.getSpec.getContainers.size() === 1) + assert(createdPod.getSpec.getContainers.get(0).getName === + SecondTestConfigurationStep.containerName) + } + + test("The client should create the secondary Kubernetes resources.") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues + assert(otherCreatedResources.size === 1) + val createdResource = Iterables.getOnlyElement(otherCreatedResources).asInstanceOf[Secret] + assert(createdResource.getMetadata.getName === FirstTestConfigurationStep.secretName) + assert(createdResource.getData.asScala === + Map(FirstTestConfigurationStep.secretKey -> FirstTestConfigurationStep.secretData)) + val ownerReference = Iterables.getOnlyElement(createdResource.getMetadata.getOwnerReferences) + assert(ownerReference.getName === createdPod.getMetadata.getName) + assert(ownerReference.getKind === DRIVER_POD_KIND) + assert(ownerReference.getUid === DRIVER_POD_UID) + assert(ownerReference.getApiVersion === DRIVER_POD_API_VERSION) + } + + test("The client should attach the driver container with the appropriate JVM options.") { + val sparkConf = new SparkConf(false) + .set("spark.logConf", "true") + .set( + org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, + "-XX:+|-HeapDumpOnOutOfMemoryError") + val submissionClient = new Client( + submissionSteps, + sparkConf, + kubernetesClient, + false, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + val createdPod = createdPodArgumentCaptor.getValue + val driverContainer = Iterables.getOnlyElement(createdPod.getSpec.getContainers) + assert(driverContainer.getName === SecondTestConfigurationStep.containerName) + val driverJvmOptsEnv = Iterables.getOnlyElement(driverContainer.getEnv) + assert(driverJvmOptsEnv.getName === ENV_DRIVER_JAVA_OPTS) + val driverJvmOpts = driverJvmOptsEnv.getValue.split(" ").toSet + assert(driverJvmOpts.contains("-Dspark.logConf=true")) + assert(driverJvmOpts.contains( + s"-D${SecondTestConfigurationStep.sparkConfKey}=" + + SecondTestConfigurationStep.sparkConfValue)) + assert(driverJvmOpts.contains( + "-XX:+|-HeapDumpOnOutOfMemoryError")) + } + + test("Waiting for app completion should stall on the watcher") { + val submissionClient = new Client( + submissionSteps, + new SparkConf(false), + kubernetesClient, + true, + "spark", + loggingPodStatusWatcher) + submissionClient.run() + verify(loggingPodStatusWatcher).awaitCompletion() + } + +} + +private object FirstTestConfigurationStep extends DriverConfigurationStep { + + val podName = "test-pod" + val secretName = "test-secret" + val labelKey = "first-submit" + val labelValue = "true" + val secretKey = "secretKey" + val secretData = "secretData" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .withName(podName) + .addToLabels(labelKey, labelValue) + .endMetadata() + .build() + val additionalResource = new SecretBuilder() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .addToData(secretKey, secretData) + .build() + driverSpec.copy( + driverPod = modifiedPod, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(additionalResource)) + } +} + +private object SecondTestConfigurationStep extends DriverConfigurationStep { + + val annotationKey = "second-submit" + val annotationValue = "submitted" + val sparkConfKey = "spark.custom-conf" + val sparkConfValue = "custom-conf-value" + val containerName = "driverContainer" + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val modifiedPod = new PodBuilder(driverSpec.driverPod) + .editMetadata() + .addToAnnotations(annotationKey, annotationValue) + .endMetadata() + .build() + val resolvedSparkConf = driverSpec.driverSparkConf.clone().set(sparkConfKey, sparkConfValue) + val modifiedContainer = new ContainerBuilder(driverSpec.driverContainer) + .withName(containerName) + .build() + driverSpec.copy( + driverPod = modifiedPod, + driverSparkConf = resolvedSparkConf, + driverContainer = modifiedContainer) + } +} 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/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..e4f221ad99cc5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -0,0 +1,95 @@ +/* + * 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 + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep} + +private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { + + private val NAMESPACE = "default" + private val APP_ID = "spark-app-id" + private val LAUNCH_TIME = 975256L + private val APP_NAME = "spark" + 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") + + test("Base submission steps without an init-container or python files.") { + val sparkConf = new SparkConf(false) + .set("spark.jars", "local:///var/apps/jars/jar1.jar") + 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) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 3) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + } + + test("Submission steps with an init-container.") { + val sparkConf = new SparkConf(false) + .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar") + 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) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 4) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + assert(steps(3).isInstanceOf[InitContainerBootstrapStep]) + } + + test("Submission steps with python files.") { + val sparkConf = new SparkConf(false) + val mainAppResource = PythonMainAppResource("local:///var/apps/python/main.py") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + val steps = orchestrator.getAllConfigurationSteps() + assert(steps.size === 4) + assert(steps(0).isInstanceOf[BaseDriverConfigurationStep]) + assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep]) + assert(steps(2).isInstanceOf[DependencyResolutionStep]) + assert(steps(3).isInstanceOf[PythonStep]) + } +} 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/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala new file mode 100644 index 0000000000000..0de1955884c8e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala @@ -0,0 +1,49 @@ +/* + * 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 + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient + +private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite { + + private val SPARK_CONF = new SparkConf() + .set(KUBERNETES_SHUFFLE_USE_SERVICE_ACCOUNT_CREDENTIALS, false) + + test("Run kubernetes shuffle service.") { + val shuffleService = new KubernetesExternalShuffleService( + SPARK_CONF, + new SecurityManager(SPARK_CONF)) + + val shuffleClient = new KubernetesExternalShuffleClient( + SparkTransportConf.fromSparkConf(SPARK_CONF, "shuffle"), + new SecurityManager(SPARK_CONF), + false, + false) + + shuffleService.start() + shuffleClient.init("newapp") + + // verifies that we can connect to the shuffle service and send + // it a message. + shuffleClient.registerDriverWithShuffleService("localhost", 7337) + shuffleService.stop() + } +} 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/kubernetes/submit/SSLFilePairs.scala new file mode 100644 index 0000000000000..5240128743b76 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala @@ -0,0 +1,23 @@ +/* + * 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 + +import java.io.File + +case class KeyAndCertPem(keyPem: File, certPem: File) + +case class KeyStoreAndTrustStore(keyStore: File, trustStore: File) 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/kubernetes/submit/SubmittedDependencyUploaderSuite.scala new file mode 100644 index 0000000000000..96fa92c254297 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala @@ -0,0 +1,154 @@ +/* + * 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 + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} +import java.util.UUID + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import okhttp3.RequestBody +import okio.Okio +import org.mockito.{ArgumentCaptor, Mockito} +import org.scalatest.BeforeAndAfter +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.util.Utils + +private[spark] class SubmittedDependencyUploaderSuite extends SparkFunSuite with BeforeAndAfter { + import SubmittedDependencyUploaderSuite.createTempFile + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + private val BASE_64 = BaseEncoding.base64() + private val APP_ID = "app-id" + private val LABELS = Map("label1" -> "label1value", "label2" -> "label2value") + private val NAMESPACE = "namespace" + private val STAGING_SERVER_URI = "http://localhost:8000" + private val LOCAL_JARS = Seq(createTempFile("jar"), createTempFile("jar")) + private val JARS = Seq("hdfs://localhost:9000/jars/jar1.jar", + s"file://${LOCAL_JARS.head}", + LOCAL_JARS(1)) + private val LOCAL_FILES = Seq(createTempFile("txt")) + private val FILES = Seq("hdfs://localhost:9000/files/file1.txt", + LOCAL_FILES.head) + private val TRUSTSTORE_FILE = new File(createTempFile(".jks")) + private val TRUSTSTORE_PASSWORD = "trustStorePassword" + private val TRUSTSTORE_TYPE = "jks" + private val STAGING_SERVER_SSL_OPTIONS = SSLOptions( + enabled = true, + trustStore = Some(TRUSTSTORE_FILE), + trustStorePassword = Some(TRUSTSTORE_PASSWORD), + trustStoreType = Some(TRUSTSTORE_TYPE)) + private val CLIENT_KEY_FILE = createTempFile("pem") + private val CLIENT_CERT_FILE = createTempFile("pem") + private val OAUTH_TOKEN = "token" + private var retrofitClientFactory: RetrofitClientFactory = _ + private var retrofitClient: ResourceStagingServiceRetrofit = _ + private var resourcesOwnerCaptor: ArgumentCaptor[RequestBody] = _ + private var resourcesDataCaptor: ArgumentCaptor[RequestBody] = _ + + private var dependencyUploaderUnderTest: SubmittedDependencyUploader = _ + + before { + resourcesOwnerCaptor = ArgumentCaptor.forClass(classOf[RequestBody]) + resourcesDataCaptor = ArgumentCaptor.forClass(classOf[RequestBody]) + retrofitClientFactory = mock[RetrofitClientFactory] + retrofitClient = mock[ResourceStagingServiceRetrofit] + Mockito.when( + retrofitClientFactory.createRetrofitClient( + STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) + .thenReturn(retrofitClient) + val responseCall = mock[Call[SubmittedResourceIdAndSecret]] + Mockito.when(responseCall.execute()).thenReturn( + Response.success(SubmittedResourceIdAndSecret("resourceId", "resourceSecret"))) + Mockito.when(retrofitClient.uploadResources( + resourcesDataCaptor.capture(), resourcesOwnerCaptor.capture())) + .thenReturn(responseCall) + dependencyUploaderUnderTest = new SubmittedDependencyUploaderImpl( + LABELS, + NAMESPACE, + STAGING_SERVER_URI, + JARS, + FILES, + STAGING_SERVER_SSL_OPTIONS, + retrofitClientFactory) + } + + test("Uploading jars should contact the staging server with the appropriate parameters") { + dependencyUploaderUnderTest.uploadJars() + testUploadSendsCorrectFiles(LOCAL_JARS) + } + + test("Uploading files should contact the staging server with the appropriate parameters") { + dependencyUploaderUnderTest.uploadFiles() + testUploadSendsCorrectFiles(LOCAL_FILES) + } + + private def testUploadSendsCorrectFiles(expectedFiles: Seq[String]) = { + val resourceOwnerString = new String( + requestBodyBytes(resourcesOwnerCaptor.getValue), Charsets.UTF_8) + val resourceOwner = OBJECT_MAPPER.readValue(resourceOwnerString, classOf[StagedResourcesOwner]) + assert(resourceOwner.ownerLabels === LABELS) + assert(resourceOwner.ownerNamespace === NAMESPACE) + val unpackedFilesDir = Utils.createTempDir(namePrefix = "test-unpacked-files") + val compressedBytesInput = new ByteArrayInputStream( + requestBodyBytes(resourcesDataCaptor.getValue())) + CompressionUtils.unpackTarStreamToDirectory(compressedBytesInput, unpackedFilesDir) + val writtenFiles = unpackedFilesDir.listFiles + assert(writtenFiles.size === expectedFiles.size) + + expectedFiles.map(new File(_)).foreach { expectedFile => + val maybeWrittenFile = writtenFiles.find(_.getName == expectedFile.getName) + assert(maybeWrittenFile.isDefined) + maybeWrittenFile.foreach { writtenFile => + val writtenFileBytes = Files.toByteArray(writtenFile) + val expectedFileBytes = Files.toByteArray(expectedFile) + assert(expectedFileBytes.toSeq === writtenFileBytes.toSeq) + } + } + } + + private def requestBodyBytes(requestBody: RequestBody): Array[Byte] = { + Utils.tryWithResource(new ByteArrayOutputStream()) { outputStream => + Utils.tryWithResource(Okio.sink(outputStream)) { sink => + Utils.tryWithResource(Okio.buffer(sink)) { bufferedSink => + try { + requestBody.writeTo(bufferedSink) + } finally { + bufferedSink.flush() + } + } + } + outputStream.toByteArray + } + } +} + +private object SubmittedDependencyUploaderSuite { + def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} 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/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala new file mode 100644 index 0000000000000..c7d80a16a1532 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -0,0 +1,106 @@ +/* + * 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 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._ + +private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { + + private val APP_ID = "spark-app-id" + private val RESOURCE_NAME_PREFIX = "spark" + private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") + private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent" + private val APP_NAME = "spark-test" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + 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" + + test("Set all possible configurations from the user.") { + val sparkConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") + .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, "/opt/spark/spark-exmaples.jar") + .set("spark.driver.cores", "2") + .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") + .set(org.apache.spark.internal.config.DRIVER_MEMORY, 256L) + .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") + val submissionStep = new BaseDriverConfigurationStep( + APP_ID, + RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + DOCKER_IMAGE_PULL_POLICY, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + sparkConf) + val basePod = new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = basePod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + + val preparedDriverSpec = submissionStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverContainer.getName === DRIVER_CONTAINER_NAME) + assert(preparedDriverSpec.driverContainer.getImage === "spark-driver:latest") + assert(preparedDriverSpec.driverContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY) + val envs = preparedDriverSpec.driverContainer + .getEnv + .asScala + .map(env => (env.getName, env.getValue)) + .toMap + assert(envs.size === 4) + assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-exmaples.jar") + assert(envs(ENV_DRIVER_MEMORY) === "456m") + assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS) + assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2") + val resourceRequirements = preparedDriverSpec.driverContainer.getResources + val requests = resourceRequirements.getRequests.asScala + assert(requests("cpu").getAmount === "2") + assert(requests("memory").getAmount === "256M") + val limits = resourceRequirements.getLimits.asScala + assert(limits("memory").getAmount === "456M") + assert(limits("cpu").getAmount === "4") + val driverPodMetadata = preparedDriverSpec.driverPod.getMetadata + assert(driverPodMetadata.getName === "spark-driver-pod") + 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") + val resolvedSparkConf = preparedDriverSpec.driverSparkConf.getAll.toMap + val expectedSparkConf = Map( + KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", + "spark.app.id" -> APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX) + assert(resolvedSparkConf === expectedSparkConf) + + } +} 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/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala new file mode 100644 index 0000000000000..3f7ec61074b0c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala @@ -0,0 +1,79 @@ +/* + * 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.io.File + +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.constants._ + +private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { + + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/apps/jars/jar1.jar", + "file:///home/user/apps/jars/jar2.jar", + "local:///var/apps/jars/jar3.jar") + + private val SPARK_FILES = Seq( + "file:///home/user/apps/files/file1.txt", + "hdfs://localhost:9000/apps/files/file2.txt", + "local:///var/apps/files/file3.txt") + + private val JARS_DOWNLOAD_PATH = "/mnt/spark-data/jars" + private val FILES_DOWNLOAD_PATH = "/mnt/spark-data/files" + + test("Added dependencies should be resolved in Spark configuration and environment") { + val dependencyResolutionStep = new DependencyResolutionStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + val driverPod = new PodBuilder().build() + val baseDriverSpec = KubernetesDriverSpec( + driverPod = driverPod, + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val preparedDriverSpec = dependencyResolutionStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverPod === driverPod) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + val resolvedSparkJars = preparedDriverSpec.driverSparkConf.get("spark.jars").split(",").toSet + val expectedResolvedSparkJars = Set( + "hdfs://localhost:9000/apps/jars/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedSparkJars === expectedResolvedSparkJars) + val resolvedSparkFiles = preparedDriverSpec.driverSparkConf.get("spark.files").split(",").toSet + val expectedResolvedSparkFiles = Set( + s"$FILES_DOWNLOAD_PATH/file1.txt", + s"hdfs://localhost:9000/apps/files/file2.txt", + s"/var/apps/files/file3.txt") + assert(resolvedSparkFiles === expectedResolvedSparkFiles) + val driverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(driverEnv.size === 1) + assert(driverEnv.head.getName === ENV_MOUNTED_CLASSPATH) + val resolvedDriverClasspath = driverEnv.head.getValue.split(File.pathSeparator).toSet + val expectedResolvedDriverClasspath = Set( + s"$JARS_DOWNLOAD_PATH/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/var/apps/jars/jar3.jar") + assert(resolvedDriverClasspath === expectedResolvedDriverClasspath) + } +} 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/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala new file mode 100644 index 0000000000000..3d5664713a2b8 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala @@ -0,0 +1,152 @@ +/* + * 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.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} +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.util.Utils + +private[spark] class DriverKubernetesCredentialsStepSuite + extends SparkFunSuite with BeforeAndAfter { + + private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" + private var credentialsTempDirectory: File = _ + private val BASE_DRIVER_SPEC = new KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + + before { + credentialsTempDirectory = Utils.createTempDir() + } + + after { + credentialsTempDirectory.delete() + } + + test("Don't set any credentials") { + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + new SparkConf(false), KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.isEmpty) + } + + test("Only set credentials that are manually mounted.") { + val submissionSparkConf = new SparkConf(false) + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX", + "/mnt/secrets/my-token.txt") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + "/mnt/secrets/my-key.pem") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-cert.pem") + .set( + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + "/mnt/secrets/my-ca.pem") + + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC) + assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod) + assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer) + assert(preparedDriverSpec.otherKubernetesResources.isEmpty) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === submissionSparkConf.getAll.toMap) + } + + test("Mount credentials from the submission client as a secret.") { + val caCertFile = writeCredentials("ca.pem", "ca-cert") + val clientKeyFile = writeCredentials("key.pem", "key") + val clientCertFile = writeCredentials("cert.pem", "cert") + val submissionSparkConf = new SparkConf(false) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX", + "token") + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + clientKeyFile.getAbsolutePath) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + clientCertFile.getAbsolutePath) + .set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + caCertFile.getAbsolutePath) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( + submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX) + val preparedDriverSpec = kubernetesCredentialsStep.configureDriver( + BASE_DRIVER_SPEC.copy(driverSparkConf = submissionSparkConf)) + val expectedSparkConf = Map( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX" -> "", + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_KEY_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CLIENT_CERT_PATH, + s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + DRIVER_CREDENTIALS_CA_CERT_PATH, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" -> + clientKeyFile.getAbsolutePath, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" -> + clientCertFile.getAbsolutePath, + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" -> + caCertFile.getAbsolutePath) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + assert(preparedDriverSpec.otherKubernetesResources.size === 1) + val credentialsSecret = preparedDriverSpec.otherKubernetesResources.head.asInstanceOf[Secret] + assert(credentialsSecret.getMetadata.getName === + s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + val decodedSecretData = credentialsSecret.getData.asScala.map { data => + (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) + } + val expectedSecretData = Map( + DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> "ca-cert", + DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> "token", + DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> "key", + DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> "cert") + assert(decodedSecretData === expectedSecretData) + val driverPodVolumes = preparedDriverSpec.driverPod.getSpec.getVolumes.asScala + assert(driverPodVolumes.size === 1) + assert(driverPodVolumes.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverPodVolumes.head.getSecret != null) + assert(driverPodVolumes.head.getSecret.getSecretName === credentialsSecret.getMetadata.getName) + val driverContainerVolumeMount = preparedDriverSpec.driverContainer.getVolumeMounts.asScala + assert(driverContainerVolumeMount.size === 1) + assert(driverContainerVolumeMount.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME) + assert(driverContainerVolumeMount.head.getMountPath === DRIVER_CREDENTIALS_SECRETS_BASE_DIR) + } + + private def writeCredentials(credentialsFileName: String, credentialsContents: String): File = { + val credentialsFile = new File(credentialsTempDirectory, credentialsFileName) + Files.write(credentialsContents, credentialsFile, Charsets.UTF_8) + credentialsFile + } +} 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/kubernetes/submit/submitsteps/PythonStepSuite.scala new file mode 100644 index 0000000000000..ce0dcee6acc46 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala @@ -0,0 +1,76 @@ +/* + * 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 io.fabric8.kubernetes.api.model._ +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class PythonStepSuite extends SparkFunSuite with BeforeAndAfter { + private val FILE_DOWNLOAD_PATH = "/var/data/spark-files" + private val PYSPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.py", + "file:///app/files/file2.py", + "local:///app/files/file3.py", + "http://app/files/file4.py") + private val RESOLVED_PYSPARK_FILES = Seq( + FILE_DOWNLOAD_PATH + "/file1.py", + FILE_DOWNLOAD_PATH + "/file2.py", + "/app/files/file3.py", + FILE_DOWNLOAD_PATH + "/file4.py").mkString(",") + private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py" + private val RESOLVED_PYSPARK_PRIMARY_FILE = FILE_DOWNLOAD_PATH + "/file5.py" + + test("testing PySpark with --py-files both local and remote files") { + val pyStep = new PythonStep( + PYSPARK_PRIMARY_FILE, + PYSPARK_FILES, + FILE_DOWNLOAD_PATH) + val returnedDriverContainer = pyStep.configureDriver( + KubernetesDriverSpec( + new Pod(), + new Container(), + Seq.empty[HasMetadata], + new SparkConf)) + assert(returnedDriverContainer.driverContainer.getEnv + .asScala.map(env => (env.getName, env.getValue)).toMap === + Map( + "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE, + "PYSPARK_FILES" -> RESOLVED_PYSPARK_FILES)) + } + + test("testing PySpark with empty --py-files ") { + val pyStep = new PythonStep( + PYSPARK_PRIMARY_FILE, + Seq.empty[String], + FILE_DOWNLOAD_PATH) + val returnedDriverContainer = pyStep.configureDriver( + KubernetesDriverSpec( + new Pod(), + new Container(), + Seq.empty[HasMetadata], + new SparkConf)) + assert(returnedDriverContainer.driverContainer.getEnv + .asScala.map(env => (env.getName, env.getValue)).toMap === + Map( + "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE, + "PYSPARK_FILES" -> "null")) + } + +} 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/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala new file mode 100644 index 0000000000000..b11b487111496 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala @@ -0,0 +1,159 @@ +/* + * 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.io.StringReader +import java.util.Properties + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.google.common.collect.Maps +import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} +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.util.Utils + +private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) + private val CONFIG_MAP_NAME = "spark-init-config-map" + private val CONFIG_MAP_KEY = "spark-init-config-map-key" + + test("The init container bootstrap step should use all of the init container steps") { + val baseDriverSpec = KubernetesDriverSpec( + driverPod = new PodBuilder().build(), + driverContainer = new ContainerBuilder().build(), + driverSparkConf = new SparkConf(false), + otherKubernetesResources = Seq.empty[HasMetadata]) + val initContainerSteps = Seq( + FirstTestInitContainerConfigurationStep$, + SecondTestInitContainerConfigurationStep$) + val bootstrapStep = new InitContainerBootstrapStep( + initContainerSteps, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec) + assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala === + FirstTestInitContainerConfigurationStep$.additionalLabels) + val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala + assert(additionalDriverEnv.size === 1) + assert(additionalDriverEnv.head.getName === + FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey) + assert(additionalDriverEnv.head.getValue === + FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue) + val driverAnnotations = preparedDriverSpec.driverPod.getMetadata.getAnnotations.asScala + assert(driverAnnotations.size === 1) + val initContainers = OBJECT_MAPPER.readValue( + driverAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainerEnv = initContainers.head.getEnv.asScala + assert(initContainerEnv.size === 1) + assert(initContainerEnv.head.getName === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey) + assert(initContainerEnv.head.getValue === + SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue) + val expectedSparkConf = Map( + EXECUTOR_INIT_CONTAINER_CONFIG_MAP.key -> CONFIG_MAP_NAME, + EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY.key -> CONFIG_MAP_KEY, + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey -> + SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue) + assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf) + assert(preparedDriverSpec.otherKubernetesResources.size === 2) + assert(preparedDriverSpec.otherKubernetesResources.contains( + FirstTestInitContainerConfigurationStep$.additionalKubernetesResource)) + assert(preparedDriverSpec.otherKubernetesResources.exists { + case configMap: ConfigMap => + val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME + val configMapData = configMap.getData.asScala + val hasCorrectNumberOfEntries = configMapData.size == 1 + val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY) + val initContainerProperties = new Properties() + Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) { + initContainerProperties.load(_) + } + val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala + val expectedInitContainerProperties = Map( + SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey -> + SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue) + val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties + hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties + case _ => false + }) + } +} + +private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { + + val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue") + val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY" + val additionalMainContainerEnvValue = "TEST_ENV_MAIN_VALUE" + val additionalKubernetesResource = new SecretBuilder() + .withNewMetadata() + .withName("test-secret") + .endMetadata() + .addToData("secret-key", "secret-value") + .build() + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val driverPod = new PodBuilder(initContainerSpec.podToInitialize) + .editOrNewMetadata() + .addToLabels(additionalLabels.asJava) + .endMetadata() + .build() + val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer) + .addNewEnv() + .withName(additionalMainContainerEnvKey) + .withValue(additionalMainContainerEnvValue) + .endEnv() + .build() + initContainerSpec.copy( + podToInitialize = driverPod, + driverContainer = mainContainer, + initContainerDependentResources = initContainerSpec.initContainerDependentResources ++ + Seq(additionalKubernetesResource)) + } +} + +private object SecondTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep { + val additionalInitContainerEnvKey = "TEST_ENV_INIT_KEY" + val additionalInitContainerEnvValue = "TEST_ENV_INIT_VALUE" + val additionalInitContainerPropertyKey = "spark.initcontainer.testkey" + val additionalInitContainerPropertyValue = "testvalue" + val additionalDriverSparkConfKey = "spark.driver.testkey" + val additionalDriverSparkConfValue = "spark.driver.testvalue" + + override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = { + val initContainer = new ContainerBuilder(initContainerSpec.initContainer) + .addNewEnv() + .withName(additionalInitContainerEnvKey) + .withValue(additionalInitContainerEnvValue) + .endEnv() + .build() + val initContainerProperties = initContainerSpec.initContainerProperties ++ + Map(additionalInitContainerPropertyKey -> additionalInitContainerPropertyValue) + val driverSparkConf = initContainerSpec.additionalDriverSparkConf ++ + Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue) + initContainerSpec.copy( + initContainer = initContainer, + initContainerProperties = initContainerProperties, + additionalDriverSparkConf = driverSparkConf) + } +} 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/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala new file mode 100644 index 0000000000000..fe1af4bc5be2a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.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.kubernetes.submit.submitsteps.initcontainer + +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +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._ + +class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val CONFIG_MAP_NAME = "config-map" + private val CONFIG_MAP_KEY = "config-map-key" + private val POD_LABEL = Map("bootstrap" -> "true") + private val INIT_CONTAINER_NAME = "init-container" + private val DRIVER_CONTAINER_NAME = "driver-container" + + @Mock + private var podAndInitContainerBootstrap : SparkPodInitContainerBootstrap = _ + + before { + MockitoAnnotations.initMocks(this) + when(podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes( + any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] { + override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = { + val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer]) + pod.copy( + pod = + new PodBuilder(pod.pod) + .withNewMetadata() + .addToLabels("bootstrap", "true") + .endMetadata() + .withNewSpec().endSpec() + .build(), + initContainer = + new ContainerBuilder() + .withName(INIT_CONTAINER_NAME).build(), + mainContainer = + new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME).build() + )}}) + } + + test("Test of additionalDriverSparkConf with mix of remote files and jars") { + val baseInitStep = new BaseInitContainerConfigurationStep( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + podAndInitContainerBootstrap) + val expectedDriverSparkConf = Map( + INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH, + INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar", + INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt") + val initContainerSpec = InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod, + Seq.empty[HasMetadata]) + val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec) + assert(expectedDriverSparkConf === returnContainerSpec.initContainerProperties) + assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME) + assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME) + assert(returnContainerSpec.podToInitialize.getMetadata.getLabels.asScala === POD_LABEL) + } +} 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/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala new file mode 100644 index 0000000000000..1cc8007803457 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.initcontainer + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { + private val NAMESPACE = "namespace" + private val APP_RESOURCE_PREFIX = "spark-prefix" + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/jars" + private val FILES_DOWNLOAD_PATH = "/var/data/files" + private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent" + private val APP_ID = "spark-id" + private val CUSTOM_LABEL_KEY = "customLabel" + private val CUSTOM_LABEL_VALUE = "customLabelValue" + private val DEPRECATED_CUSTOM_LABEL_KEY = "deprecatedCustomLabel" + private val DEPRECATED_CUSTOM_LABEL_VALUE = "deprecatedCustomLabelValue" + private val DRIVER_LABELS = Map( + CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE, + DEPRECATED_CUSTOM_LABEL_KEY -> DEPRECATED_CUSTOM_LABEL_VALUE, + SPARK_APP_ID_LABEL -> APP_ID, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map" + private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" + private val STAGING_SERVER_URI = "http://localhost:8000" + + 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) + + 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) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 2) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps(1).isInstanceOf[SubmittedResourcesInitContainerConfigurationStep]) + } + + 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_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]) + } +} 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/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala new file mode 100644 index 0000000000000..2edaba93fe07f --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -0,0 +1,252 @@ +/* + * 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.initcontainer + +import java.io.File +import java.util.UUID + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model._ +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Matchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +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.util.Utils + +class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { + private val RESOURCE_SECRET_NAME = "secret" + private val JARS_RESOURCE_ID = "jarsID" + private val JARS_SECRET = "jarsSecret" + private val FILES_RESOURCE_ID = "filesID" + private val FILES_SECRET = "filesSecret" + private val STAGING_SERVER_URI = "http://localhost:8000" + private val SECRET_MOUNT_PATH = "/tmp" + private val RSS_SECRET = Map( + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> + BaseEncoding.base64().encode(JARS_SECRET.getBytes(Charsets.UTF_8)), + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> + BaseEncoding.base64().encode(FILES_SECRET.getBytes(Charsets.UTF_8)) + ).asJava + private var RSS_WITH_SSL_SECRET: java.util.Map[String, String] = _ + private var TRUSTSTORE_FILENAME: String = "" + private var TRUSTSTORE_FILE: File = _ + private var TRUSTSTORE_URI: Option[String] = None + private val TRUSTSTORE_PASS = "trustStorePassword" + private val TRUSTSTORE_TYPE = "jks" + private var CERT_FILENAME: String = "" + private var CERT_FILE: File = _ + private var CERT_URI: Option[String] = None + + @Mock + private var submittedDependencyUploader: SubmittedDependencyUploader = _ + @Mock + private var submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin = _ + + before { + MockitoAnnotations.initMocks(this) + TRUSTSTORE_FILENAME = createTempFile(".jks") + TRUSTSTORE_FILE = new File(TRUSTSTORE_FILENAME) + TRUSTSTORE_URI = Some(TRUSTSTORE_FILENAME) + CERT_FILENAME = createTempFile("pem") + CERT_FILE = new File(CERT_FILENAME) + CERT_URI = Some(CERT_FILENAME) + RSS_WITH_SSL_SECRET = + (RSS_SECRET.asScala ++ Map( + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY -> + BaseEncoding.base64().encode(Files.toByteArray(TRUSTSTORE_FILE)), + INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY -> + BaseEncoding.base64().encode(Files.toByteArray(CERT_FILE)) + )).asJava + when(submittedDependencyUploader.uploadJars()).thenReturn( + SubmittedResourceIdAndSecret(JARS_RESOURCE_ID, JARS_SECRET) + ) + when(submittedDependencyUploader.uploadFiles()).thenReturn( + SubmittedResourceIdAndSecret(FILES_RESOURCE_ID, FILES_SECRET) + ) + when(submittedResourcesSecretPlugin.addResourceStagingServerSecretVolumeToPod( + any[Pod])).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock) : Pod = { + val pod = invocation.getArgumentAt(0, classOf[Pod]) + new PodBuilder(pod) + .withNewMetadata() + .addToLabels("mountedSecret", "true") + .endMetadata() + .withNewSpec().endSpec() + .build() + }}) + when(submittedResourcesSecretPlugin.mountResourceStagingServerSecretIntoInitContainer( + any[Container])).thenAnswer(new Answer[Container] { + override def answer(invocation: InvocationOnMock) : Container = { + val con = invocation.getArgumentAt(0, classOf[Container]) + new ContainerBuilder(con).withName("mountedSecret").build() + }}) + } + after { + TRUSTSTORE_FILE.delete() + CERT_FILE.delete() + } + test ("testing vanilla prepareInitContainer on resources and properties") { + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + None, + None, + None, + None, + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + assert(returnedInitContainer.initContainer.getName === "mountedSecret") + assert(returnedInitContainer.podToInitialize.getMetadata.getLabels.asScala + === Map("mountedSecret" -> "true")) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.additionalDriverSparkConf === + Map( + EXECUTOR_INIT_CONTAINER_SECRET.key -> RESOURCE_SECRET_NAME, + EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> SECRET_MOUNT_PATH)) + } + + test ("testing prepareInitContainer w/ CERT and TrustStore Files w/o SSL") { + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + TRUSTSTORE_URI, + CERT_URI, + Some(TRUSTSTORE_PASS), + Some(TRUSTSTORE_TYPE), + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString, + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY", + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY" + ) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_WITH_SSL_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + + } + + test ("testing prepareInitContainer w/ local CERT and TrustStore Files w/o SSL") { + val LOCAL_TRUST_FILE = "local:///tmp/trust.jsk" + val LOCAL_CERT_FILE = "local:///tmp/cert.pem" + val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep( + RESOURCE_SECRET_NAME, + STAGING_SERVER_URI, + SECRET_MOUNT_PATH, + false, + Some(LOCAL_TRUST_FILE), + Some(LOCAL_CERT_FILE), + Some(TRUSTSTORE_PASS), + Some(TRUSTSTORE_TYPE), + submittedDependencyUploader, + submittedResourcesSecretPlugin + ) + val returnedInitContainer = + submittedResourceStep.configureInitContainer(InitContainerSpec( + Map.empty[String, String], + Map.empty[String, String], + new Container(), + new Container(), + new Pod(), + Seq.empty[HasMetadata])) + val expectedinitContainerProperties = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString, + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE, + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + "/tmp/trust.jsk", + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key -> + "/tmp/cert.pem" + ) + assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties) + assert(returnedInitContainer.initContainerDependentResources.length == 1) + val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret] + assert(secret.getData === RSS_SECRET) + assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME) + } + private def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} 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/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala new file mode 100644 index 0000000000000..f2fdf026390cd --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -0,0 +1,197 @@ +/* + * 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.rest.kubernetes + +import java.io.{ByteArrayOutputStream, File} +import java.util.UUID +import javax.ws.rs.core + +import com.google.common.base.Charsets +import com.google.common.io.Files +import okhttp3.{MediaType, ResponseBody} +import org.mockito.Matchers.any +import org.mockito.Mockito +import org.mockito.Mockito.{doAnswer, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +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.util.Utils + +class KubernetesSparkDependencyDownloadInitContainerSuite + extends SparkFunSuite with BeforeAndAfter { + import KubernetesSparkDependencyDownloadInitContainerSuite.createTempFile + private val STAGING_SERVER_URI = "http://localhost:8000" + private val TRUSTSTORE_FILE = new File(createTempFile(".jks")) + private val TRUSTSTORE_PASSWORD = "trustStorePassword" + private val TRUSTSTORE_TYPE = "jks" + private val STAGING_SERVER_SSL_OPTIONS = SSLOptions( + enabled = true, + trustStore = Some(TRUSTSTORE_FILE), + trustStorePassword = Some(TRUSTSTORE_PASSWORD), + trustStoreType = Some(TRUSTSTORE_TYPE)) + private val JARS = Seq(createTempFile("jar"), createTempFile("jar")) + private val FILES = Seq(createTempFile("txt"), createTempFile("csv")) + private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") + private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") + private val JARS_RESOURCE_ID = "jarsId" + private val FILES_RESOURCE_ID = "filesId" + + private var downloadJarsDir: File = _ + private var downloadFilesDir: File = _ + private var downloadJarsSecretValue: String = _ + private var downloadFilesSecretValue: String = _ + private var jarsCompressedBytes: Array[Byte] = _ + private var filesCompressedBytes: Array[Byte] = _ + private var retrofitClientFactory: RetrofitClientFactory = _ + private var retrofitClient: ResourceStagingServiceRetrofit = _ + private var fileFetcher: FileFetcher = _ + + override def beforeAll(): Unit = { + jarsCompressedBytes = compressPathsToBytes(JARS) + filesCompressedBytes = compressPathsToBytes(FILES) + downloadJarsSecretValue = Files.toString( + new File(DOWNLOAD_JARS_SECRET_LOCATION), Charsets.UTF_8) + downloadFilesSecretValue = Files.toString( + new File(DOWNLOAD_FILES_SECRET_LOCATION), Charsets.UTF_8) + } + + before { + downloadJarsDir = Utils.createTempDir() + downloadFilesDir = Utils.createTempDir() + retrofitClientFactory = mock[RetrofitClientFactory] + retrofitClient = mock[ResourceStagingServiceRetrofit] + fileFetcher = mock[FileFetcher] + when(retrofitClientFactory.createRetrofitClient( + STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) + .thenReturn(retrofitClient) + } + + after { + downloadJarsDir.delete() + downloadFilesDir.delete() + } + + test("Downloads from resource staging server should unpack response body to directories") { + val downloadJarsCall = mock[Call[ResponseBody]] + val downloadFilesCall = mock[Call[ResponseBody]] + val sparkConf = getSparkConfForResourceStagingServerDownloads + val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + retrofitClientFactory, + fileFetcher, + resourceStagingServerSslOptions = STAGING_SERVER_SSL_OPTIONS) + when(retrofitClient.downloadResources(JARS_RESOURCE_ID, downloadJarsSecretValue)) + .thenReturn(downloadJarsCall) + when(retrofitClient.downloadResources(FILES_RESOURCE_ID, downloadFilesSecretValue)) + .thenReturn(downloadFilesCall) + val jarsResponseBody = ResponseBody.create( + MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), jarsCompressedBytes) + val filesResponseBody = ResponseBody.create( + MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), filesCompressedBytes) + doAnswer(new InvokeCallbackAnswer(downloadJarsCall, jarsResponseBody)) + .when(downloadJarsCall) + .enqueue(any()) + doAnswer(new InvokeCallbackAnswer(downloadFilesCall, filesResponseBody)) + .when(downloadFilesCall) + .enqueue(any()) + initContainerUnderTest.run() + checkWrittenFilesAreTheSameAsOriginal(JARS, downloadJarsDir) + checkWrittenFilesAreTheSameAsOriginal(FILES, downloadFilesDir) + Mockito.verifyZeroInteractions(fileFetcher) + } + + test("Downloads from remote server should invoke the file fetcher") { + val sparkConf = getSparkConfForRemoteFileDownloads + val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + retrofitClientFactory, + fileFetcher, + resourceStagingServerSslOptions = STAGING_SERVER_SSL_OPTIONS) + initContainerUnderTest.run() + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir) + Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/file.txt", downloadFilesDir) + + } + + private def getSparkConfForResourceStagingServerDownloads: SparkConf = { + new SparkConf(true) + .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) + .set(INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER, JARS_RESOURCE_ID) + .set(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION, DOWNLOAD_JARS_SECRET_LOCATION) + .set(INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER, FILES_RESOURCE_ID) + .set(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION, DOWNLOAD_FILES_SECRET_LOCATION) + .set(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) + .set(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath) + .set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) + .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, TRUSTSTORE_FILE.getAbsolutePath) + .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD, TRUSTSTORE_PASSWORD) + .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE, TRUSTSTORE_TYPE) + } + + private def getSparkConfForRemoteFileDownloads: SparkConf = { + new SparkConf(true) + .set(INIT_CONTAINER_REMOTE_JARS, + "http://localhost:9000/jar1.jar,hdfs://localhost:9000/jar2.jar") + .set(INIT_CONTAINER_REMOTE_FILES, + "http://localhost:9000/file.txt") + .set(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) + .set(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath) + } + + private def checkWrittenFilesAreTheSameAsOriginal( + originalFiles: Iterable[String], downloadDir: File): Unit = { + originalFiles.map(new File(_)).foreach { file => + val writtenFile = new File(downloadDir, file.getName) + assert(writtenFile.exists) + val originalJarContents = Seq(Files.toByteArray(file): _*) + val writtenJarContents = Seq(Files.toByteArray(writtenFile): _*) + assert(writtenJarContents === originalJarContents) + } + } + + private def compressPathsToBytes(paths: Iterable[String]): Array[Byte] = { + Utils.tryWithResource(new ByteArrayOutputStream()) { compressedBytes => + CompressionUtils.writeTarGzipToStream (compressedBytes, paths) + compressedBytes.toByteArray + } + } +} + +private object KubernetesSparkDependencyDownloadInitContainerSuite { + def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} + +private class InvokeCallbackAnswer(call: Call[ResponseBody], responseBody: ResponseBody) + extends Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val callback = invocationOnMock.getArgumentAt(0, classOf[Callback[ResponseBody]]) + val response = Response.success(responseBody) + callback.onResponse(call, response) + } +} 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/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala new file mode 100644 index 0000000000000..3bb318d713a54 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala @@ -0,0 +1,147 @@ +/* + * 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.rest.kubernetes + +import java.io.{File, FileInputStream, StringWriter} +import java.security.KeyStore + +import com.google.common.base.Charsets +import com.google.common.io.Files +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.util.Utils + +class ResourceStagingServerSslOptionsProviderSuite extends SparkFunSuite with BeforeAndAfter { + + private var sslTempDir: File = _ + private var keyStoreFile: File = _ + + private var sparkConf: SparkConf = _ + private var sslOptionsProvider: ResourceStagingServerSslOptionsProvider = _ + + before { + sslTempDir = Utils.createTempDir(namePrefix = "resource-staging-server-ssl-test") + keyStoreFile = new File(sslTempDir, "keyStore.jks") + sparkConf = new SparkConf(true) + sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf) + } + + test("Default SparkConf does not have TLS enabled.") { + assert(sslOptionsProvider.getSslOptions === SSLOptions()) + assert(!sslOptionsProvider.getSslOptions.enabled) + keyStoreFile.delete() + sslTempDir.delete() + } + + test("Setting keyStore, key password, and key field directly.") { + sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true") + .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", keyStoreFile.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStorePassword") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "keyPassword") + val sslOptions = sslOptionsProvider.getSslOptions + assert(sslOptions.enabled, "SSL should be enabled.") + assert(sslOptions.keyStore.map(_.getAbsolutePath) === Some(keyStoreFile.getAbsolutePath), + "Incorrect keyStore path or it was not set.") + assert(sslOptions.keyStorePassword === Some("keyStorePassword"), + "Incorrect keyStore password or it was not set.") + assert(sslOptions.keyPassword === Some("keyPassword"), + "Incorrect key password or it was not set.") + } + + test("Setting key and certificate pem files should write an appropriate keyStore.") { + val keyAndCertPem = SSLUtils.generateKeyCertPemPair("127.0.0.1") + sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPem", + keyAndCertPem.keyPem.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.serverCertPem", + keyAndCertPem.certPem.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStorePassword") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "keyPassword") + val sslOptions = sslOptionsProvider.getSslOptions + assert(sslOptions.enabled, "SSL should be enabled.") + assert(sslOptions.keyStore.isDefined, "KeyStore should be defined.") + sslOptions.keyStore.foreach { keyStoreFile => + val keyStore = KeyStore.getInstance(KeyStore.getDefaultType) + Utils.tryWithResource(new FileInputStream(keyStoreFile)) { + keyStore.load(_, "keyStorePassword".toCharArray) + } + val key = keyStore.getKey("key", "keyPassword".toCharArray) + compareJcaPemObjectToFileString(key, keyAndCertPem.keyPem) + val certificate = keyStore.getCertificateChain("key")(0) + compareJcaPemObjectToFileString(certificate, keyAndCertPem.certPem) + } + } + + test("Setting pem files without setting passwords should use random passwords.") { + val keyAndCertPem = SSLUtils.generateKeyCertPemPair("127.0.0.1") + sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPem", + keyAndCertPem.keyPem.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.serverCertPem", + keyAndCertPem.certPem.getAbsolutePath) + val sslOptions = sslOptionsProvider.getSslOptions + assert(sslOptions.enabled, "SSL should be enabled.") + assert(sslOptions.keyStore.isDefined, "KeyStore should be defined.") + assert(sslOptions.keyStorePassword.isDefined) + assert(sslOptions.keyPassword.isDefined) + for { + keyStoreFile <- sslOptions.keyStore + keyStorePassword <- sslOptions.keyStorePassword + keyPassword <- sslOptions.keyPassword + } { + val keyStore = KeyStore.getInstance(KeyStore.getDefaultType) + Utils.tryWithResource(new FileInputStream(keyStoreFile)) { + keyStore.load(_, keyStorePassword.toCharArray) + } + val key = keyStore.getKey("key", keyPassword.toCharArray) + compareJcaPemObjectToFileString(key, keyAndCertPem.keyPem) + val certificate = keyStore.getCertificateChain("key")(0) + compareJcaPemObjectToFileString(certificate, keyAndCertPem.certPem) + } + } + + test("Using password files should read from the appropriate locations.") { + val keyStorePasswordFile = new File(sslTempDir, "keyStorePassword.txt") + Files.write("keyStorePassword", keyStorePasswordFile, Charsets.UTF_8) + val keyPasswordFile = new File(sslTempDir, "keyPassword.txt") + Files.write("keyPassword", keyPasswordFile, Charsets.UTF_8) + sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true") + .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", keyStoreFile.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile", + keyStorePasswordFile.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile", + keyPasswordFile.getAbsolutePath) + val sslOptions = sslOptionsProvider.getSslOptions + assert(sslOptions.keyStorePassword === Some("keyStorePassword"), + "Incorrect keyStore password or it was not set.") + assert(sslOptions.keyPassword === Some("keyPassword"), + "Incorrect key password or it was not set.") + } + + private def compareJcaPemObjectToFileString(pemObject: Any, pemFile: File): Unit = { + Utils.tryWithResource(new StringWriter()) { stringWriter => + Utils.tryWithResource(new JcaPEMWriter(stringWriter)) { pemWriter => + pemWriter.writeObject(pemObject) + } + val pemFileAsString = Files.toString(pemFile, Charsets.UTF_8) + assert(stringWriter.toString === pemFileAsString) + } + } +} 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/kubernetes/ResourceStagingServerSuite.scala new file mode 100644 index 0000000000000..1bcd85a611e00 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala @@ -0,0 +1,185 @@ +/* + * 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.rest.kubernetes + +import java.net.ServerSocket +import javax.ws.rs.core.MediaType + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import com.google.common.io.ByteStreams +import okhttp3.{RequestBody, ResponseBody} +import org.eclipse.jetty.server.Server +import org.scalatest.BeforeAndAfter +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.internal.Logging +import org.apache.spark.util.Utils + +/** + * Tests for {@link ResourceStagingServer} and its APIs. Note that this is not an end-to-end + * integration test, and as such does not upload and download files in tar.gz as would be done + * in production. Thus we use the retrofit clients directly despite the fact that in practice + * we would likely want to create an opinionated abstraction on top of the retrofit client; we + * can test this abstraction layer separately, however. This test is mainly for checking that + * we've configured the Jetty server correctly and that the endpoints reached over HTTP can + * receive streamed uploads and can stream downloads. + */ +class ResourceStagingServerSuite extends SparkFunSuite with BeforeAndAfter with Logging { + + private val MAX_SERVER_START_ATTEMPTS = 5 + private var serviceImpl: ResourceStagingService = _ + private var stagedResourcesCleaner: StagedResourcesCleaner = _ + private var server: Option[ResourceStagingServer] = None + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + + private val sslOptionsProvider = new SettableReferenceSslOptionsProvider() + + before { + stagedResourcesCleaner = mock[StagedResourcesCleaner] + serviceImpl = new ResourceStagingServiceImpl( + new StagedResourcesStoreImpl(Utils.createTempDir()), stagedResourcesCleaner) + } + + after { + server.foreach { s => + try { + s.stop() + } catch { + case e: Throwable => + log.warn("Failed to stop the resource staging server.", e) + } + } + server = None + } + + test("Accept file and jar uploads and downloads") { + val serverPort = startServer() + runUploadAndDownload(SSLOptions(), serverPort) + } + + test("Enable SSL on the server") { + val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair( + ipAddress = "127.0.0.1", + keyStorePassword = "keyStore", + keyPassword = "key", + trustStorePassword = "trustStore") + val sslOptions = SSLOptions( + enabled = true, + keyStore = Some(keyStoreAndTrustStore.keyStore), + keyStorePassword = Some("keyStore"), + keyPassword = Some("key"), + trustStore = Some(keyStoreAndTrustStore.trustStore), + trustStorePassword = Some("trustStore")) + sslOptionsProvider.setOptions(sslOptions) + val serverPort = startServer() + runUploadAndDownload(sslOptions, serverPort) + } + + private def runUploadAndDownload(sslOptions: SSLOptions, serverPort: Int): Unit = { + val scheme = if (sslOptions.enabled) "https" else "http" + val retrofitService = RetrofitClientFactoryImpl.createRetrofitClient( + s"$scheme://127.0.0.1:$serverPort/", + classOf[ResourceStagingServiceRetrofit], + sslOptions) + val resourcesBytes = Array[Byte](1, 2, 3, 4) + val labels = Map("label1" -> "label1Value", "label2" -> "label2value") + val namespace = "namespace" + val resourcesOwner = StagedResourcesOwner( + ownerLabels = labels, + ownerNamespace = namespace, + ownerType = StagedResourcesOwnerType.Pod) + val resourcesOwnerJson = OBJECT_MAPPER.writeValueAsString(resourcesOwner) + val resourcesOwnerRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), resourcesOwnerJson) + val resourcesRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.MULTIPART_FORM_DATA), resourcesBytes) + val uploadResponse = retrofitService.uploadResources( + resourcesRequestBody, resourcesOwnerRequestBody) + val resourceIdentifier = getTypedResponseResult(uploadResponse) + checkResponseBodyBytesMatches( + retrofitService.downloadResources( + resourceIdentifier.resourceId, resourceIdentifier.resourceSecret), resourcesBytes) + } + + private def getTypedResponseResult[T](call: Call[T]): T = { + val response = call.execute() + assert(response.code() >= 200 && response.code() < 300, Option(response.errorBody()) + .map(_.string()) + .getOrElse("Error executing HTTP request, but error body was not provided.")) + val callResult = response.body() + assert(callResult != null) + callResult + } + + private def checkResponseBodyBytesMatches(call: Call[ResponseBody], bytes: Array[Byte]): Unit = { + val responseBody = getTypedResponseResult(call) + val downloadedBytes = ByteStreams.toByteArray(responseBody.byteStream()) + assert(downloadedBytes.toSeq === bytes) + } + + private def startServer(): Int = { + var currentAttempt = 0 + var successfulStart = false + var latestServerPort = new ServerSocket(0).getLocalPort + while (currentAttempt < MAX_SERVER_START_ATTEMPTS && !successfulStart) { + val newServer = new ResourceStagingServer(latestServerPort, serviceImpl, sslOptionsProvider) + try { + newServer.start() + successfulStart = true + server = Some(newServer) + } catch { + case e: Throwable => + try { + newServer.stop() + } catch { + case e1: Throwable => + log.warn("Failed to stop a resource staging server that failed to start.", e1) + } + + if (Utils.isBindCollision(e)) { + currentAttempt += 1 + latestServerPort = latestServerPort + 1 + if (currentAttempt == MAX_SERVER_START_ATTEMPTS) { + throw new RuntimeException(s"Failed to bind to a random port" + + s" $MAX_SERVER_START_ATTEMPTS times. Last attempted port: $latestServerPort", e) + } else { + logWarning(s"Attempt $currentAttempt/$MAX_SERVER_START_ATTEMPTS failed to start" + + s" server on port $latestServerPort.", e) + } + } else { + throw e + } + } + } + logInfo(s"Started resource staging server on port $latestServerPort.") + latestServerPort + } +} + +private class SettableReferenceSslOptionsProvider extends ResourceStagingServerSslOptionsProvider { + private var options = SSLOptions() + + def setOptions(newOptions: SSLOptions): Unit = { + this.options = newOptions + } + + override def getSslOptions: SSLOptions = options +} 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/kubernetes/StagedResourcesCleanerSuite.scala new file mode 100644 index 0000000000000..8b398a9891f34 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala @@ -0,0 +1,149 @@ +/* + * 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.rest.kubernetes + +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import io.fabric8.kubernetes.api.model.{DoneableNamespace, DoneablePod, Namespace, NamespaceList, Pod, PodList, PodListBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource, Resource} +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{eq => mockitoEq} +import org.mockito.Mockito.{never, verify, when} +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Clock + +private[spark] class StagedResourcesCleanerSuite extends SparkFunSuite with BeforeAndAfter { + + private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + private type PODSWITHLABELS = FilterWatchListDeletable[ + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + private type PODSINNAMESPACE = NonNamespaceOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + private type NAMESPACES = NonNamespaceOperation[ + Namespace, NamespaceList, DoneableNamespace, Resource[Namespace, DoneableNamespace]] + private type NAMESPACEWITHNAME = Resource[Namespace, DoneableNamespace] + + private val INITIAL_ACCESS_EXPIRATION_MS = 5000L + private val CURRENT_TIME = 10000L + private val RESOURCE_ID = "resource-id" + private val POD_NAMESPACE = "namespace" + private val POD_LABELS = Map("label1" -> "label1value", "label2" -> "label2value") + private val RESOURCES_OWNER = StagedResourcesOwner( + ownerNamespace = POD_NAMESPACE, + ownerLabels = POD_LABELS, + ownerType = StagedResourcesOwnerType.Pod) + + @Mock + private var stagedResourcesStore: StagedResourcesStore = _ + @Mock + private var kubernetesClient: KubernetesClient = _ + @Mock + private var clock: Clock = _ + @Mock + private var cleanerExecutorService: ScheduledExecutorService = _ + @Mock + private var podOperations: PODS = _ + @Mock + private var podsInNamespaceOperations: PODSINNAMESPACE = _ + @Mock + private var podsWithLabelsOperations: PODSWITHLABELS = _ + @Mock + private var namespaceOperations: NAMESPACES = _ + @Mock + private var namedNamespaceOperations: NAMESPACEWITHNAME = _ + private var cleanerUnderTest: StagedResourcesCleaner = _ + + before { + MockitoAnnotations.initMocks(this) + cleanerUnderTest = new StagedResourcesCleanerImpl( + stagedResourcesStore, + kubernetesClient, + cleanerExecutorService, + clock, + INITIAL_ACCESS_EXPIRATION_MS) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabels(POD_LABELS.asJava)).thenReturn(podsWithLabelsOperations) + when(kubernetesClient.namespaces()).thenReturn(namespaceOperations) + } + + test("Clean the resource if it is never accessed for the expiration interval.") { + val cleanupRunnable = startCleanupAndGetCleanupRunnable() + cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER) + when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS) + cleanupRunnable.run() + verify(stagedResourcesStore).removeResources(RESOURCE_ID) + verify(kubernetesClient, never()).pods() + } + + test("Don't clean the resource if it is accessed in the expiration interval" + + " and there are owners available.") { + val cleanupRunnable = startCleanupAndGetCleanupRunnable() + cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER) + cleanerUnderTest.markResourceAsUsed(RESOURCE_ID) + when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS) + when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations) + when(namedNamespaceOperations.get()).thenReturn(new Namespace()) + when(podOperations.inNamespace(POD_NAMESPACE)).thenReturn(podsInNamespaceOperations) + when(podsInNamespaceOperations.withLabels(POD_LABELS.asJava)) + .thenReturn(podsWithLabelsOperations) + when(podsWithLabelsOperations.list()).thenReturn( + new PodListBuilder().addNewItemLike(new Pod()).endItem().build()) + cleanupRunnable.run() + verify(stagedResourcesStore, never()).removeResources(RESOURCE_ID) + } + + test("Clean the resource if no owners are available.") { + val cleanupRunnable = startCleanupAndGetCleanupRunnable() + cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER) + cleanerUnderTest.markResourceAsUsed(RESOURCE_ID) + when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS) + when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations) + when(namedNamespaceOperations.get()).thenReturn(new Namespace()) + when(podOperations.inNamespace(POD_NAMESPACE)).thenReturn(podsInNamespaceOperations) + when(podsInNamespaceOperations.withLabels(POD_LABELS.asJava)) + .thenReturn(podsWithLabelsOperations) + when(podsWithLabelsOperations.list()).thenReturn(new PodListBuilder().build()) + cleanupRunnable.run() + verify(stagedResourcesStore).removeResources(RESOURCE_ID) + } + + test("Clean up the resource if the namespace does not exist.") { + val cleanupRunnable = startCleanupAndGetCleanupRunnable() + cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER) + cleanerUnderTest.markResourceAsUsed(RESOURCE_ID) + when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS) + when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations) + when(namedNamespaceOperations.get()).thenReturn(null) + cleanupRunnable.run() + verify(stagedResourcesStore).removeResources(RESOURCE_ID) + } + + private def startCleanupAndGetCleanupRunnable(): Runnable = { + val captor = ArgumentCaptor.forClass(classOf[Runnable]) + cleanerUnderTest.start() + verify(cleanerExecutorService).scheduleAtFixedRate( + captor.capture(), + mockitoEq(30000L), + mockitoEq(30000L), + mockitoEq(TimeUnit.MILLISECONDS)) + captor.getValue + } +} 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/kubernetes/StagedResourcesStoreSuite.scala new file mode 100644 index 0000000000000..6b5737ebf2e23 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala @@ -0,0 +1,86 @@ +/* + * 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.rest.kubernetes + +import java.io.{ByteArrayInputStream, File} +import java.nio.file.Paths + +import com.google.common.io.Files +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Utils + +private[spark] class StagedResourcesStoreSuite extends SparkFunSuite with BeforeAndAfter { + + private val resourceBytes = Array[Byte](1, 2, 3, 4) + private val namespace = "namespace" + private var dependencyRootDir: File = _ + private var stagedResourcesStore: StagedResourcesStore = _ + + before { + dependencyRootDir = Utils.createTempDir() + stagedResourcesStore = new StagedResourcesStoreImpl(dependencyRootDir) + } + + after { + dependencyRootDir.delete() + } + + test("Uploads should write data to the underlying disk") { + val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) { + resourceStream => + stagedResourcesStore.addResources(namespace, resourceStream) + } + val resourceNamespaceDir = Paths.get(dependencyRootDir.getAbsolutePath, "namespace").toFile + assert(resourceNamespaceDir.isDirectory, s"Resource namespace dir was not created at" + + s" ${resourceNamespaceDir.getAbsolutePath} or is not a directory.") + val resourceDirs = resourceNamespaceDir.listFiles() + assert(resourceDirs.length === 1, s"Resource root directory did not have exactly one" + + s" subdirectory. Got: ${resourceDirs.map(_.getAbsolutePath).mkString(",")}") + assert(resourceDirs(0).getName === resourceIdAndSecret.resourceId) + val resourceTgz = new File(resourceDirs(0), "resources.data") + assert(resourceTgz.isFile, + s"Resources written to ${resourceTgz.getAbsolutePath} does not exist or is not a file.") + val resourceTgzBytes = Files.toByteArray(resourceTgz) + assert(resourceTgzBytes.toSeq === resourceBytes.toSeq, "Incorrect resource bytes were written.") + } + + test("Uploading and then getting should return a stream with the written bytes.") { + val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) { + resourceStream => + stagedResourcesStore.addResources(namespace, resourceStream) + } + val resources = stagedResourcesStore.getResources(resourceIdAndSecret.resourceId) + assert(resources.map(_.resourcesFile) + .map(Files.toByteArray) + .exists(resourceBytes.sameElements(_))) + assert(resources.exists(_.resourceId == resourceIdAndSecret.resourceId)) + assert(resources.exists(_.resourceSecret == resourceIdAndSecret.resourceSecret)) + } + + test("Uploading and then deleting should result in the resource directory being deleted.") { + val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) { + resourceStream => + stagedResourcesStore.addResources(namespace, resourceStream) + } + stagedResourcesStore.removeResources(resourceIdAndSecret.resourceId) + val resourceNamespaceDir = Paths.get(dependencyRootDir.getAbsolutePath, "namespace").toFile + assert(resourceNamespaceDir.listFiles().isEmpty) + assert(stagedResourcesStore.getResources(resourceIdAndSecret.resourceId).isEmpty) + } +} 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/kubernetes/KubernetesTaskSetManagerSuite.scala new file mode 100644 index 0000000000000..864ff40d88c5c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala @@ -0,0 +1,149 @@ +/* + * 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.mutable.ArrayBuffer + +import io.fabric8.kubernetes.api.model.{Pod, PodSpec, PodStatus} +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.scheduler.{FakeTask, FakeTaskScheduler, HostTaskLocation, TaskLocation} + +class KubernetesTaskSetManagerSuite extends SparkFunSuite with BeforeAndAfter { + + val sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, + ("execA", "10.0.0.1"), ("execB", "10.0.0.2"), ("execC", "10.0.0.3")) + val backend = mock(classOf[KubernetesClusterSchedulerBackend]) + sched.backend = backend + + before { + sc.conf.remove(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED) + } + + test("Find pending tasks for executors using executor pod IP addresses") { + val taskSet = FakeTask.createTaskSet(3, + Seq(TaskLocation("10.0.0.1", "execA")), // Task 0 runs on executor pod 10.0.0.1. + Seq(TaskLocation("10.0.0.1", "execA")), // Task 1 runs on executor pod 10.0.0.1. + Seq(TaskLocation("10.0.0.2", "execB")) // Task 2 runs on executor pod 10.0.0.2. + ) + + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0)) + assert(manager.getPendingTasksForHost("10.0.0.2") == ArrayBuffer(2)) + } + + test("Find pending tasks for executors using cluster node names that executor pods run on") { + val taskSet = FakeTask.createTaskSet(2, + Seq(HostTaskLocation("kube-node1")), // Task 0's partition belongs to datanode on kube-node1 + Seq(HostTaskLocation("kube-node1")) // Task 1's partition belongs to datanode on kube-node2 + ) + val spec1 = mock(classOf[PodSpec]) + when(spec1.getNodeName).thenReturn("kube-node1") + val pod1 = mock(classOf[Pod]) + when(pod1.getSpec).thenReturn(spec1) + when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1)) + + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0)) + } + + test("Find pending tasks for executors using cluster node IPs that executor pods run on") { + val taskSet = FakeTask.createTaskSet(2, + Seq(HostTaskLocation("196.0.0.5")), // Task 0's partition belongs to datanode on 196.0.0.5. + Seq(HostTaskLocation("196.0.0.5")) // Task 1's partition belongs to datanode on 196.0.0.5. + ) + val spec1 = mock(classOf[PodSpec]) + when(spec1.getNodeName).thenReturn("kube-node1") + val pod1 = mock(classOf[Pod]) + when(pod1.getSpec).thenReturn(spec1) + val status1 = mock(classOf[PodStatus]) + when(status1.getHostIP).thenReturn("196.0.0.5") + when(pod1.getStatus).thenReturn(status1) + when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1)) + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0)) + } + + test("Test DNS lookup is disabled by default for cluster node full hostnames") { + assert(!sc.conf.get(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED)) + } + + test("Find pending tasks for executors, but avoid looking up cluster node FQDNs from DNS") { + sc.conf.set(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED, false) + val taskSet = FakeTask.createTaskSet(2, + Seq(HostTaskLocation("kube-node1.domain1")), // Task 0's partition belongs to datanode here. + Seq(HostTaskLocation("kube-node1.domain1")) // task 1's partition belongs to datanode here. + ) + val spec1 = mock(classOf[PodSpec]) + when(spec1.getNodeName).thenReturn("kube-node1") + val pod1 = mock(classOf[Pod]) + when(pod1.getSpec).thenReturn(spec1) + val status1 = mock(classOf[PodStatus]) + when(status1.getHostIP).thenReturn("196.0.0.5") + when(pod1.getStatus).thenReturn(status1) + val inetAddressUtil = mock(classOf[InetAddressUtil]) + when(inetAddressUtil.getFullHostName("196.0.0.5")).thenReturn("kube-node1.domain1") + when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1)) + + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer()) + } + + test("Find pending tasks for executors using cluster node FQDNs that executor pods run on") { + sc.conf.set(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED, true) + val taskSet = FakeTask.createTaskSet(2, + Seq(HostTaskLocation("kube-node1.domain1")), // Task 0's partition belongs to datanode here. + Seq(HostTaskLocation("kube-node1.domain1")) // task 1's partition belongs to datanode here. + ) + val spec1 = mock(classOf[PodSpec]) + when(spec1.getNodeName).thenReturn("kube-node1") + val pod1 = mock(classOf[Pod]) + when(pod1.getSpec).thenReturn(spec1) + val status1 = mock(classOf[PodStatus]) + when(status1.getHostIP).thenReturn("196.0.0.5") + when(pod1.getStatus).thenReturn(status1) + val inetAddressUtil = mock(classOf[InetAddressUtil]) + when(inetAddressUtil.getFullHostName("196.0.0.5")).thenReturn("kube-node1.domain1") + when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1)) + + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0)) + } + + test("Return empty pending tasks for executors when all look up fail") { + val taskSet = FakeTask.createTaskSet(1, + Seq(HostTaskLocation("kube-node1.domain1")) // task 0's partition belongs to datanode here. + ) + val spec1 = mock(classOf[PodSpec]) + when(spec1.getNodeName).thenReturn("kube-node2") + val pod1 = mock(classOf[Pod]) + when(pod1.getSpec).thenReturn(spec1) + val status1 = mock(classOf[PodStatus]) + when(status1.getHostIP).thenReturn("196.0.0.6") + when(pod1.getStatus).thenReturn(status1) + val inetAddressUtil = mock(classOf[InetAddressUtil]) + when(inetAddressUtil.getFullHostName("196.0.0.6")).thenReturn("kube-node2.domain1") + when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1)) + + val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil) + assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer()) + } +} diff --git a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml new file mode 100644 index 0000000000000..31184794972db --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -0,0 +1,125 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.3.1-SNAPSHOT + ../../../pom.xml + + + spark-docker-minimal-bundle_2.11 + Spark Project Docker Minimal Bundle + http://spark.apache.org/ + pom + + + kubernetes-docker-minimal-bundle + none + pre-integration-test + + + + + org.apache.spark + spark-assembly_${scala.binary.version} + ${project.version} + pom + + + + + com.google.guava + guava + ${hadoop.deps.scope} + + + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + docker-dist + pre-integration-test + + single + + + + src/main/assembly/docker-assembly.xml + + posix + + + + + + + + + + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + + + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + + + spark-ganglia-lgpl + + + org.apache.spark + spark-ganglia-lgpl_${scala.binary.version} + ${project.version} + + + + + diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml new file mode 100644 index 0000000000000..2b48d366256fe --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml @@ -0,0 +1,75 @@ + + + docker-dist + + tar.gz + dir + + false + + + + ${project.parent.basedir}/sbin/ + + sbin + + **/* + + + + + ${project.parent.basedir}/bin/ + + bin + + **/* + + + + + ${project.parent.basedir}/conf/ + + conf + + **/* + + + + + src/main/docker/ + + dockerfiles + + **/* + + + + + + jars + true + false + runtime + false + + org.apache.spark:spark-assembly_${scala.binary.version}:pom + org.spark-project.spark:unused + + + + diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile new file mode 100644 index 0000000000000..6dcc7511c0dd9 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile @@ -0,0 +1,48 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-driver-py:latest -f dockerfiles/driver-py/Dockerfile . + +ADD examples /opt/spark/examples +ADD python /opt/spark/python + +RUN apk add --no-cache python && \ + python -m ensurepip && \ + rm -r /usr/lib/python*/ensurepip && \ + pip install --upgrade pip setuptools && \ + rm -r /root/.cache +# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES +# RUN apk add --update alpine-sdk python-dev +# RUN pip install numpy + +ENV PYTHON_VERSION 2.7.13 +ENV PYSPARK_PYTHON python +ENV PYSPARK_DRIVER_PYTHON python +ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH} + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH \ + -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY \ + $SPARK_DRIVER_CLASS $PYSPARK_PRIMARY $PYSPARK_FILES $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile new file mode 100644 index 0000000000000..6bbff8ef64a0f --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -0,0 +1,31 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile . + +COPY examples /opt/spark/examples + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile new file mode 100644 index 0000000000000..7a65a4f879376 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile @@ -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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor-py:latest -f dockerfiles/executor-py/Dockerfile . + +ADD examples /opt/spark/examples +ADD python /opt/spark/python + +RUN apk add --no-cache python && \ + python -m ensurepip && \ + rm -r /usr/lib/python*/ensurepip && \ + pip install --upgrade pip setuptools && \ + rm -r /root/.cache +# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES +# RUN apk add --update alpine-sdk python-dev +# RUN pip install numpy + +ENV PYTHON_VERSION 2.7.13 +ENV PYSPARK_PYTHON python +ENV PYSPARK_DRIVER_PYTHON python +ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH} + +# TODO support spark.executor.extraClassPath +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP \ No newline at end of file diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile new file mode 100644 index 0000000000000..b3b0acc3b64b8 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -0,0 +1,32 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . + +COPY examples /opt/spark/examples + +# TODO support spark.executor.extraClassPath +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \ + exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP 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 new file mode 100644 index 0000000000000..4bafe25e2608f --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile @@ -0,0 +1,24 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# 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 [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.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 new file mode 100644 index 0000000000000..9ca96be0f1a88 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile @@ -0,0 +1,25 @@ +# +# 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. +# + +FROM spark-base + + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# 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 [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.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 new file mode 100644 index 0000000000000..ccb2f1a03d88c --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile @@ -0,0 +1,26 @@ +# +# 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. +# + +FROM spark-base + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# 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 . + +COPY examples /opt/spark/examples + +ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService", "1" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile new file mode 100644 index 0000000000000..b0925e3bb0416 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile @@ -0,0 +1,35 @@ +# 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. +# + +FROM openjdk:8-alpine + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-base:latest -f dockerfiles/spark-base/Dockerfile . + +RUN apk upgrade --no-cache && \ + apk add --no-cache bash tini && \ + mkdir -p /opt/spark && \ + touch /opt/spark/RELEASE + +COPY jars /opt/spark/jars +COPY bin /opt/spark/bin +COPY sbin /opt/spark/sbin +COPY conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml new file mode 100644 index 0000000000000..94c982d5e1108 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml @@ -0,0 +1,36 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.3.1-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs-helpers_2.11 + jar + Spark Project Kubernetes Integration Tests Spark Jobs Helpers + + kubernetes-integration-tests-spark-jobs-helpers + + + + + 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/kubernetes/integrationtest/PiHelper.java new file mode 100644 index 0000000000000..99d982397bb6e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java @@ -0,0 +1,33 @@ +/* + * 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.integrationtest; + +/** + * Primarily extracted so that a separate jar can be added as a dependency for the + * test Spark job. + */ +public class PiHelper { + public static int helpPi() { + double x = Math.random() * 2 - 1; + double y = Math.random() * 2 - 1; + if (x*x + y*y < 1) { + return 1; + } else { + return 0; + } + } +} diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml new file mode 100644 index 0000000000000..9183bbc4f72e8 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml @@ -0,0 +1,54 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.3.1-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs_2.11 + jar + Spark Project Kubernetes Integration Tests Spark Jobs + + kubernetes-integration-tests-spark-jobs + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + provided + + + 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/kubernetes/integrationtest/jobs/FileExistenceTest.scala new file mode 100644 index 0000000000000..8994c998bffee --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala @@ -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. + */ +package org.apache.spark.deploy.kubernetes.integrationtest.jobs + +import java.nio.file.Paths + +import com.google.common.base.Charsets +import com.google.common.io.Files + +import org.apache.spark.SparkException +import org.apache.spark.sql.SparkSession + +private[spark] object FileExistenceTest { + + def main(args: Array[String]): Unit = { + if (args.length < 2) { + throw new IllegalArgumentException( + s"Invalid args: ${args.mkString}, " + + "Usage: FileExistenceTest ") + } + // Can't use SparkContext.textFile since the file is local to the driver + val file = Paths.get(args(0)).toFile + if (!file.exists()) { + throw new SparkException(s"Failed to find file at ${file.getAbsolutePath}") + } else { + // scalastyle:off println + val contents = Files.toString(file, Charsets.UTF_8) + if (args(1) != contents) { + throw new SparkException(s"Contents do not match. Expected: ${args(1)}," + + s" actual: $contents") + } else { + println(s"File found at ${file.getAbsolutePath} with correct contents.") + } + // scalastyle:on println + } + while (true) { + Thread.sleep(600000) + } + } + +} 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/kubernetes/integrationtest/jobs/GroupByTest.scala new file mode 100644 index 0000000000000..fe47d42485b24 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala @@ -0,0 +1,54 @@ +/* + * 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.integrationtest.jobs + +import java.util.Random + +import org.apache.spark.sql.SparkSession + +object GroupByTest { + def main(args: Array[String]) { + val spark = SparkSession + .builder + .appName("GroupBy Test") + .getOrCreate() + + val numMappers = if (args.length > 0) args(0).toInt else 5 + val numKVPairs = if (args.length > 1) args(1).toInt else 200000 + val valSize = if (args.length > 2) args(2).toInt else 2 + val numReducers = if (args.length > 3) args(3).toInt else numMappers + + val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p => + val ranGen = new Random + val arr1 = new Array[(Int, Array[Byte])](numKVPairs) + for (i <- 0 until numKVPairs) { + val byteArr = new Array[Byte](valSize) + ranGen.nextBytes(byteArr) + arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr) + } + arr1 + }.cache() + // Enforce that everything has been calculated and in cache + pairs1.count() + + // scalastyle:off println + println("The Result is", pairs1.groupByKey(numReducers).count()) + // scalastyle:on println + spark.stop() + } +} + 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/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala new file mode 100644 index 0000000000000..d3372749f999e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -0,0 +1,47 @@ +/* + * 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.integrationtest.jobs + +import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper +import org.apache.spark.sql.SparkSession + +// Equivalent to SparkPi except does not stop the Spark Context +// at the end and spins forever, so other things can inspect the +// Spark UI immediately after the fact. +private[spark] object SparkPiWithInfiniteWait { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName("Spark Pi") + .getOrCreate() + val slices = if (args.length > 0) args(0).toInt else 10 + val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow + val count = spark.sparkContext.parallelize(1 until n, slices).map { _ => + PiHelper.helpPi() + }.reduce(_ + _) + // scalastyle:off println + println("Pi is roughly " + 4.0 * count / (n - 1)) + // scalastyle:on println + + // Spin forever to keep the Spark UI active, so other things can inspect the job. + while (true) { + Thread.sleep(600000) + } + } + +} diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml new file mode 100644 index 0000000000000..32db53c670337 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -0,0 +1,389 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.3.1-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests_2.11 + + kubernetes-integration-tests + + jar + Spark Project Kubernetes Integration Tests + + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + test + test-jar + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + test + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + tar.gz + docker-dist + test + + + * + * + + + + + com.google.guava + guava + test + + 18.0 + + + com.spotify + docker-client + test + + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + com.fasterxml.jackson.core + jackson-databind + + + org.glassfish.jersey.core + jersey-client + + + org.glassfish.jersey.core + jersey-common + + + javax.ws.rs + jsr311-api + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-test-spark-jobs + pre-integration-test + + copy + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/integration-tests-spark-jobs + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/integration-tests-spark-jobs-helpers + + + + + + copy-test-spark-jobs-to-docker-dist + pre-integration-test + + copy + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/examples/integration-tests-jars + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/examples/integration-tests-jars + + + + + + unpack-docker-bundle + pre-integration-test + + unpack + + + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + docker-dist + tar.gz + true + ${project.build.directory}/docker/ + + + + + + + + maven-resources-plugin + 3.0.2 + + + copy-integration-test-http-server-dockerfile + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/dockerfiles + + + src/main/docker + true + + + + + + copy-integration-python + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/python + + + ${project.parent.basedir}/python + + ${project.parent.basedir}/python/.egg + ${project.parent.basedir}/python/dist + + + + + + + copy-integration-data + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/data + + + ${project.parent.basedir}/data + true + + + + + + copy-integration-licenses + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/licenses + + + ${project.parent.basedir}/licenses + true + + + + + + copy-integration-examples-jar + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/examples/jars + + + ${project.parent.basedir}/examples/target/scala-2.11/jars + true + + + + + + copy-integration-examples-src + pre-integration-test + + copy-resources + + + ${project.build.directory}/docker/examples/src/main + + + ${project.parent.basedir}/examples/src/main + true + + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.6 + + + create-release-file + pre-integration-test + + run + + + + + + + + + + + com.googlecode.maven-download-plugin + download-maven-plugin + 1.3.0 + + + download-minikube-linux + pre-integration-test + + wget + + + https://storage.googleapis.com/minikube/releases/v0.16.0/minikube-linux-amd64 + ${project.build.directory}/minikube-bin/linux-amd64 + minikube + + + + download-minikube-darwin + pre-integration-test + + wget + + + https://storage.googleapis.com/minikube/releases/v0.16.0/minikube-darwin-amd64 + ${project.build.directory}/minikube-bin/darwin-amd64 + minikube + + + + + + + org.scalatest + scalatest-maven-plugin + + + test + + test + + + + (?<!Suite) + + + + integration-test + integration-test + + test + + + + + + + + + diff --git a/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile b/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile new file mode 100644 index 0000000000000..e26d207cf4397 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile @@ -0,0 +1,21 @@ +# +# 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. +# + +# Simple asset server that can provide the integration test jars over HTTP. +FROM trinitronx/python-simplehttpserver:travis-12 + +ADD examples/integration-tests-jars /var/www diff --git a/resource-managers/kubernetes/integration-tests/src/test/python/pi.py b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py new file mode 100755 index 0000000000000..e3f0c4aeef1b7 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py @@ -0,0 +1,46 @@ +from __future__ import print_function +# +# 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. +# + +import sys +from random import random +from operator import add + +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + """ + Usage: pi [partitions] + """ + spark = SparkSession\ + .builder\ + .appName("PythonPi")\ + .getOrCreate() + + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + n = 100000 * partitions + + def f(_): + x = random() * 2 - 1 + y = random() * 2 - 1 + return 1 if x ** 2 + y ** 2 < 1 else 0 + + count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) + + spark.stop() diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..866126bc3c1c2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties @@ -0,0 +1,31 @@ +# +# 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. +# + +# Set everything to be logged to the file target/integration-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/integration-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from a few verbose libraries. +log4j.logger.com.sun.jersey=WARN +log4j.logger.org.apache.hadoop=WARN +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.mortbay=WARN +log4j.logger.org.spark_project.jetty=WARN 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/kubernetes/integrationtest/KubernetesSuite.scala new file mode 100644 index 0000000000000..c6cd6a74c88d1 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -0,0 +1,374 @@ +/* + * 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.integrationtest + +import java.io.File +import java.nio.file.Paths +import java.util.UUID + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.internal.readiness.Readiness +import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +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.launcher.SparkLauncher +import org.apache.spark.util.Utils + +private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { + import KubernetesSuite._ + private val testBackend = IntegrationTestBackendFactory.getTestBackend() + private val APP_LOCATOR_LABEL = UUID.randomUUID().toString.replaceAll("-", "") + private var kubernetesTestComponents: KubernetesTestComponents = _ + private var sparkConf: SparkConf = _ + private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ + private var staticAssetServerLauncher: StaticAssetServerLauncher = _ + + override def beforeAll(): Unit = { + testBackend.initialize() + kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient) + resourceStagingServerLauncher = new ResourceStagingServerLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) + staticAssetServerLauncher = new StaticAssetServerLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) + } + + override def afterAll(): Unit = { + testBackend.cleanUp() + } + + before { + 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") + kubernetesTestComponents.createNamespace() + } + + after { + kubernetesTestComponents.deleteNamespace() + } + + test("Run PySpark Job on file from SUBMITTER with --py-files") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + launchStagingServer(SSLOptions(), None) + sparkConf + .set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) + .set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + + runPySparkPiAndVerifyCompletion( + PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION, + Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION) + ) + } + + test("Run PySpark Job on file from CONTAINER with spark.jar defined") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf + .set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest")) + .set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest")) + + runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String]) + } + + test("Simple submission test with the resource staging server.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + launchStagingServer(SSLOptions(), None) + runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Enable SSL on the resource staging server") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair( + ipAddress = Minikube.getMinikubeIp, + keyStorePassword = "keyStore", + keyPassword = "key", + trustStorePassword = "trustStore") + sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", + keyStoreAndTrustStore.keyStore.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.trustStore", + keyStoreAndTrustStore.trustStore.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key") + .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore") + launchStagingServer(SSLOptions( + enabled = true, + keyStore = Some(keyStoreAndTrustStore.keyStore), + trustStore = Some(keyStoreAndTrustStore.trustStore), + keyStorePassword = Some("keyStore"), + keyPassword = Some("key"), + trustStorePassword = Some("trustStore")), + None) + runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Use container-local resources without the resource staging server") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Dynamic executor scaling basic test") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + launchStagingServer(SSLOptions(), None) + createShuffleServiceDaemonSet() + + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf.set("spark.dynamicAllocation.enabled", "true") + 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) + sparkConf.set("spark.app.name", "group-by-test") + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), + GROUP_BY_MAIN_CLASS, + Seq("The Result is"), + Array.empty[String], + Seq.empty[String]) + } + + test("Use remote resources without the resource staging server.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() + sparkConf.setJars(Seq( + s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}", + s"$assetServerUri/${HELPER_JAR_FILE.getName}" + )) + runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) + } + + test("Mix remote resources with submitted ones.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + launchStagingServer(SSLOptions(), None) + val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() + sparkConf.setJars(Seq( + SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}" + )) + runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) + } + + test("Use key and certificate PEM files for TLS.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + val keyAndCertificate = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp) + launchStagingServer( + SSLOptions(enabled = true), + Some(keyAndCertificate)) + sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) + .set( + RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath) + runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Use client key and client cert file when requesting executors") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + sparkConf.setJars(Seq( + CONTAINER_LOCAL_MAIN_APP_RESOURCE, + CONTAINER_LOCAL_HELPER_JAR_PATH)) + sparkConf.set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX", + kubernetesTestComponents.clientConfig.getClientKeyFile) + sparkConf.set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX", + kubernetesTestComponents.clientConfig.getClientCertFile) + sparkConf.set( + s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", + kubernetesTestComponents.clientConfig.getCaCertFile) + runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) + } + + test("Added files should be placed in the driver's working directory.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + val testExistenceFileTempDir = Utils.createTempDir(namePrefix = "test-existence-file-temp-dir") + val testExistenceFile = new File(testExistenceFileTempDir, "input.txt") + Files.write(TEST_EXISTENCE_FILE_CONTENTS, testExistenceFile, Charsets.UTF_8) + launchStagingServer(SSLOptions(), None) + sparkConf.set("spark.files", testExistenceFile.getAbsolutePath) + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE), + FILE_EXISTENCE_MAIN_CLASS, + Seq(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."), + Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS), + Seq.empty[String]) + } + + test("Use a very long application name.") { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40) + runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE) + } + + private def launchStagingServer( + resourceStagingServerSslOptions: SSLOptions, keyAndCertPem: Option[KeyAndCertPem]): Unit = { + assume(testBackend.name == MINIKUBE_TEST_BACKEND) + + val resourceStagingServerPort = resourceStagingServerLauncher.launchStagingServer( + resourceStagingServerSslOptions, keyAndCertPem) + val resourceStagingServerUriScheme = if (resourceStagingServerSslOptions.enabled) { + "https" + } else { + "http" + } + sparkConf.set(RESOURCE_STAGING_SERVER_URI, + s"$resourceStagingServerUriScheme://" + + s"${Minikube.getMinikubeIp}:$resourceStagingServerPort") + } + + private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { + runSparkApplicationAndVerifyCompletion( + JavaMainAppResource(appResource), + SPARK_PI_MAIN_CLASS, + Seq("Pi is roughly 3"), + Array.empty[String], + Seq.empty[String]) + } + + private def runPySparkPiAndVerifyCompletion( + appResource: String, otherPyFiles: Seq[String]): Unit = { + runSparkApplicationAndVerifyCompletion( + PythonMainAppResource(appResource), + PYSPARK_PI_MAIN_CLASS, + Seq("Submitting 5 missing tasks from ResultStage", "Pi is roughly 3"), + Array("5"), + otherPyFiles) + } + + private def runSparkApplicationAndVerifyCompletion( + appResource: MainAppResource, + mainClass: String, + expectedLogOnCompletion: Seq[String], + appArgs: Array[String], + otherPyFiles: Seq[String]): Unit = { + val clientArguments = ClientArguments( + mainAppResource = appResource, + mainClass = mainClass, + driverArgs = appArgs, + otherPyFiles = otherPyFiles) + Client.run(sparkConf, clientArguments) + val driverPod = kubernetesTestComponents.kubernetesClient + .pods() + .withLabel("spark-app-locator", APP_LOCATOR_LABEL) + .list() + .getItems + .get(0) + Eventually.eventually(TIMEOUT, INTERVAL) { + expectedLogOnCompletion.foreach { e => + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains(e), "The application did not complete.") + } + } + } + + private def createShuffleServiceDaemonSet(): Unit = { + val ds = kubernetesTestComponents.kubernetesClient.extensions().daemonSets() + .createNew() + .withNewMetadata() + .withName("shuffle") + .endMetadata() + .withNewSpec() + .withNewTemplate() + .withNewMetadata() + .withLabels(Map("app" -> "spark-shuffle-service").asJava) + .endMetadata() + .withNewSpec() + .addNewVolume() + .withName("shuffle-dir") + .withNewHostPath() + .withPath("/tmp") + .endHostPath() + .endVolume() + .addNewContainer() + .withName("shuffle") + .withImage("spark-shuffle:latest") + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName("shuffle-dir") + .withMountPath("/tmp") + .endVolumeMount() + .endContainer() + .endSpec() + .endTemplate() + .endSpec() + .done() + + // wait for daemonset to become available. + Eventually.eventually(TIMEOUT, INTERVAL) { + val pods = kubernetesTestComponents.kubernetesClient.pods() + .withLabel("app", "spark-shuffle-service").list().getItems + + if (pods.size() == 0 || !Readiness.isReady(pods.get(0))) { + throw ShuffleNotReadyException + } + } + } +} + +private[spark] object KubernetesSuite { + val EXAMPLES_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs") + .toFile + .listFiles()(0) + + val HELPER_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs-helpers") + .toFile + .listFiles()(0) + val SUBMITTER_LOCAL_MAIN_APP_RESOURCE = s"file://${EXAMPLES_JAR_FILE.getAbsolutePath}" + val CONTAINER_LOCAL_MAIN_APP_RESOURCE = s"local:///opt/spark/examples/" + + s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" + val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + + 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" + + ".integrationtest.jobs.SparkPiWithInfiniteWait" + val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" + val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = + "local:///opt/spark/examples/src/main/python/pi.py" + 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" + + ".integrationtest.jobs.FileExistenceTest" + val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.GroupByTest" + val TEST_EXISTENCE_FILE_CONTENTS = "contents" + + case object ShuffleNotReadyException extends Exception +} 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/kubernetes/integrationtest/KubernetesTestComponents.scala new file mode 100644 index 0000000000000..0ca1f482269db --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala @@ -0,0 +1,71 @@ +/* + * 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.integrationtest + +import java.util.UUID + +import io.fabric8.kubernetes.client.DefaultKubernetesClient +import org.scalatest.concurrent.Eventually +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ + +private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { + + val namespace = UUID.randomUUID().toString.replaceAll("-", "") + val kubernetesClient = defaultClient.inNamespace(namespace) + val clientConfig = kubernetesClient.getConfiguration + + def createNamespace(): Unit = { + defaultClient.namespaces.createNew() + .withNewMetadata() + .withName(namespace) + .endMetadata() + .done() + } + + def deleteNamespace(): Unit = { + defaultClient.namespaces.withName(namespace).delete() + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val namespaceList = defaultClient + .namespaces() + .list() + .getItems() + .asScala + require(!namespaceList.exists(_.getMetadata.getName == namespace)) + } + } + + def newSparkConf(): SparkConf = { + new SparkConf(true) + .setMaster(s"k8s://${kubernetesClient.getMasterUrl}") + .set(KUBERNETES_NAMESPACE, namespace) + .set(DRIVER_DOCKER_IMAGE, + System.getProperty("spark.docker.test.driverImage", "spark-driver:latest")) + .set(EXECUTOR_DOCKER_IMAGE, + System.getProperty("spark.docker.test.executorImage", "spark-executor:latest")) + .setJars(Seq(KubernetesSuite.HELPER_JAR_FILE.getAbsolutePath)) + .set("spark.executor.memory", "500m") + .set("spark.executor.cores", "1") + .set("spark.executors.instances", "1") + .set("spark.app.name", "spark-test-app") + .set("spark.ui.enabled", "true") + .set("spark.testing", "false") + .set(WAIT_FOR_APP_COMPLETION, false) + } +} 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/kubernetes/integrationtest/ProcessUtils.scala new file mode 100644 index 0000000000000..4008007b72fc4 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala @@ -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. + */ +package org.apache.spark.deploy.kubernetes.integrationtest + +import java.io.{BufferedReader, InputStreamReader} +import java.util.concurrent.TimeUnit + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +object ProcessUtils extends Logging { + /** + * executeProcess is used to run a command and return the output if it + * completes within timeout seconds. + */ + def executeProcess(fullCommand: Array[String], timeout: Long): Seq[String] = { + val pb = new ProcessBuilder().command(fullCommand: _*) + pb.redirectErrorStream(true) + val proc = pb.start() + val outputLines = new ArrayBuffer[String] + + Utils.tryWithResource(new InputStreamReader(proc.getInputStream)) { procOutput => + Utils.tryWithResource(new BufferedReader(procOutput)) { (bufferedOutput: BufferedReader) => + var line: String = null + do { + line = bufferedOutput.readLine() + if (line != null) { + logInfo(line) + outputLines += line + } + } while (line != null) + } + } + assert(proc.waitFor(timeout, TimeUnit.SECONDS), + s"Timed out while executing ${fullCommand.mkString(" ")}") + assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}") + outputLines.toSeq + } +} 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/kubernetes/integrationtest/ResourceStagingServerLauncher.scala new file mode 100644 index 0000000000000..e5e1b1f085f9f --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala @@ -0,0 +1,192 @@ +/* + * 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.integrationtest + +import java.io.{File, StringWriter} +import java.util.Properties + +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, Endpoints, HTTPGetActionBuilder, KeyToPathBuilder, Pod, PodBuilder, SecretBuilder, ServiceBuilder} +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.util.Utils + +/** + * Launches a pod that runs the resource staging server, exposing it over a NodePort. + */ +private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesClient) { + + private val SECRETS_ROOT_DIR = "/mnt/secrets/spark-staging" + private val KEYSTORE_SECRET_KEY = "keyStore" + private val KEYSTORE_FILE = s"$SECRETS_ROOT_DIR/$KEYSTORE_SECRET_KEY" + private val KEY_PEM_SECRET_KEY = "keyPem" + private val CERT_PEM_SECRET_KEY = "certPem" + private val KEY_PEM_FILE = s"$SECRETS_ROOT_DIR/$KEY_PEM_SECRET_KEY" + private val CERT_PEM_FILE = s"$SECRETS_ROOT_DIR/$CERT_PEM_SECRET_KEY" + private val SSL_SECRET_NAME = "resource-staging-server-ssl-secrets" + private val PROPERTIES_FILE_NAME = "staging-server.properties" + private val PROPERTIES_DIR = "/var/data/spark-staging-server" + private val PROPERTIES_FILE_PATH = s"$PROPERTIES_DIR/$PROPERTIES_FILE_NAME" + + // Returns the NodePort the staging server is listening on + def launchStagingServer( + sslOptions: SSLOptions, + keyAndCertPem: Option[KeyAndCertPem] = None): Int = { + val stagingServerProperties = new Properties() + val stagingServerSecret = sslOptions.keyStore.map { keyStore => + val keyStoreBytes = Files.toByteArray(keyStore) + val keyStoreBase64 = BaseEncoding.base64().encode(keyStoreBytes) + Map(KEYSTORE_SECRET_KEY -> keyStoreBase64) + }.orElse { + keyAndCertPem.map { keyAndCert => + val keyPemBytes = Files.toByteArray(keyAndCert.keyPem) + val keyPemBase64 = BaseEncoding.base64().encode(keyPemBytes) + val certPemBytes = Files.toByteArray(keyAndCert.certPem) + val certPemBase64 = BaseEncoding.base64().encode(certPemBytes) + Map(KEY_PEM_SECRET_KEY -> keyPemBase64, CERT_PEM_SECRET_KEY -> certPemBase64) + } + }.map { secretData => + new SecretBuilder() + .withNewMetadata().withName(SSL_SECRET_NAME).endMetadata() + .withData(secretData.asJava) + .build() + } + stagingServerProperties.setProperty( + RESOURCE_STAGING_SERVER_SSL_ENABLED.key, sslOptions.enabled.toString) + sslOptions.keyStorePassword.foreach { password => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", password) + } + sslOptions.keyPassword.foreach { password => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyPassword", password) + } + sslOptions.keyStore.foreach { _ => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyStore", KEYSTORE_FILE) + } + keyAndCertPem.foreach { _ => + stagingServerProperties.setProperty( + RESOURCE_STAGING_SERVER_KEY_PEM.key, KEY_PEM_FILE) + } + keyAndCertPem.foreach { _ => + stagingServerProperties.setProperty( + RESOURCE_STAGING_SERVER_CERT_PEM.key, CERT_PEM_FILE) + } + val propertiesWriter = new StringWriter() + stagingServerProperties.store(propertiesWriter, "Resource staging server properties.") + val stagingServerConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(s"staging-server-properties") + .endMetadata() + .addToData("staging-server", propertiesWriter.toString) + .build() + val probePingHttpGet = new HTTPGetActionBuilder() + .withScheme(if (sslOptions.enabled) "HTTPS" else "HTTP") + .withPath("/api/v0/ping") + .withNewPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .build() + val basePod = new PodBuilder() + .withNewMetadata() + .withName("resource-staging-server") + .addToLabels("resource-staging-server", "staging-server") + .endMetadata() + .withNewSpec() + .addNewVolume() + .withName("staging-server-properties") + .withNewConfigMap() + .withName(stagingServerConfigMap.getMetadata.getName) + .withItems( + new KeyToPathBuilder() + .withKey("staging-server") + .withPath(PROPERTIES_FILE_NAME) + .build()) + .endConfigMap() + .endVolume() + .addNewContainer() + .withName("staging-server-container") + .withImage("spark-resource-staging-server:latest") + .withImagePullPolicy("IfNotPresent") + .withNewReadinessProbe() + .withHttpGet(probePingHttpGet) + .endReadinessProbe() + .addNewVolumeMount() + .withName("staging-server-properties") + .withMountPath(PROPERTIES_DIR) + .endVolumeMount() + .addToArgs(PROPERTIES_FILE_PATH) + .endContainer() + .endSpec() + val withMountedKeyStorePod = stagingServerSecret.map { secret => + basePod.editSpec() + .addNewVolume() + .withName("keystore-volume") + .withNewSecret() + .withSecretName(secret.getMetadata.getName) + .endSecret() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate("staging-server-container")) + .addNewVolumeMount() + .withName("keystore-volume") + .withMountPath(SECRETS_ROOT_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + }.getOrElse(basePod).build() + val stagingServerService = new ServiceBuilder() + .withNewMetadata() + .withName("resource-staging-server") + .endMetadata() + .withNewSpec() + .withType("NodePort") + .addToSelector("resource-staging-server", "staging-server") + .addNewPort() + .withName("staging-server-port") + .withPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .withNewTargetPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .endPort() + .endSpec() + .build() + val stagingServerPodReadyWatcher = new SparkReadinessWatcher[Pod] + val serviceReadyWatcher = new SparkReadinessWatcher[Endpoints] + val allResources = Seq( + stagingServerService, + stagingServerConfigMap, + withMountedKeyStorePod) ++ + stagingServerSecret.toSeq + Utils.tryWithResource(kubernetesClient.pods() + .withName(withMountedKeyStorePod.getMetadata.getName) + .watch(stagingServerPodReadyWatcher)) { _ => + Utils.tryWithResource(kubernetesClient.endpoints() + .withName(stagingServerService.getMetadata.getName) + .watch(serviceReadyWatcher)) { _ => + kubernetesClient.resourceList(allResources: _*).createOrReplace() + stagingServerPodReadyWatcher.waitUntilReady() + serviceReadyWatcher.waitUntilReady() + } + } + kubernetesClient.services().withName(stagingServerService.getMetadata.getName).get() + .getSpec + .getPorts + .get(0) + .getNodePort + } +} 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/kubernetes/integrationtest/SparkReadinessWatcher.scala new file mode 100644 index 0000000000000..20517eb2fc2a6 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala @@ -0,0 +1,41 @@ +/* + * 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.integrationtest + +import java.util.concurrent.TimeUnit + +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model.HasMetadata +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.internal.readiness.Readiness + +private[spark] class SparkReadinessWatcher[T <: HasMetadata] extends Watcher[T] { + + private val signal = SettableFuture.create[Boolean] + + override def eventReceived(action: Action, resource: T): Unit = { + if ((action == Action.MODIFIED || action == Action.ADDED) && + Readiness.isReady(resource)) { + signal.set(true) + } + } + + override def onClose(cause: KubernetesClientException): Unit = {} + + def waitUntilReady(): Boolean = signal.get(30, TimeUnit.SECONDS) +} 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/kubernetes/integrationtest/StaticAssetServerLauncher.scala new file mode 100644 index 0000000000000..6b483769f5254 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.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.deploy.kubernetes.integrationtest + +import io.fabric8.kubernetes.api.model.{HTTPGetActionBuilder, Pod} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.util.Utils + +/** + * Launches a simple HTTP server which provides jars that can be downloaded by Spark applications + * in integration tests. + */ +private[spark] class StaticAssetServerLauncher(kubernetesClient: KubernetesClient) { + + // Returns the HTTP Base URI of the server. + def launchStaticAssetServer(): String = { + val readinessWatcher = new SparkReadinessWatcher[Pod] + val probePingHttpGet = new HTTPGetActionBuilder() + .withNewPort(8080) + .withScheme("HTTP") + .withPath("/") + .build() + Utils.tryWithResource(kubernetesClient + .pods() + .withName("integration-test-static-assets") + .watch(readinessWatcher)) { _ => + val pod = kubernetesClient.pods().createNew() + .withNewMetadata() + .withName("integration-test-static-assets") + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName("static-asset-server-container") + .withImage("spark-integration-test-asset-server:latest") + .withImagePullPolicy("IfNotPresent") + .withNewReadinessProbe() + .withHttpGet(probePingHttpGet) + .endReadinessProbe() + .endContainer() + .endSpec() + .done() + readinessWatcher.waitUntilReady() + val podIP = kubernetesClient.pods().withName(pod.getMetadata.getName).get() + .getStatus + .getPodIP + s"http://$podIP:8080" + } + } +} 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/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala new file mode 100644 index 0000000000000..1ef096be4af02 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala @@ -0,0 +1,40 @@ +/* + * 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.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 + +private[spark] class GCETestBackend(val master: String) extends IntegrationTestBackend { + private var defaultClient: DefaultKubernetesClient = _ + + override def initialize(): Unit = { + var k8ConfBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(resolveK8sMaster(master)) + defaultClient = new DefaultKubernetesClient(k8ConfBuilder.build) + } + + override def getKubernetesClient(): DefaultKubernetesClient = { + defaultClient + } + + override def name(): String = GCE_TEST_BACKEND +} 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/kubernetes/integrationtest/backend/IntegrationTestBackend.scala new file mode 100644 index 0000000000000..c5bc923dd51a6 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala @@ -0,0 +1,39 @@ +/* + * 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.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 + +private[spark] trait IntegrationTestBackend { + def name(): String + def initialize(): Unit + def getKubernetesClient(): DefaultKubernetesClient + def cleanUp(): Unit = {} +} + +private[spark] object IntegrationTestBackendFactory { + def getTestBackend(): IntegrationTestBackend = { + Option(System.getProperty("spark.kubernetes.test.master")) + .map(new GCETestBackend(_)) + .getOrElse(new MinikubeTestBackend()) + } +} 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/kubernetes/integrationtest/backend/minikube/Minikube.scala new file mode 100644 index 0000000000000..7c4b344e8f72b --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala @@ -0,0 +1,125 @@ +/* + * 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.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.internal.Logging +import org.apache.spark.util.Utils + +// TODO support windows +private[spark] object Minikube extends Logging { + private val MINIKUBE_EXECUTABLE_DEST = if (Utils.isMac) { + Paths.get("target", "minikube-bin", "darwin-amd64", "minikube").toFile + } else if (Utils.isWindows) { + throw new IllegalStateException("Executing Minikube based integration tests not yet " + + " available on Windows.") + } else { + Paths.get("target", "minikube-bin", "linux-amd64", "minikube").toFile + } + + private val EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE = "Minikube is not downloaded, expected at " + + s"${MINIKUBE_EXECUTABLE_DEST.getAbsolutePath}" + + private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60 + + def startMinikube(): Unit = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + if (getMinikubeStatus != MinikubeStatus.RUNNING) { + executeMinikube("start", "--memory", "6000", "--cpus", "8") + } else { + logInfo("Minikube is already started.") + } + } + + def getMinikubeIp: String = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + val outputs = executeMinikube("ip") + .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$")) + assert(outputs.size == 1, "Unexpected amount of output from minikube ip") + outputs.head + } + + def getMinikubeStatus: MinikubeStatus.Value = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + val statusString = executeMinikube("status") + .filter(_.contains("minikubeVM: ")) + .head + .replaceFirst("minikubeVM: ", "") + MinikubeStatus.unapply(statusString) + .getOrElse(throw new IllegalStateException(s"Unknown status $statusString")) + } + + def getDockerEnv: Map[String, String] = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + executeMinikube("docker-env", "--shell", "bash") + .filter(_.startsWith("export")) + .map(_.replaceFirst("export ", "").split('=')) + .map(arr => (arr(0), arr(1).replaceAllLiterally("\"", ""))) + .toMap + } + + def deleteMinikube(): Unit = synchronized { + assert(MINIKUBE_EXECUTABLE_DEST.exists, EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE) + if (getMinikubeStatus != MinikubeStatus.DOES_NOT_EXIST) { + executeMinikube("delete") + } else { + logInfo("Minikube was already not running.") + } + } + + def getKubernetesClient: DefaultKubernetesClient = synchronized { + val kubernetesMaster = s"https://${getMinikubeIp}:8443" + val userHome = System.getProperty("user.home") + val kubernetesConf = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(kubernetesMaster) + .withCaCertFile(Paths.get(userHome, ".minikube", "ca.crt").toFile.getAbsolutePath) + .withClientCertFile(Paths.get(userHome, ".minikube", "apiserver.crt").toFile.getAbsolutePath) + .withClientKeyFile(Paths.get(userHome, ".minikube", "apiserver.key").toFile.getAbsolutePath) + .build() + new DefaultKubernetesClient(kubernetesConf) + } + + def executeMinikubeSsh(command: String): Unit = { + executeMinikube("ssh", command) + } + + private def executeMinikube(action: String, args: String*): Seq[String] = { + if (!MINIKUBE_EXECUTABLE_DEST.canExecute) { + if (!MINIKUBE_EXECUTABLE_DEST.setExecutable(true)) { + throw new IllegalStateException("Failed to make the Minikube binary executable.") + } + } + ProcessUtils.executeProcess(Array(MINIKUBE_EXECUTABLE_DEST.getAbsolutePath, action) ++ args, + MINIKUBE_STARTUP_TIMEOUT_SECONDS) + } +} + +private[spark] object MinikubeStatus extends Enumeration { + + val RUNNING = status("Running") + val STOPPED = status("Stopped") + val DOES_NOT_EXIST = status("Does Not Exist") + val SAVED = status("Saved") + + def status(value: String): Value = new Val(nextId, value) + def unapply(s: String): Option[Value] = values.find(s == _.toString) +} 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/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala new file mode 100644 index 0000000000000..461264877edc2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -0,0 +1,45 @@ +/* + * 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.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 + +private[spark] class MinikubeTestBackend extends IntegrationTestBackend { + private var defaultClient: DefaultKubernetesClient = _ + + override def initialize(): Unit = { + Minikube.startMinikube() + new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + defaultClient = Minikube.getKubernetesClient + } + + override def getKubernetesClient(): DefaultKubernetesClient = { + defaultClient + } + + override def cleanUp(): Unit = { + if (!System.getProperty("spark.docker.test.persistMinikube", "false").toBoolean) { + Minikube.deleteMinikube() + } + } + + override def name(): String = MINIKUBE_TEST_BACKEND +} 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/kubernetes/integrationtest/constants.scala new file mode 100644 index 0000000000000..bfded1003fc25 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.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.deploy.kubernetes.integrationtest + +package object constants { + val MINIKUBE_TEST_BACKEND = "minikube" + val GCE_TEST_BACKEND = "gce" +} 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/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala new file mode 100644 index 0000000000000..e240fcf953f8c --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -0,0 +1,107 @@ +/* + * 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.integrationtest.docker + +import java.io.File +import java.net.URI +import java.nio.file.Paths + +import scala.collection.JavaConverters._ + +import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler} +import org.apache.http.client.utils.URIBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Seconds, Span} + +import org.apache.spark.internal.Logging +import org.apache.spark.util.RedirectThread + + + +private[spark] class SparkDockerImageBuilder + (private val dockerEnv: Map[String, String]) extends Logging{ + + private val DOCKER_BUILD_PATH = Paths.get("target", "docker") + // Dockerfile paths must be relative to the build path. + private val BASE_DOCKER_FILE = "dockerfiles/spark-base/Dockerfile" + private val DRIVER_DOCKER_FILE = "dockerfiles/driver/Dockerfile" + private val DRIVERPY_DOCKER_FILE = "dockerfiles/driver-py/Dockerfile" + private val EXECUTOR_DOCKER_FILE = "dockerfiles/executor/Dockerfile" + private val EXECUTORPY_DOCKER_FILE = "dockerfiles/executor-py/Dockerfile" + private val SHUFFLE_SERVICE_DOCKER_FILE = "dockerfiles/shuffle-service/Dockerfile" + private val INIT_CONTAINER_DOCKER_FILE = "dockerfiles/init-container/Dockerfile" + private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-server/Dockerfile" + private val STATIC_ASSET_SERVER_DOCKER_FILE = + "dockerfiles/integration-test-asset-server/Dockerfile" + private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + private val dockerHost = dockerEnv.getOrElse("DOCKER_HOST", + throw new IllegalStateException("DOCKER_HOST env not found.")) + + private val originalDockerUri = URI.create(dockerHost) + private val httpsDockerUri = new URIBuilder() + .setHost(originalDockerUri.getHost) + .setPort(originalDockerUri.getPort) + .setScheme("https") + .build() + + private val dockerCerts = dockerEnv.getOrElse("DOCKER_CERT_PATH", + throw new IllegalStateException("DOCKER_CERT_PATH env not found.")) + + private val dockerClient = new DefaultDockerClient.Builder() + .uri(httpsDockerUri) + .dockerCertificates(DockerCertificates + .builder() + .dockerCertPath(Paths.get(dockerCerts)) + .build().get()) + .build() + + def buildSparkDockerImages(): Unit = { + Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } + // Building Python distribution environment + val pythonExec = sys.env.get("PYSPARK_DRIVER_PYTHON") + .orElse(sys.env.get("PYSPARK_PYTHON")) + .getOrElse("/usr/bin/python") + val builder = new ProcessBuilder( + Seq(pythonExec, "setup.py", "sdist").asJava) + builder.directory(new File(DOCKER_BUILD_PATH.toFile, "python")) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + new RedirectThread(process.getInputStream, System.out, "redirect output").start() + val exitCode = process.waitFor() + if (exitCode != 0) { + logInfo(s"exitCode: $exitCode") + } + buildImage("spark-base", BASE_DOCKER_FILE) + buildImage("spark-driver", DRIVER_DOCKER_FILE) + buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE) + buildImage("spark-executor", EXECUTOR_DOCKER_FILE) + buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE) + buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE) + buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE) + buildImage("spark-init", INIT_CONTAINER_DOCKER_FILE) + buildImage("spark-integration-test-asset-server", STATIC_ASSET_SERVER_DOCKER_FILE) + } + + private def buildImage(name: String, dockerFile: String): Unit = { + dockerClient.build( + DOCKER_BUILD_PATH, + name, + dockerFile, + new LoggingBuildHandler()) + } +} 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/kubernetes/integrationtest/restapis/SparkRestApiV1.scala new file mode 100644 index 0000000000000..7a3b06b1b5e58 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala @@ -0,0 +1,50 @@ +/* + * 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.integrationtest.restapis + +import java.util.{List => JList} +import javax.ws.rs._ +import javax.ws.rs.core.MediaType + +import org.apache.spark.status.api.v1._ + +@Path("/api/v1") +@Consumes(Array(MediaType.APPLICATION_JSON)) +@Produces(Array(MediaType.APPLICATION_JSON)) +trait SparkRestApiV1 { + + @GET + @Path("/applications") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getApplications( + @QueryParam("status") applicationStatuses: JList[ApplicationStatus]): Seq[ApplicationInfo] + + @GET + @Path("applications/{appId}/stages") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getStages( + @PathParam("appId") appId: String, + @QueryParam("status") statuses: JList[StageStatus]): Seq[StageData] + + @GET + @Path("applications/{appId}/executors") + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + def getExecutors(@PathParam("appId") appId: String): Seq[ExecutorSummary] +} diff --git a/resource-managers/kubernetes/integration-tests/test-data/input.txt b/resource-managers/kubernetes/integration-tests/test-data/input.txt new file mode 100644 index 0000000000000..dfe437bdebebc --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/test-data/input.txt @@ -0,0 +1 @@ +Contents diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index c58e0f43b2ac7..160f0aedd2293 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 37e7dccd2e27d..80adae9015935 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index e219cfde12656..41c096ae2b824 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -68,12 +68,12 @@ class StateOperatorProgress private[sql]( * incremented. * @param durationMs The amount of time taken to perform various operations in milliseconds. * @param eventTime Statistics of event time seen in this batch. It may contain the following keys: - * { + * {{{ * "max" -> "2016-12-05T20:54:20.827Z" // maximum event time seen in this trigger * "min" -> "2016-12-05T20:54:20.827Z" // minimum event time seen in this trigger * "avg" -> "2016-12-05T20:54:20.827Z" // average event time seen in this trigger * "watermark" -> "2016-12-05T20:54:20.827Z" // watermark used in this trigger - * } + * }}} * All timestamps are in ISO8601 format, i.e. UTC timestamps. * @param stateOperators Information about operators in the query that store state. * @param sources detailed statistics on data being read from each of the streaming sources. diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 468d758a77884..5ae2130732055 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 7bf4fc0df45e8..d1bbaea15f557 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 06569e6ee2231..72e558d5aae77 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 35d53b30191a5..dc3f922ba0ccb 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 38374b5ae5a3b..f4d5f05a810be 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.0 + 2.1.0-k8s-0.3.1-SNAPSHOT ../pom.xml