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..484fef67dc180 100644 --- a/README.md +++ b/README.md @@ -1,3 +1,41 @@ +# Apache Spark On Kubernetes + +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. + +## Getting Started + +- [Usage guide](docs/running-on-kubernetes.md) 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 + +## Why does this fork exist? + +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. + +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. + +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. + +## Who are we? + +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): + +- Google +- Haiwen +- Hyperpilot +- Intel +- Palantir +- Pepperdata +- Red Hat + +-------------------- + +(original README below) + # Apache Spark Spark is a fast and general cluster computing system for Big Data. It provides diff --git a/assembly/pom.xml b/assembly/pom.xml index aebfd12227751..c1f2c5b29f7e8 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.1.0-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..7d016120e44d7 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.1.0-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 93790979d7b26..718f609178e24 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.1.0-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53cb8dd815d81..d543991cb6a94 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.1.0-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 89bee8567fc74..384ef55b6f8a9 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.1.0-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 7b45b23e9c546..3d8eb2703ed6b 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.1.0-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 9b84f1e0c1dfc..cd53039ed9a47 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.1.0-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/core/pom.xml b/core/pom.xml index bbe07006109ea..23510cb50bcb7 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.1.0-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..002b29d5564e1 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.isPython || args.isR => + printErrorAndExit("Kubernetes does not currently support python or 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,13 @@ object SparkSubmit { } } + if (isKubernetesCluster) { + childMainClass = "org.apache.spark.deploy.kubernetes.Client" + childArgs += args.primaryResource + childArgs += args.mainClass + childArgs ++= args.childArgs + } + // 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/dev/.rat-excludes b/dev/.rat-excludes index 6be1c72bc6cfb..f69567d8f6752 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -103,3 +103,4 @@ org.apache.spark.scheduler.ExternalClusterManager org.apache.spark.deploy.yarn.security.ServiceCredentialProvider spark-warehouse structured-streaming/* +org.apache.spark.deploy.rest.kubernetes.DriverServiceManager diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 6ea319e4362ab..62706b0fffedc 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -175,6 +175,13 @@ echo "Build flags: $@" >> "$DISTDIR/RELEASE" # Copy jars cp "$SPARK_HOME"/assembly/target/scala*/jars/* "$DISTDIR/jars/" +# Copy docker files +mkdir -p "$DISTDIR/dockerfiles/driver" +mkdir -p "$DISTDIR/dockerfiles/executor" +DOCKERFILES_SRC="$SPARK_HOME/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker" +cp "$DOCKERFILES_SRC/driver/Dockerfile" "$DISTDIR/dockerfiles/driver/Dockerfile" +cp "$DOCKERFILES_SRC/executor/Dockerfile" "$DISTDIR/dockerfiles/executor/Dockerfile" + # 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 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/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..c1f3a3ca653b9 --- /dev/null +++ b/docs/running-on-kubernetes.md @@ -0,0 +1,336 @@ +--- +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/). +* You must have appropriate permissions to create and list [pods](https://kubernetes.io/docs/user-guide/pods/), [nodes](https://kubernetes.io/docs/admin/node/) and [services](https://kubernetes.io/docs/user-guide/services/) in your cluster. You can verify that you can list these resources by running `kubectl get nodes`, `kubectl get pods` and `kubectl get svc` which should give you a list of nodes, pods and services (if any) respectively. +* You must [build Spark with Kubernetes support](../resource-managers/kubernetes/README.md#building-spark-with-kubernetes-support) from source. + +## 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-k8s-support-0.1.0-alpha.1
    Spark Executor Imagekubespark/spark-executor:v2.1.0-k8s-support-0.1.0-alpha.1
    + +You may also build these docker images from sources, or customize them as required. Spark distributions include the Docker files for the driver and the executor at +`dockerfiles/driver/Dockerfile` and `dockerfiles/executor/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-driver:latest -f dockerfiles/driver/Dockerfile . + docker build -t registry-host:5000/spark-executor:latest -f dockerfiles/executor/Dockerfile . + docker push registry-host:5000/spark-driver:latest + docker push registry-host:5000/spark-executor:latest + +## 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-k8s-support-0.1.0-alpha.1 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-k8s-support-0.1.0-alpha.1 \ + 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 SSL 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. + +### Specifying input files + +Spark supports specifying JAR paths that are either on the submitting host's disk, or are located on the disk of the +driver and executors. Refer to the [application submission](submitting-applications.html#advanced-dependency-management) +section for details. Note that files specified with the `local://` scheme should be added to the container image of both +the driver and the executors. Files without a scheme or with the scheme `file://` are treated as being on the disk of +the submitting machine, and are uploaded to the driver running in Kubernetes before launching the application. + +### Accessing Kubernetes Clusters + +For details about running on public cloud environments, such as Google Container Engine (GKE), refer to [running Spark in the cloud with Kubernetes](running-on-kubernetes-cloud.md). + +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-k8s-support-0.1.0-alpha.1 \ + --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:v2.1.0-k8s-support-0.1.0-alpha.1 \ + 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. + +## Advanced + +### Setting Up SSL For Submitting the Driver + +When submitting to Kubernetes, a pod is started for the driver, and the pod starts an HTTP server. This HTTP server +receives the driver's configuration, including uploaded driver jars, from the client before starting the application. +Spark supports using SSL to encrypt the traffic in this bootstrapping process. It is recommended to configure this +whenever possible. + +See the [security page](security.html) and [configuration](configuration.html) sections for more information on +configuring SSL; use the prefix `spark.ssl.kubernetes.submit` in configuring the SSL-related fields in the context +of submitting to Kubernetes. For example, to set the trustStore used when the local machine communicates with the driver +pod in starting the application, set `spark.ssl.kubernetes.submit.trustStore`. + +One note about the keyStore is that it can be specified as either a file on the client machine or a file in the +container image's disk. Thus `spark.ssl.kubernetes.submit.keyStore` can be a URI with a scheme of either `file:` +or `local:`. A scheme of `file:` corresponds to the keyStore being located on the client machine; it is mounted onto +the driver container as a [secret volume](https://kubernetes.io/docs/user-guide/secrets/). When the URI has the scheme +`local:`, the file is assumed to already be on the container's disk at the appropriate path. + +### Submission of Local Files through Ingress/External controller + +Kubernetes pods run with their own IP address space. If Spark is run in cluster mode, the driver pod may not be +accessible to the submitter. However, the submitter needs to send local dependencies from its local disk to the driver +pod. + +By default, Spark will place a [Service](https://kubernetes.io/docs/user-guide/services/#type-nodeport) with a NodePort +that is opened on every node. The submission client will then contact the driver at one of the node's +addresses with the appropriate service port. + +There may be cases where the nodes cannot be reached by the submission client. For example, the cluster may +only be reachable through an external load balancer. The user may provide their own external URI for Spark driver +services. To use a your own external URI instead of a node's IP and node port, first set +`spark.kubernetes.driver.serviceManagerType` to `ExternalAnnotation`. A service will be created with the annotation +`spark-job.alpha.apache.org/provideExternalUri`, and this service routes to the driver pod. You will need to run a +separate process that watches the API server for services that are created with this annotation in the application's +namespace (set by `spark.kubernetes.namespace`). The process should determine a URI that routes to this service +(potentially configuring infrastructure to handle the URI behind the scenes), and patch the service to include an +annotation `spark-job.alpha.apache.org/resolvedExternalUri`, which has its value as the external URI that your process +has provided (e.g. `https://example.com:8080/my-job`). + +Note that the URI provided in the annotation needs to route traffic to the appropriate destination on the pod, which has +a empty path portion of the URI. This means the external URI provider will likely need to rewrite the path from the +external URI to the destination on the pod, e.g. https://example.com:8080/spark-app-1/submit will need to route traffic +to https://:/. Note that the paths of these two URLs are different. + +If the above is confusing, keep in mind that this functionality is only necessary if the submitter cannot reach any of +the nodes at the driver's node port. It is recommended to use the default configuration with the node port service +whenever possible. + +### 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.submit.caCertFile(none) + CA cert file for connecting to Kubernetes over SSL. This file should be located on the submitting machine's disk. +
    spark.kubernetes.submit.clientKeyFile(none) + Client key file for authenticating against the Kubernetes API server. This file should be located on the submitting + machine's disk. +
    spark.kubernetes.submit.clientCertFile(none) + Client cert file for authenticating against the Kubernetes API server. This file should be located on the submitting + machine's disk. +
    spark.kubernetes.submit.oauthToken(none) + OAuth token to use when authenticating against the against the Kubernetes API server. Note that unlike the other + authentication options, this should be the exact string value of the token to use for the authentication. +
    spark.kubernetes.submit.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. +
    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.submissionServerMemory256m + The amount of memory to allocate for the driver submission server. +
    spark.kubernetes.driver.memoryOverhead(driverMemory + driverSubmissionServerMemory) * 0.10, with minimum of 384 + 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 size (typically 6-10%). +
    spark.kubernetes.driver.labels(none) + 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) + 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.driverSubmitTimeout60s + Time to wait for the driver pod to start running before aborting its execution. +
    spark.kubernetes.driver.service.exposeUiPortfalse + Whether to expose the driver Web UI port as a service NodePort. Turned off by default because NodePort is a limited + resource. +
    spark.kubernetes.submit.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.report.interval1s + Interval between reports of the current Spark job status in cluster mode. +
    spark.kubernetes.driver.serviceManagerTypeNodePort + A tag indicating which class to use for creating the Kubernetes service and determining its URI for the submission + client. Valid values are currently NodePort and ExternalAnnotation. By default, a service + is created with the NodePort type, and the driver will be contacted at one of the nodes at the port + that the nodes expose for the service. If the nodes cannot be contacted from the submitter's machine, consider + setting this to ExternalAnnotation as described in "Determining the Driver Base URI" above. One may + also include a custom implementation of org.apache.spark.deploy.rest.kubernetes.DriverServiceManager on + the submitter's classpath - spark-submit service loads an instance of that class. To use the custom + implementation, set this value to the custom implementation's return value of + DriverServiceManager#getServiceManagerType(). This method should only be done as a last resort. +
    + +## 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 use a fixed number of executors. Dynamic allocation is not supported. +* 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..e3f7529ecbcec 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.1.0-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 4061c5f089c54..60fa11d8a5043 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 6cfc47ef00e2a..a21ec3f3d7fcb 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 58caf35f65a16..25e34698d831e 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index ed32fc0ec4c18..8058732e72e74 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index a3f3907573f21..fc326931315a3 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 9ae4461db64a2..b023bc49203b2 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index f7276d0bd2197..03cd565f93025 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 52c88150137e3..5e294e6acc006 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 93b49bcf615b6..3654ede0e192a 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index cdfd29e3a9208..f9d61028e906a 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index c6a79aa86bcf0..1ba318d8d39c2 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 3fa28aa81f214..fcc4456ef4a13 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.1.0-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 5c828780600cd..2bef062c0e0b3 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.1.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 1818bc80ea78a..78ca270157dcf 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.1.0-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index d60a633b87699..1f0549f901f47 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.1.0-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..e97743cf1bbd9 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.1.0-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 6dcb44cebb254..24718fd46a3ec 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.1.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 5cf3a7f3e0f5e..384b6af178d43 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.1.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 49f12703c04df..9cfaf6eb65323 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.1.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ @@ -136,6 +136,8 @@ 10.12.1.1 1.8.1 1.6.0 + 8.18.0 + 1.52 9.2.16.v20160414 3.1.0 0.8.0 @@ -222,6 +224,7 @@ 64m 512m 512m + @@ -303,6 +306,32 @@ chill-java ${chill.version} + + + com.netflix.feign + feign-core + ${feign.version} + + + com.netflix.feign + feign-okhttp + ${feign.version} + + + com.netflix.feign + feign-jackson + ${feign.version} + + + com.netflix.feign + feign-jaxrs + ${feign.version} + + + org.bouncycastle + bcpkix-jdk15on + ${bouncycastle.version} + @@ -617,6 +646,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 @@ -2052,7 +2086,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.1.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes_2.11 + jar + Spark Project Kubernetes + + kubernetes + 2.2.1 + + + + + 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.netflix.feign + feign-core + + + com.netflix.feign + feign-okhttp + + + com.netflix.feign + feign-jackson + + + com.netflix.feign + feign-jaxrs + + + javax.ws.rs + jsr311-api + + + + + com.fasterxml.jackson.jaxrs + jackson-jaxrs-json-provider + + + javax.ws.rs + javax.ws.rs-api + + + + com.google.guava + guava + + + + + + + + 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.deploy.rest.kubernetes.DriverServiceManager b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.rest.kubernetes.DriverServiceManager new file mode 100644 index 0000000000000..56203ee38ac99 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.rest.kubernetes.DriverServiceManager @@ -0,0 +1,2 @@ +org.apache.spark.deploy.rest.kubernetes.ExternalSuppliedUrisDriverServiceManager +org.apache.spark.deploy.rest.kubernetes.NodePortUrisDriverServiceManager 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/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala new file mode 100644 index 0000000000000..6f715ebad2d75 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/Client.scala @@ -0,0 +1,740 @@ +/* + * 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.security.SecureRandom +import java.util.ServiceLoader +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import com.google.common.base.Charsets +import com.google.common.io.Files +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{ConfigBuilder => K8SConfigBuilder, DefaultKubernetesClient, KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import org.apache.commons.codec.binary.Base64 +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.rest.{AppResource, ContainerAppResource, KubernetesCreateSubmissionRequest, RemoteAppResource, UploadedAppResource} +import org.apache.spark.deploy.rest.kubernetes._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ShutdownHookManager, Utils} + +private[spark] class Client( + sparkConf: SparkConf, + mainClass: String, + mainAppResource: String, + appArgs: Array[String]) extends Logging { + import Client._ + + private val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + private val master = resolveK8sMaster(sparkConf.get("spark.master")) + + private val launchTime = System.currentTimeMillis + private val appName = sparkConf.getOption("spark.app.name") + .getOrElse("spark") + private val kubernetesAppId = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + private val secretName = s"$SUBMISSION_APP_SECRET_PREFIX-$kubernetesAppId" + private val secretDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId" + private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) + private val uiPort = sparkConf.getInt("spark.ui.port", DEFAULT_UI_PORT) + private val driverSubmitTimeoutSecs = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) + private val driverServiceManagerType = sparkConf.get(DRIVER_SERVICE_MANAGER_TYPE) + private val sparkFiles = sparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + private val sparkJars = sparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + + // Memory settings + private val driverMemoryMb = sparkConf.get(org.apache.spark.internal.config.DRIVER_MEMORY) + private val driverSubmitServerMemoryMb = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_SERVER_MEMORY) + private val driverSubmitServerMemoryString = sparkConf.get( + KUBERNETES_DRIVER_SUBMIT_SERVER_MEMORY.key, + KUBERNETES_DRIVER_SUBMIT_SERVER_MEMORY.defaultValueString) + private val driverContainerMemoryMb = driverMemoryMb + driverSubmitServerMemoryMb + private val memoryOverheadMb = sparkConf + .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverContainerMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + private val driverContainerMemoryWithOverhead = driverContainerMemoryMb + memoryOverheadMb + + private val waitForAppCompletion: Boolean = sparkConf.get(WAIT_FOR_APP_COMPLETION) + + private val secretBase64String = { + val secretBytes = new Array[Byte](128) + SECURE_RANDOM.nextBytes(secretBytes) + Base64.encodeBase64String(secretBytes) + } + + private val serviceAccount = sparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) + private val customAnnotations = sparkConf.get(KUBERNETES_DRIVER_ANNOTATIONS) + + private val kubernetesResourceCleaner = new KubernetesResourceCleaner + + def run(): Unit = { + logInfo(s"Starting application $kubernetesAppId in Kubernetes...") + val submitterLocalFiles = KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkFiles) + val submitterLocalJars = KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkJars) + (submitterLocalFiles ++ submitterLocalJars).foreach { file => + if (!new File(Utils.resolveURI(file).getPath).isFile) { + throw new SparkException(s"File $file does not exist or is a directory.") + } + } + if (KubernetesFileUtils.isUriLocalFile(mainAppResource) && + !new File(Utils.resolveURI(mainAppResource).getPath).isFile) { + throw new SparkException(s"Main app resource file $mainAppResource is not a file or" + + s" is a directory.") + } + val driverServiceManager = getDriverServiceManager + val parsedCustomLabels = parseKeyValuePairs(customLabels, KUBERNETES_DRIVER_LABELS.key, + "labels") + parsedCustomLabels.keys.foreach { key => + require(key != SPARK_APP_ID_LABEL, "Label with key" + + s" $SPARK_APP_ID_LABEL cannot be used in" + + " spark.kubernetes.driver.labels, as it is reserved for Spark's" + + " internal configuration.") + } + val parsedCustomAnnotations = parseKeyValuePairs( + customAnnotations, + KUBERNETES_DRIVER_ANNOTATIONS.key, + "annotations") + var k8ConfBuilder = new K8SConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withNamespace(namespace) + sparkConf.get(KUBERNETES_CA_CERT_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withCaCertFile(f) + } + sparkConf.get(KUBERNETES_CLIENT_KEY_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientKeyFile(f) + } + sparkConf.get(KUBERNETES_CLIENT_CERT_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientCertFile(f) + } + sparkConf.get(KUBERNETES_OAUTH_TOKEN).foreach { token => + k8ConfBuilder = k8ConfBuilder.withOauthToken(token) + // Remove the oauth token from Spark conf so that its doesn't appear in the Spark UI. + sparkConf.set(KUBERNETES_OAUTH_TOKEN, "") + } + + val k8ClientConfig = k8ConfBuilder.build + Utils.tryWithResource(new DefaultKubernetesClient(k8ClientConfig)) { kubernetesClient => + driverServiceManager.start(kubernetesClient, kubernetesAppId, sparkConf) + // start outer watch for status logging of driver pod + // only enable interval logging if in waitForAppCompletion mode + val loggingInterval = if (waitForAppCompletion) sparkConf.get(REPORT_INTERVAL) else 0 + val driverPodCompletedLatch = new CountDownLatch(1) + val loggingWatch = new LoggingPodStatusWatcher(driverPodCompletedLatch, kubernetesAppId, + loggingInterval) + Utils.tryWithResource(kubernetesClient + .pods() + .withName(kubernetesAppId) + .watch(loggingWatch)) { _ => + val resourceCleanShutdownHook = ShutdownHookManager.addShutdownHook(() => + kubernetesResourceCleaner.deleteAllRegisteredResourcesFromKubernetes(kubernetesClient)) + val cleanupServiceManagerHook = ShutdownHookManager.addShutdownHook( + ShutdownHookManager.DEFAULT_SHUTDOWN_PRIORITY)( + () => driverServiceManager.stop()) + // Place the error hook at a higher priority in order for the error hook to run before + // the stop hook. + val serviceManagerErrorHook = ShutdownHookManager.addShutdownHook( + ShutdownHookManager.DEFAULT_SHUTDOWN_PRIORITY + 1)(() => + driverServiceManager.handleSubmissionError( + new SparkException("Submission shutting down early..."))) + try { + val sslConfigurationProvider = new SslConfigurationProvider( + sparkConf, kubernetesAppId, kubernetesClient, kubernetesResourceCleaner) + val submitServerSecret = kubernetesClient.secrets().createNew() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .withData(Map((SUBMISSION_APP_SECRET_NAME, secretBase64String)).asJava) + .withType("Opaque") + .done() + kubernetesResourceCleaner.registerOrUpdateResource(submitServerSecret) + val sslConfiguration = sslConfigurationProvider.getSslConfiguration() + val driverKubernetesSelectors = (Map( + SPARK_DRIVER_LABEL -> kubernetesAppId, + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_APP_NAME_LABEL -> appName) + ++ parsedCustomLabels) + val (driverPod, driverService) = launchDriverKubernetesComponents( + kubernetesClient, + driverServiceManager, + parsedCustomLabels, + parsedCustomAnnotations, + submitServerSecret, + sslConfiguration) + configureOwnerReferences( + kubernetesClient, + submitServerSecret, + sslConfiguration.sslSecrets, + driverPod, + driverService) + submitApplicationToDriverServer( + kubernetesClient, + driverServiceManager, + sslConfiguration, + driverService, + submitterLocalFiles, + submitterLocalJars) + // Now that the application has started, persist the components that were created beyond + // the shutdown hook. We still want to purge the one-time secrets, so do not unregister + // those. + kubernetesResourceCleaner.unregisterResource(driverPod) + kubernetesResourceCleaner.unregisterResource(driverService) + } catch { + case e: Throwable => + driverServiceManager.handleSubmissionError(e) + throw e + } finally { + Utils.tryLogNonFatalError { + kubernetesResourceCleaner.deleteAllRegisteredResourcesFromKubernetes(kubernetesClient) + } + Utils.tryLogNonFatalError { + driverServiceManager.stop() + } + + // Remove the shutdown hooks that would be redundant + Utils.tryLogNonFatalError { + ShutdownHookManager.removeShutdownHook(resourceCleanShutdownHook) + } + Utils.tryLogNonFatalError { + ShutdownHookManager.removeShutdownHook(cleanupServiceManagerHook) + } + Utils.tryLogNonFatalError { + ShutdownHookManager.removeShutdownHook(serviceManagerErrorHook) + } + } + // wait if configured to do so + if (waitForAppCompletion) { + logInfo(s"Waiting for application $kubernetesAppId to finish...") + driverPodCompletedLatch.await() + logInfo(s"Application $kubernetesAppId finished.") + } else { + logInfo(s"Application $kubernetesAppId successfully launched.") + } + } + } + } + + private def submitApplicationToDriverServer( + kubernetesClient: KubernetesClient, + driverServiceManager: DriverServiceManager, + sslConfiguration: SslConfiguration, + driverService: Service, + submitterLocalFiles: Iterable[String], + submitterLocalJars: Iterable[String]): Unit = { + sparkConf.getOption("spark.app.id").foreach { id => + logWarning(s"Warning: Provided app id in spark.app.id as $id will be" + + s" overridden as $kubernetesAppId") + } + sparkConf.set(KUBERNETES_DRIVER_POD_NAME, kubernetesAppId) + sparkConf.set(KUBERNETES_DRIVER_SERVICE_NAME, driverService.getMetadata.getName) + sparkConf.set("spark.app.id", kubernetesAppId) + sparkConf.setIfMissing("spark.app.name", appName) + sparkConf.setIfMissing("spark.driver.port", DEFAULT_DRIVER_PORT.toString) + sparkConf.setIfMissing("spark.blockmanager.port", + DEFAULT_BLOCKMANAGER_PORT.toString) + val driverSubmitter = buildDriverSubmissionClient( + kubernetesClient, + driverServiceManager, + driverService, + sslConfiguration) + // Sanity check to see if the driver submitter is even reachable. + driverSubmitter.ping() + logInfo(s"Submitting local resources to driver pod for application " + + s"$kubernetesAppId ...") + val submitRequest = buildSubmissionRequest(submitterLocalFiles, submitterLocalJars) + driverSubmitter.submitApplication(submitRequest) + logInfo("Successfully submitted local resources and driver configuration to" + + " driver pod.") + // After submitting, adjust the service to only expose the Spark UI + val uiServiceType = if (sparkConf.get(EXPOSE_KUBERNETES_DRIVER_SERVICE_UI_PORT)) "NodePort" + else "ClusterIP" + val uiServicePort = new ServicePortBuilder() + .withName(UI_PORT_NAME) + .withPort(uiPort) + .withNewTargetPort(uiPort) + .build() + val resolvedService = kubernetesClient.services().withName(kubernetesAppId).edit() + .editSpec() + .withType(uiServiceType) + .withPorts(uiServicePort) + .endSpec() + .done() + kubernetesResourceCleaner.registerOrUpdateResource(resolvedService) + logInfo("Finished submitting application to Kubernetes.") + } + + private def launchDriverKubernetesComponents( + kubernetesClient: KubernetesClient, + driverServiceManager: DriverServiceManager, + customLabels: Map[String, String], + customAnnotations: Map[String, String], + submitServerSecret: Secret, + sslConfiguration: SslConfiguration): (Pod, Service) = { + val driverKubernetesSelectors = (Map( + SPARK_DRIVER_LABEL -> kubernetesAppId, + SPARK_APP_ID_LABEL -> kubernetesAppId, + SPARK_APP_NAME_LABEL -> appName) + ++ customLabels) + val endpointsReadyFuture = SettableFuture.create[Endpoints] + val endpointsReadyWatcher = new DriverEndpointsReadyWatcher(endpointsReadyFuture) + val serviceReadyFuture = SettableFuture.create[Service] + val serviceReadyWatcher = new DriverServiceReadyWatcher(serviceReadyFuture) + val podReadyFuture = SettableFuture.create[Pod] + val podWatcher = new DriverPodReadyWatcher(podReadyFuture) + Utils.tryWithResource(kubernetesClient + .pods() + .withName(kubernetesAppId) + .watch(podWatcher)) { _ => + Utils.tryWithResource(kubernetesClient + .services() + .withName(kubernetesAppId) + .watch(serviceReadyWatcher)) { _ => + Utils.tryWithResource(kubernetesClient + .endpoints() + .withName(kubernetesAppId) + .watch(endpointsReadyWatcher)) { _ => + val serviceTemplate = createDriverServiceTemplate(driverKubernetesSelectors) + val driverService = kubernetesClient.services().create( + driverServiceManager.customizeDriverService(serviceTemplate).build()) + kubernetesResourceCleaner.registerOrUpdateResource(driverService) + val driverPod = createDriverPod( + kubernetesClient, + driverKubernetesSelectors, + customAnnotations, + submitServerSecret, + sslConfiguration) + waitForReadyKubernetesComponents(kubernetesClient, endpointsReadyFuture, + serviceReadyFuture, podReadyFuture) + (driverPod, driverService) + } + } + } + } + + /** + * Sets the owner reference for all the kubernetes components to link to the driver pod. + * + * @return The driver service after it has been adjusted to reflect the new owner + * reference. + */ + private def configureOwnerReferences( + kubernetesClient: KubernetesClient, + submitServerSecret: Secret, + sslSecrets: Array[Secret], + driverPod: Pod, + driverService: Service): Service = { + val driverPodOwnerRef = new OwnerReferenceBuilder() + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withController(true) + .build() + sslSecrets.foreach(secret => { + val updatedSecret = kubernetesClient.secrets().withName(secret.getMetadata.getName).edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + kubernetesResourceCleaner.registerOrUpdateResource(updatedSecret) + }) + val updatedSubmitServerSecret = kubernetesClient + .secrets() + .withName(submitServerSecret.getMetadata.getName) + .edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + kubernetesResourceCleaner.registerOrUpdateResource(updatedSubmitServerSecret) + val updatedService = kubernetesClient + .services() + .withName(driverService.getMetadata.getName) + .edit() + .editMetadata() + .addToOwnerReferences(driverPodOwnerRef) + .endMetadata() + .done() + kubernetesResourceCleaner.registerOrUpdateResource(updatedService) + updatedService + } + + private def waitForReadyKubernetesComponents( + kubernetesClient: KubernetesClient, + endpointsReadyFuture: SettableFuture[Endpoints], + serviceReadyFuture: SettableFuture[Service], + podReadyFuture: SettableFuture[Pod]) = { + try { + podReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver pod successfully created in Kubernetes cluster.") + } catch { + case e: Throwable => + val finalErrorMessage: String = buildSubmitFailedErrorMessage(kubernetesClient, e) + logError(finalErrorMessage, e) + throw new SparkException(finalErrorMessage, e) + } + try { + serviceReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver service created successfully in Kubernetes.") + } catch { + case e: Throwable => + throw new SparkException(s"The driver service was not ready" + + s" in $driverSubmitTimeoutSecs seconds.", e) + } + try { + endpointsReadyFuture.get(driverSubmitTimeoutSecs, TimeUnit.SECONDS) + logInfo("Driver endpoints ready to receive application submission") + } catch { + case e: Throwable => + throw new SparkException(s"The driver service endpoint was not ready" + + s" in $driverSubmitTimeoutSecs seconds.", e) + } + } + + private def createDriverPod( + kubernetesClient: KubernetesClient, + driverKubernetesSelectors: Map[String, String], + customAnnotations: Map[String, String], + submitServerSecret: Secret, + sslConfiguration: SslConfiguration): Pod = { + val containerPorts = buildContainerPorts() + val probePingHttpGet = new HTTPGetActionBuilder() + .withScheme(if (sslConfiguration.sslOptions.enabled) "HTTPS" else "HTTP") + .withPath("/v1/submissions/ping") + .withNewPort(SUBMISSION_SERVER_PORT_NAME) + .build() + val driverMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryMb}M") + .build() + val driverMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${driverContainerMemoryWithOverhead}M") + .build() + val driverPod = kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors.asJava) + .withAnnotations(customAnnotations.asJava) + .endMetadata() + .withNewSpec() + .withRestartPolicy("Never") + .addNewVolume() + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(submitServerSecret.getMetadata.getName) + .endSecret() + .endVolume() + .addToVolumes(sslConfiguration.sslPodVolumes: _*) + .withServiceAccount(serviceAccount) + .addNewContainer() + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName(SUBMISSION_APP_SECRET_VOLUME_NAME) + .withMountPath(secretDirectory) + .withReadOnly(true) + .endVolumeMount() + .addToVolumeMounts(sslConfiguration.sslPodVolumeMounts: _*) + .addNewEnv() + .withName(ENV_SUBMISSION_SECRET_LOCATION) + .withValue(s"$secretDirectory/$SUBMISSION_APP_SECRET_NAME") + .endEnv() + .addNewEnv() + .withName(ENV_SUBMISSION_SERVER_PORT) + .withValue(SUBMISSION_SERVER_PORT.toString) + .endEnv() + // Note that SPARK_DRIVER_MEMORY only affects the REST server via spark-class. + .addNewEnv() + .withName(ENV_DRIVER_MEMORY) + .withValue(driverSubmitServerMemoryString) + .endEnv() + .addToEnv(sslConfiguration.sslPodEnvVars: _*) + .withNewResources() + .addToRequests("memory", driverMemoryQuantity) + .addToLimits("memory", driverMemoryLimitQuantity) + .endResources() + .withPorts(containerPorts.asJava) + .withNewReadinessProbe().withHttpGet(probePingHttpGet).endReadinessProbe() + .endContainer() + .endSpec() + .done() + kubernetesResourceCleaner.registerOrUpdateResource(driverPod) + driverPod + } + + private def createDriverServiceTemplate(driverKubernetesSelectors: Map[String, String]) + : ServiceBuilder = { + val driverSubmissionServicePort = new ServicePortBuilder() + .withName(SUBMISSION_SERVER_PORT_NAME) + .withPort(SUBMISSION_SERVER_PORT) + .withNewTargetPort(SUBMISSION_SERVER_PORT) + .build() + new ServiceBuilder() + .withNewMetadata() + .withName(kubernetesAppId) + .withLabels(driverKubernetesSelectors.asJava) + .endMetadata() + .withNewSpec() + .withSelector(driverKubernetesSelectors.asJava) + .withPorts(driverSubmissionServicePort) + .endSpec() + } + + private class DriverPodReadyWatcher(resolvedDriverPod: SettableFuture[Pod]) extends Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + if ((action == Action.ADDED || action == Action.MODIFIED) + && pod.getStatus.getPhase == "Running" + && !resolvedDriverPod.isDone) { + pod.getStatus + .getContainerStatuses + .asScala + .find(status => + status.getName == DRIVER_CONTAINER_NAME && status.getReady) + .foreach { _ => resolvedDriverPod.set(pod) } + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver pod readiness watch closed.", cause) + } + } + + private class DriverEndpointsReadyWatcher(resolvedDriverEndpoints: SettableFuture[Endpoints]) + extends Watcher[Endpoints] { + override def eventReceived(action: Action, endpoints: Endpoints): Unit = { + if ((action == Action.ADDED) || (action == Action.MODIFIED) + && endpoints.getSubsets.asScala.nonEmpty + && endpoints.getSubsets.asScala.exists(_.getAddresses.asScala.nonEmpty) + && !resolvedDriverEndpoints.isDone) { + resolvedDriverEndpoints.set(endpoints) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver endpoints readiness watch closed.", cause) + } + } + + private class DriverServiceReadyWatcher(resolvedDriverService: SettableFuture[Service]) + extends Watcher[Service] { + override def eventReceived(action: Action, service: Service): Unit = { + if ((action == Action.ADDED) || (action == Action.MODIFIED) + && !resolvedDriverService.isDone) { + resolvedDriverService.set(service) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("Driver service readiness watch closed.", cause) + } + } + + private def buildSubmitFailedErrorMessage( + kubernetesClient: KubernetesClient, + e: Throwable): String = { + val driverPod = try { + kubernetesClient.pods().withName(kubernetesAppId).get() + } catch { + case throwable: Throwable => + logError(s"Timed out while waiting $driverSubmitTimeoutSecs seconds for the" + + " driver pod to start, but an error occurred while fetching the driver" + + " pod's details.", throwable) + throw new SparkException(s"Timed out while waiting $driverSubmitTimeoutSecs" + + " seconds for the driver pod to start. Unfortunately, in attempting to fetch" + + " the latest state of the pod, another error was thrown. Check the logs for" + + " the error that was thrown in looking up the driver pod.", e) + } + val topLevelMessage = s"The driver pod with name ${driverPod.getMetadata.getName}" + + s" in namespace ${driverPod.getMetadata.getNamespace} was not ready in" + + s" $driverSubmitTimeoutSecs seconds." + val podStatusPhase = if (driverPod.getStatus.getPhase != null) { + s"Latest phase from the pod is: ${driverPod.getStatus.getPhase}" + } else { + "The pod had no final phase." + } + val podStatusMessage = if (driverPod.getStatus.getMessage != null) { + s"Latest message from the pod is: ${driverPod.getStatus.getMessage}" + } else { + "The pod had no final message." + } + val failedDriverContainerStatusString = driverPod.getStatus + .getContainerStatuses + .asScala + .find(_.getName == DRIVER_CONTAINER_NAME) + .map(status => { + val lastState = status.getState + if (lastState.getRunning != null) { + "Driver container last state: Running\n" + + s"Driver container started at: ${lastState.getRunning.getStartedAt}" + } else if (lastState.getWaiting != null) { + "Driver container last state: Waiting\n" + + s"Driver container wait reason: ${lastState.getWaiting.getReason}\n" + + s"Driver container message: ${lastState.getWaiting.getMessage}\n" + } else if (lastState.getTerminated != null) { + "Driver container last state: Terminated\n" + + s"Driver container started at: ${lastState.getTerminated.getStartedAt}\n" + + s"Driver container finished at: ${lastState.getTerminated.getFinishedAt}\n" + + s"Driver container exit reason: ${lastState.getTerminated.getReason}\n" + + s"Driver container exit code: ${lastState.getTerminated.getExitCode}\n" + + s"Driver container message: ${lastState.getTerminated.getMessage}" + } else { + "Driver container last state: Unknown" + } + }).getOrElse("The driver container wasn't found in the pod; expected to find" + + s" container with name $DRIVER_CONTAINER_NAME") + s"$topLevelMessage\n" + + s"$podStatusPhase\n" + + s"$podStatusMessage\n\n$failedDriverContainerStatusString" + } + + private def buildContainerPorts(): Seq[ContainerPort] = { + Seq((DRIVER_PORT_NAME, sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT)), + (BLOCK_MANAGER_PORT_NAME, + sparkConf.getInt("spark.blockManager.port", DEFAULT_BLOCKMANAGER_PORT)), + (SUBMISSION_SERVER_PORT_NAME, SUBMISSION_SERVER_PORT), + (UI_PORT_NAME, uiPort)).map(port => new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build()) + } + + private def buildSubmissionRequest( + submitterLocalFiles: Iterable[String], + submitterLocalJars: Iterable[String]): KubernetesCreateSubmissionRequest = { + val mainResourceUri = Utils.resolveURI(mainAppResource) + val resolvedAppResource: AppResource = Option(mainResourceUri.getScheme) + .getOrElse("file") match { + case "file" => + val appFile = new File(mainResourceUri.getPath) + val fileBytes = Files.toByteArray(appFile) + val fileBase64 = Base64.encodeBase64String(fileBytes) + UploadedAppResource(resourceBase64Contents = fileBase64, name = appFile.getName) + case "local" => ContainerAppResource(mainAppResource) + case other => RemoteAppResource(other) + } + val uploadFilesBase64Contents = CompressionUtils.createTarGzip(submitterLocalFiles.map( + Utils.resolveURI(_).getPath)) + val uploadJarsBase64Contents = CompressionUtils.createTarGzip(submitterLocalJars.map( + Utils.resolveURI(_).getPath)) + KubernetesCreateSubmissionRequest( + appResource = resolvedAppResource, + mainClass = mainClass, + appArgs = appArgs, + secret = secretBase64String, + sparkProperties = sparkConf.getAll.toMap, + uploadedJarsBase64Contents = uploadJarsBase64Contents, + uploadedFilesBase64Contents = uploadFilesBase64Contents) + } + + private def buildDriverSubmissionClient( + kubernetesClient: KubernetesClient, + driverServiceManager: DriverServiceManager, + service: Service, + sslConfiguration: SslConfiguration): KubernetesSparkRestApi = { + val serviceUris = driverServiceManager.getDriverServiceSubmissionServerUris(service) + require(serviceUris.nonEmpty, "No uris found to contact the driver!") + HttpClientUtil.createClient[KubernetesSparkRestApi]( + uris = serviceUris, + maxRetriesPerServer = 10, + sslSocketFactory = sslConfiguration + .driverSubmitClientSslContext + .getSocketFactory, + trustContext = sslConfiguration + .driverSubmitClientTrustManager + .orNull, + connectTimeoutMillis = 5000) + } + + private 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]) + } + + private def getDriverServiceManager: DriverServiceManager = { + val driverServiceManagerLoader = ServiceLoader.load(classOf[DriverServiceManager]) + val matchingServiceManagers = driverServiceManagerLoader + .iterator() + .asScala + .filter(_.getServiceManagerType == driverServiceManagerType) + .toList + require(matchingServiceManagers.nonEmpty, + s"No driver service manager found matching type $driverServiceManagerType") + require(matchingServiceManagers.size == 1, "Multiple service managers found" + + s" matching type $driverServiceManagerType, got: " + + matchingServiceManagers.map(_.getClass).toList.mkString(",")) + matchingServiceManagers.head + } +} + +private[spark] object Client extends Logging { + + private[spark] val SECURE_RANDOM = new SecureRandom() + + def main(args: Array[String]): Unit = { + require(args.length >= 2, s"Too few arguments. Usage: ${getClass.getName} " + + s" []") + val mainAppResource = args(0) + val mainClass = args(1) + val appArgs = args.drop(2) + val sparkConf = new SparkConf(true) + new Client( + mainAppResource = mainAppResource, + mainClass = mainClass, + sparkConf = sparkConf, + appArgs = appArgs).run() + } + + 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/KubernetesClientBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.scala new file mode 100644 index 0000000000000..89369b30694ee --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesClientBuilder.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 + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient} + +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] object KubernetesClientBuilder { + private val API_SERVER_TOKEN = new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH) + private val CA_CERT_FILE = new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH) + + /** + * Creates a {@link KubernetesClient}, expecting to be from + * within the context of a pod. When doing so, credentials files + * are picked up from canonical locations, as they are injected + * into the pod's disk space. + */ + def buildFromWithinPod( + kubernetesNamespace: String): DefaultKubernetesClient = { + var clientConfigBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(KUBERNETES_MASTER_INTERNAL_URL) + .withNamespace(kubernetesNamespace) + + if (CA_CERT_FILE.isFile) { + clientConfigBuilder = clientConfigBuilder.withCaCertFile(CA_CERT_FILE.getAbsolutePath) + } + + if (API_SERVER_TOKEN.isFile) { + clientConfigBuilder = clientConfigBuilder.withOauthToken( + Files.toString(API_SERVER_TOKEN, Charsets.UTF_8)) + } + new DefaultKubernetesClient(clientConfigBuilder.build) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesResourceCleaner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesResourceCleaner.scala new file mode 100644 index 0000000000000..6329bb1359516 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesResourceCleaner.scala @@ -0,0 +1,53 @@ +/* + * 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.HasMetadata +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.mutable + +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +private[spark] class KubernetesResourceCleaner extends Logging { + + private val resources = mutable.HashMap.empty[(String, String), HasMetadata] + + // Synchronized because deleteAllRegisteredResourcesFromKubernetes may be called from a + // shutdown hook + def registerOrUpdateResource(resource: HasMetadata): Unit = synchronized { + resources.put((resource.getMetadata.getName, resource.getKind), resource) + } + + def unregisterResource(resource: HasMetadata): Unit = synchronized { + resources.remove((resource.getMetadata.getName, resource.getKind)) + } + + def deleteAllRegisteredResourcesFromKubernetes(kubernetesClient: KubernetesClient): Unit = { + synchronized { + val resourceCount = resources.size + logInfo(s"Deleting ${resourceCount} registered Kubernetes resources...") + resources.values.foreach { resource => + Utils.tryLogNonFatalError { + kubernetesClient.resource(resource).delete() + } + } + resources.clear() + logInfo(s"Deleted ${resourceCount} registered Kubernetes resources.") + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala new file mode 100644 index 0000000000000..17c3db8331ac4 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/LoggingPodStatusWatcher.scala @@ -0,0 +1,128 @@ +/* + * 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.{CountDownLatch, Executors, TimeUnit} + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + +import org.apache.spark.internal.Logging + +/** + * 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 podCompletedFuture a CountDownLatch that is set to true when the watched pod finishes + * @param appId + * @param interval ms between each state request. If set to 0 or a negative number, the periodic + * logging will be disabled. + */ +private[kubernetes] class LoggingPodStatusWatcher(podCompletedFuture: CountDownLatch, + appId: String, + interval: Long) + extends Watcher[Pod] with Logging { + + // start timer for periodic logging + private val scheduler = Executors.newScheduledThreadPool(1) + private val logRunnable: Runnable = new Runnable { + override def run() = logShortStatus() + } + if (interval > 0) { + scheduler.scheduleWithFixedDelay(logRunnable, 0, interval, TimeUnit.MILLISECONDS) + } + + private var pod: Option[Pod] = Option.empty + private def phase: String = pod.map(_.getStatus().getPhase()).getOrElse("unknown") + private def status: String = pod.map(_.getStatus().getContainerStatuses().toString()) + .getOrElse("unknown") + + 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 = { + + 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) + ) + + // Use more loggable format if value is null or empty + details.map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" + }.mkString("") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SslConfigurationProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SslConfigurationProvider.scala new file mode 100644 index 0000000000000..4c031fcba91ab --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SslConfigurationProvider.scala @@ -0,0 +1,203 @@ +/* + * 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.FileInputStream +import java.security.{KeyStore, SecureRandom} +import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.{EnvVar, EnvVarBuilder, Secret, Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException, SSLOptions} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Utils + +private[spark] case class SslConfiguration( + sslOptions: SSLOptions, + isKeyStoreLocalFile: Boolean, + sslPodEnvVars: Array[EnvVar], + sslPodVolumes: Array[Volume], + sslPodVolumeMounts: Array[VolumeMount], + sslSecrets: Array[Secret], + driverSubmitClientTrustManager: Option[X509TrustManager], + driverSubmitClientSslContext: SSLContext) + +private[spark] class SslConfigurationProvider( + sparkConf: SparkConf, + kubernetesAppId: String, + kubernetesClient: KubernetesClient, + kubernetesResourceCleaner: KubernetesResourceCleaner) { + private val SECURE_RANDOM = new SecureRandom() + private val sslSecretsName = s"$SUBMISSION_SSL_SECRETS_PREFIX-$kubernetesAppId" + private val sslSecretsDirectory = s"$DRIVER_CONTAINER_SECRETS_BASE_DIR/$kubernetesAppId-ssl" + + def getSslConfiguration(): SslConfiguration = { + val (driverSubmitSslOptions, isKeyStoreLocalFile) = parseDriverSubmitSslOptions() + if (driverSubmitSslOptions.enabled) { + val sslSecretsMap = mutable.HashMap[String, String]() + val sslEnvs = mutable.Buffer[EnvVar]() + val secrets = mutable.Buffer[Secret]() + driverSubmitSslOptions.keyStore.foreach(store => { + val resolvedKeyStoreFile = if (isKeyStoreLocalFile) { + if (!store.isFile) { + throw new SparkException(s"KeyStore specified at $store is not a file or" + + s" does not exist.") + } + val keyStoreBytes = Files.toByteArray(store) + val keyStoreBase64 = BaseEncoding.base64().encode(keyStoreBytes) + sslSecretsMap += (SUBMISSION_SSL_KEYSTORE_SECRET_NAME -> keyStoreBase64) + s"$sslSecretsDirectory/$SUBMISSION_SSL_KEYSTORE_SECRET_NAME" + } else { + store.getAbsolutePath + } + sslEnvs += new EnvVarBuilder() + .withName(ENV_SUBMISSION_KEYSTORE_FILE) + .withValue(resolvedKeyStoreFile) + .build() + }) + driverSubmitSslOptions.keyStorePassword.foreach(password => { + val passwordBase64 = BaseEncoding.base64().encode(password.getBytes(Charsets.UTF_8)) + sslSecretsMap += (SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME -> passwordBase64) + sslEnvs += new EnvVarBuilder() + .withName(ENV_SUBMISSION_KEYSTORE_PASSWORD_FILE) + .withValue(s"$sslSecretsDirectory/$SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME") + .build() + }) + driverSubmitSslOptions.keyPassword.foreach(password => { + val passwordBase64 = BaseEncoding.base64().encode(password.getBytes(Charsets.UTF_8)) + sslSecretsMap += (SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME -> passwordBase64) + sslEnvs += new EnvVarBuilder() + .withName(ENV_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE) + .withValue(s"$sslSecretsDirectory/$SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME") + .build() + }) + driverSubmitSslOptions.keyStoreType.foreach(storeType => { + sslEnvs += new EnvVarBuilder() + .withName(ENV_SUBMISSION_KEYSTORE_TYPE) + .withValue(storeType) + .build() + }) + sslEnvs += new EnvVarBuilder() + .withName(ENV_SUBMISSION_USE_SSL) + .withValue("true") + .build() + val sslVolume = new VolumeBuilder() + .withName(SUBMISSION_SSL_SECRETS_VOLUME_NAME) + .withNewSecret() + .withSecretName(sslSecretsName) + .endSecret() + .build() + val sslVolumeMount = new VolumeMountBuilder() + .withName(SUBMISSION_SSL_SECRETS_VOLUME_NAME) + .withReadOnly(true) + .withMountPath(sslSecretsDirectory) + .build() + val sslSecrets = kubernetesClient.secrets().createNew() + .withNewMetadata() + .withName(sslSecretsName) + .endMetadata() + .withData(sslSecretsMap.asJava) + .withType("Opaque") + .done() + kubernetesResourceCleaner.registerOrUpdateResource(sslSecrets) + secrets += sslSecrets + val (driverSubmitClientTrustManager, driverSubmitClientSslContext) = + buildSslConnectionConfiguration(driverSubmitSslOptions) + SslConfiguration( + driverSubmitSslOptions, + isKeyStoreLocalFile, + sslEnvs.toArray, + Array(sslVolume), + Array(sslVolumeMount), + secrets.toArray, + driverSubmitClientTrustManager, + driverSubmitClientSslContext) + } else { + SslConfiguration( + driverSubmitSslOptions, + isKeyStoreLocalFile, + Array[EnvVar](), + Array[Volume](), + Array[VolumeMount](), + Array[Secret](), + None, + SSLContext.getDefault) + } + } + + private def parseDriverSubmitSslOptions(): (SSLOptions, Boolean) = { + val maybeKeyStore = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_KEYSTORE) + val resolvedSparkConf = sparkConf.clone() + val (isLocalKeyStore, resolvedKeyStore) = maybeKeyStore.map(keyStore => { + val keyStoreURI = Utils.resolveURI(keyStore) + val isProvidedKeyStoreLocal = keyStoreURI.getScheme match { + case "file" | null => true + case "local" => false + case _ => throw new SparkException(s"Invalid KeyStore URI $keyStore; keyStore URI" + + " for submit server must have scheme file:// or local:// (no scheme defaults" + + " to file://)") + } + (isProvidedKeyStoreLocal, Option.apply(keyStoreURI.getPath)) + }).getOrElse((false, Option.empty[String])) + resolvedKeyStore.foreach { + resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_KEYSTORE, _) + } + sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE).foreach { trustStore => + val trustStoreURI = Utils.resolveURI(trustStore) + trustStoreURI.getScheme match { + case "file" | null => + resolvedSparkConf.set(KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE, trustStoreURI.getPath) + case _ => throw new SparkException(s"Invalid trustStore URI $trustStore; trustStore URI" + + " for submit server must have no scheme, or scheme file://") + } + } + val securityManager = new SparkSecurityManager(resolvedSparkConf) + (securityManager.getSSLOptions(KUBERNETES_SUBMIT_SSL_NAMESPACE), isLocalKeyStore) + } + + private def buildSslConnectionConfiguration(driverSubmitSslOptions: SSLOptions): + (Option[X509TrustManager], SSLContext) = { + driverSubmitSslOptions.trustStore.map(trustStoreFile => { + val trustManagerFactory = TrustManagerFactory.getInstance( + TrustManagerFactory.getDefaultAlgorithm) + val trustStore = KeyStore.getInstance( + driverSubmitSslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)) + if (!trustStoreFile.isFile) { + throw new SparkException(s"TrustStore file at ${trustStoreFile.getAbsolutePath}" + + s" does not exist or is not a file.") + } + Utils.tryWithResource(new FileInputStream(trustStoreFile)) { trustStoreStream => + driverSubmitSslOptions.trustStorePassword match { + case Some(password) => + trustStore.load(trustStoreStream, password.toCharArray) + case None => trustStore.load(trustStoreStream, null) + } + } + trustManagerFactory.init(trustStore) + val trustManagers = trustManagerFactory.getTrustManagers + val sslContext = SSLContext.getInstance("TLSv1.2") + sslContext.init(null, trustManagers, SECURE_RANDOM) + (Option.apply(trustManagers(0).asInstanceOf[X509TrustManager]), sslContext) + }).getOrElse((Option.empty[X509TrustManager], SSLContext.getDefault)) + } +} 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..0c4269080335f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -0,0 +1,259 @@ +/* + * 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.rest.kubernetes.NodePortUrisDriverServiceManager +import org.apache.spark.internal.config.ConfigBuilder +import org.apache.spark.network.util.ByteUnit + +package object config { + + 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. + """.stripMargin) + .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. + """.stripMargin) + .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. + """.stripMargin) + .stringConf + .createWithDefault(s"spark-executor:$sparkVersion") + + private[spark] val KUBERNETES_CA_CERT_FILE = + ConfigBuilder("spark.kubernetes.submit.caCertFile") + .doc(""" + | CA cert file for connecting to Kubernetes over SSL. This + | file should be located on the submitting machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_CLIENT_KEY_FILE = + ConfigBuilder("spark.kubernetes.submit.clientKeyFile") + .doc(""" + | Client key file for authenticating against the Kubernetes + | API server. This file should be located on the submitting + | machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_CLIENT_CERT_FILE = + ConfigBuilder("spark.kubernetes.submit.clientCertFile") + .doc(""" + | Client cert file for authenticating against the + | Kubernetes API server. This file should be located on + | the submitting machine's disk. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_OAUTH_TOKEN = + ConfigBuilder("spark.kubernetes.submit.oauthToken") + .doc(""" + | OAuth token to use when authenticating against the + | against the Kubernetes API server. Note that unlike + | the other authentication options, this should be the + | exact string value of the token to use for the + | authentication. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_SERVICE_ACCOUNT_NAME = + ConfigBuilder("spark.kubernetes.submit.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. + """.stripMargin) + .stringConf + .createWithDefault("default") + + // 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%). + """.stripMargin) + .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%). + """.stripMargin) + .bytesConf(ByteUnit.MiB) + .createOptional + + 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. + """.stripMargin) + .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. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SUBMIT_TIMEOUT = + ConfigBuilder("spark.kubernetes.driverSubmitTimeout") + .doc(""" + | Time to wait for the driver process to start running + | before aborting its execution. + """.stripMargin) + .timeConf(TimeUnit.SECONDS) + .createWithDefault(60L) + + private[spark] val KUBERNETES_DRIVER_SUBMIT_KEYSTORE = + ConfigBuilder("spark.ssl.kubernetes.submit.keyStore") + .doc(""" + | KeyStore file for the driver submission server listening + | on SSL. Can be pre-mounted on the driver container + | or uploaded from the submitting client. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SUBMIT_TRUSTSTORE = + ConfigBuilder("spark.ssl.kubernetes.submit.trustStore") + .doc(""" + | TrustStore containing certificates for communicating + | to the driver submission server over SSL. + """.stripMargin) + .stringConf + .createOptional + + private[spark] val DRIVER_SUBMIT_SSL_ENABLED = + ConfigBuilder("spark.ssl.kubernetes.submit.enabled") + .doc(""" + | Whether or not to use SSL when sending the + | application dependencies to the driver pod. + | + """.stripMargin) + .booleanConf + .createWithDefault(false) + + private[spark] val KUBERNETES_DRIVER_SERVICE_NAME = + ConfigBuilder("spark.kubernetes.driver.service.name") + .doc(""" + | Kubernetes service that exposes the driver pod + | for external access. + """.stripMargin) + .internal() + .stringConf + .createOptional + + private[spark] val KUBERNETES_DRIVER_SUBMIT_SERVER_MEMORY = + ConfigBuilder("spark.kubernetes.driver.submissionServerMemory") + .doc(""" + | The amount of memory to allocate for the driver submission server. + """.stripMargin) + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("256m") + + private[spark] val EXPOSE_KUBERNETES_DRIVER_SERVICE_UI_PORT = + ConfigBuilder("spark.kubernetes.driver.service.exposeUiPort") + .doc(""" + | Whether to expose the driver Web UI port as a service NodePort. Turned off by default + | because NodePort is a limited resource. Use alternatives such as Ingress if possible. + """.stripMargin) + .booleanConf + .createWithDefault(false) + + private[spark] val KUBERNETES_DRIVER_POD_NAME = + ConfigBuilder("spark.kubernetes.driver.pod.name") + .doc(""" + | Name of the driver pod. + """.stripMargin) + .internal() + .stringConf + .createOptional + + private[spark] val DRIVER_SERVICE_MANAGER_TYPE = + ConfigBuilder("spark.kubernetes.driver.serviceManagerType") + .doc(s""" + | A tag indicating which class to use for creating the + | Kubernetes service and determining its URI for the submission + | client. + """.stripMargin) + .stringConf + .createWithDefault(NodePortUrisDriverServiceManager.TYPE) + + private[spark] val WAIT_FOR_APP_COMPLETION = + ConfigBuilder("spark.kubernetes.submit.waitAppCompletion") + .doc( + """ + | In cluster mode, whether to wait for the application to finish before exiting the + | launcher process. + """.stripMargin) + .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. + """.stripMargin) + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("1s") +} 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..4af065758e674 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +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-id" + private[spark] val SPARK_APP_NAME_LABEL = "spark-app-name" + private[spark] val SPARK_EXECUTOR_ID_LABEL = "spark-exec-id" + + // Secrets + private[spark] val DRIVER_CONTAINER_SECRETS_BASE_DIR = "/var/run/secrets/spark-submission" + private[spark] val SUBMISSION_APP_SECRET_NAME = "spark-submission-server-secret" + private[spark] val SUBMISSION_APP_SECRET_PREFIX = "spark-submission-server-secret" + private[spark] val SUBMISSION_APP_SECRET_VOLUME_NAME = "spark-submission-secret-volume" + private[spark] val SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME = + "spark-submission-server-key-password" + private[spark] val SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME = + "spark-submission-server-keystore-password" + private[spark] val SUBMISSION_SSL_KEYSTORE_SECRET_NAME = "spark-submission-server-keystore" + private[spark] val SUBMISSION_SSL_SECRETS_PREFIX = "spark-submission-server-ssl" + private[spark] val SUBMISSION_SSL_SECRETS_VOLUME_NAME = "spark-submission-server-ssl-secrets" + + // 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 UI_PORT_NAME = "spark-ui-port" + private[spark] val SUBMISSION_SERVER_PORT_NAME = "submit-server" + private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private[spark] val DRIVER_PORT_NAME = "driver" + private[spark] val EXECUTOR_PORT_NAME = "executor" + + // Environment Variables + private[spark] val ENV_SUBMISSION_SECRET_LOCATION = "SPARK_SUBMISSION_SECRET_LOCATION" + private[spark] val ENV_SUBMISSION_SERVER_PORT = "SPARK_SUBMISSION_SERVER_PORT" + private[spark] val ENV_SUBMISSION_KEYSTORE_FILE = "SPARK_SUBMISSION_KEYSTORE_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_PASSWORD_FILE = + "SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE = + "SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE" + private[spark] val ENV_SUBMISSION_KEYSTORE_TYPE = "SPARK_SUBMISSION_KEYSTORE_TYPE" + private[spark] val ENV_SUBMISSION_USE_SSL = "SPARK_SUBMISSION_USE_SSL" + 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_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" + + // Annotation keys + private[spark] val ANNOTATION_PROVIDE_EXTERNAL_URI = + "spark-job.alpha.apache.org/provideExternalUri" + private[spark] val ANNOTATION_RESOLVED_EXTERNAL_URI = + "spark-job.alpha.apache.org/resolvedExternalUri" + + // Miscellaneous + private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + private[spark] val KUBERNETES_SUBMIT_SSL_NAMESPACE = "kubernetes.submit" + 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/rest/KubernetesRestProtocolMessages.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala new file mode 100644 index 0000000000000..0d2d1a1c6f5e3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/KubernetesRestProtocolMessages.scala @@ -0,0 +1,65 @@ +/* + * 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 + +import com.fasterxml.jackson.annotation.{JsonSubTypes, JsonTypeInfo} + +import org.apache.spark.SPARK_VERSION + +case class KubernetesCreateSubmissionRequest( + appResource: AppResource, + mainClass: String, + appArgs: Array[String], + sparkProperties: Map[String, String], + secret: String, + uploadedJarsBase64Contents: TarGzippedData, + uploadedFilesBase64Contents: TarGzippedData) extends SubmitRestProtocolRequest { + message = "create" + clientSparkVersion = SPARK_VERSION +} + +case class TarGzippedData( + dataBase64: String, + blockSize: Int = 10240, + recordSize: Int = 512, + encoding: String +) + +@JsonTypeInfo( + use = JsonTypeInfo.Id.NAME, + include = JsonTypeInfo.As.PROPERTY, + property = "type") +@JsonSubTypes(value = Array( + new JsonSubTypes.Type(value = classOf[UploadedAppResource], name = "UploadedAppResource"), + new JsonSubTypes.Type(value = classOf[ContainerAppResource], name = "ContainerLocalAppResource"), + new JsonSubTypes.Type(value = classOf[RemoteAppResource], name = "RemoteAppResource"))) +abstract class AppResource + +case class UploadedAppResource( + resourceBase64Contents: String, + name: String = "spark-app-resource") extends AppResource + +case class ContainerAppResource(resourcePath: String) extends AppResource + +case class RemoteAppResource(resource: String) extends AppResource + +class PingResponse extends SubmitRestProtocolResponse { + val text = "pong" + message = "pong" + serverSparkVersion = SPARK_VERSION +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.scala new file mode 100644 index 0000000000000..7204cb874aaec --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/CompressionUtils.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.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import com.google.common.io.Files +import org.apache.commons.codec.binary.Base64 +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.deploy.rest.TarGzippedData +import org.apache.spark.internal.Logging +import org.apache.spark.util.{ByteBufferOutputStream, 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 + + /** + * Compresses all of the given paths into a gzipped-tar archive, returning the compressed data in + * memory as an instance of {@link TarGzippedData}. The files are taken without consideration to + * their original folder structure, and are added to the tar archive in a flat hierarchy. + * Directories are not allowed, and duplicate file names are de-duplicated by appending a numeric + * suffix to the file name, before the file extension. For example, if paths a/b.txt and b/b.txt + * were provided, then the files added to the tar archive would be b.txt and b-1.txt. + * @param paths A list of file paths to be archived + * @return An in-memory representation of the compressed data. + */ + def createTarGzip(paths: Iterable[String]): TarGzippedData = { + val compressedBytesStream = Utils.tryWithResource(new ByteBufferOutputStream()) { raw => + Utils.tryWithResource(new GZIPOutputStream(raw)) { 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(file, resolvedFileName) + tarStream.putArchiveEntry(tarEntry) + Utils.tryWithResource(new FileInputStream(file)) { fileInput => + IOUtils.copy(fileInput, tarStream) + } + tarStream.closeArchiveEntry() + } + } + } + raw + } + val compressedAsBase64 = Base64.encodeBase64String(compressedBytesStream.toByteBuffer.array) + TarGzippedData( + dataBase64 = compressedAsBase64, + blockSize = BLOCK_SIZE, + recordSize = RECORD_SIZE, + encoding = ENCODING + ) + } + + /** + * Decompresses the provided tar archive to a directory. + * @param compressedData In-memory representation of the compressed data, ideally created via + * {@link createTarGzip}. + * @param rootOutputDir Directory to write the output files to. All files from the tarball + * are written here in a flat hierarchy. + * @return List of file paths for each file that was unpacked from the archive. + */ + def unpackAndWriteCompressedFiles( + compressedData: TarGzippedData, + rootOutputDir: File): Seq[String] = { + val paths = mutable.Buffer.empty[String] + val compressedBytes = Base64.decodeBase64(compressedData.dataBase64) + if (!rootOutputDir.exists) { + if (!rootOutputDir.mkdirs) { + throw new IllegalStateException(s"Failed to create output directory for unpacking" + + s" files at ${rootOutputDir.getAbsolutePath}") + } + } else if (rootOutputDir.isFile) { + throw new IllegalArgumentException(s"Root dir for writing decompressed files: " + + s"${rootOutputDir.getAbsolutePath} exists and is not a directory.") + } + Utils.tryWithResource(new ByteArrayInputStream(compressedBytes)) { compressedBytesStream => + Utils.tryWithResource(new GZIPInputStream(compressedBytesStream)) { gzipped => + Utils.tryWithResource(new TarArchiveInputStream( + gzipped, + compressedData.blockSize, + compressedData.recordSize, + compressedData.encoding)) { tarInputStream => + var nextTarEntry = tarInputStream.getNextTarEntry + while (nextTarEntry != null) { + val outputFile = new File(rootOutputDir, nextTarEntry.getName) + Utils.tryWithResource(new FileOutputStream(outputFile)) { fileOutputStream => + IOUtils.copy(tarInputStream, fileOutputStream) + } + paths += outputFile.getAbsolutePath + nextTarEntry = tarInputStream.getNextTarEntry + } + } + } + } + paths.toSeq + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/DriverServiceManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/DriverServiceManager.scala new file mode 100644 index 0000000000000..d92c0247e2a35 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/DriverServiceManager.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.rest.kubernetes + +import io.fabric8.kubernetes.api.model.{Service, ServiceBuilder} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.SparkConf + +/** + * Implementations of this interface are responsible for exposing the driver pod by: + * - Creating a Kubernetes Service that is backed by the driver pod, and + * - Providing one or more URIs that the service can be reached at from the submission client. + * + * In general, one should not need to implement custom variants of this interface. Consider + * if the built-in service managers, NodePort and ExternalAnnotation, suit your needs first. + * + * This API is in an alpha state and may break without notice. + */ +trait DriverServiceManager { + + protected var kubernetesClient: KubernetesClient = _ + protected var serviceName: String = _ + protected var sparkConf: SparkConf = _ + + /** + * The tag that identifies this service manager type. This service manager will be loaded + * only if the Spark configuration spark.kubernetes.driver.serviceManagerType matches this + * value. + */ + def getServiceManagerType: String + + final def start( + kubernetesClient: KubernetesClient, + serviceName: String, + sparkConf: SparkConf): Unit = { + this.kubernetesClient = kubernetesClient + this.serviceName = serviceName + this.sparkConf = sparkConf + onStart(kubernetesClient, serviceName, sparkConf) + } + + /** + * Guaranteed to be called before {@link createDriverService} or + * {@link getDriverServiceSubmissionServerUris} is called. + */ + protected def onStart( + kubernetesClient: KubernetesClient, + serviceName: String, + sparkConf: SparkConf): Unit = {} + + /** + * Customize the driver service that overlays on the driver pod. + * + * Implementations are expected to take the service template and adjust it + * according to the particular needs of how the Service will be accessed by + * URIs provided in {@link getDriverServiceSubmissionServerUris}. + * + * @param driverServiceTemplate Base settings for the driver service. + * @return The same ServiceBuilder object with any required customizations. + */ + def customizeDriverService(driverServiceTemplate: ServiceBuilder): ServiceBuilder + + /** + * Return the set of URIs that can be used to reach the submission server that + * is running on the driver pod. + */ + def getDriverServiceSubmissionServerUris(driverService: Service): Set[String] + + /** + * Called when the Spark application failed to start. Allows the service + * manager to clean up any state it may have created that should not be persisted + * in the case of an unsuccessful launch. Note that stop() is still called + * regardless if this method is called. + */ + def handleSubmissionError(cause: Throwable): Unit = {} + + final def stop(): Unit = onStop() + + /** + * Perform any cleanup of this service manager. + * the super implementation. + */ + protected def onStop(): Unit = {} +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ExternalSuppliedUrisDriverServiceManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ExternalSuppliedUrisDriverServiceManager.scala new file mode 100644 index 0000000000000..257571b5a9d3e --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ExternalSuppliedUrisDriverServiceManager.scala @@ -0,0 +1,105 @@ +/* + * 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.TimeUnit + +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model.{Service, ServiceBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils + +/** + * Creates the service with an annotation that is expected to be detected by another process + * which the user provides and is not built in this project. When the external process detects + * the creation of the service with the appropriate annotation, it is expected to populate the + * value of a second annotation that is the URI of the driver submission server. + */ +private[spark] class ExternalSuppliedUrisDriverServiceManager + extends DriverServiceManager with Logging { + + private val externalUriFuture = SettableFuture.create[String] + private var externalUriSetWatch: Option[Watch] = None + + override def onStart( + kubernetesClient: KubernetesClient, + serviceName: String, + sparkConf: SparkConf): Unit = { + externalUriSetWatch = Some(kubernetesClient + .services() + .withName(serviceName) + .watch(new ExternalUriSetWatcher(externalUriFuture))) + } + + override def getServiceManagerType: String = ExternalSuppliedUrisDriverServiceManager.TYPE + + override def customizeDriverService(driverServiceTemplate: ServiceBuilder): ServiceBuilder = { + require(serviceName != null, "Service name was null; was start() called?") + driverServiceTemplate + .editMetadata() + .addToAnnotations(ANNOTATION_PROVIDE_EXTERNAL_URI, "true") + .endMetadata() + .editSpec() + .withType("ClusterIP") + .endSpec() + } + + override def getDriverServiceSubmissionServerUris(driverService: Service): Set[String] = { + val timeoutSeconds = sparkConf.get(KUBERNETES_DRIVER_SUBMIT_TIMEOUT) + require(externalUriSetWatch.isDefined, "The watch that listens for the provision of" + + " the external URI was not started; was start() called?") + Set(externalUriFuture.get(timeoutSeconds, TimeUnit.SECONDS)) + } + + override def onStop(): Unit = { + Utils.tryLogNonFatalError { + externalUriSetWatch.foreach(_.close()) + externalUriSetWatch = None + } + } +} + +private[spark] object ExternalSuppliedUrisDriverServiceManager { + val TYPE = "ExternalAnnotation" +} + +private[spark] class ExternalUriSetWatcher(externalUriFuture: SettableFuture[String]) + extends Watcher[Service] with Logging { + + override def eventReceived(action: Action, service: Service): Unit = { + if (action == Action.MODIFIED && !externalUriFuture.isDone) { + service + .getMetadata + .getAnnotations + .asScala + .get(ANNOTATION_RESOLVED_EXTERNAL_URI) + .foreach(externalUriFuture.set) + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logDebug("External URI set watcher closed.", cause) + } +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala new file mode 100644 index 0000000000000..576f7058f20ee --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/HttpClientUtil.scala @@ -0,0 +1,69 @@ +/* + * 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 javax.net.ssl.{SSLContext, SSLSocketFactory, X509TrustManager} + +import com.fasterxml.jackson.databind.{DeserializationFeature, ObjectMapper} +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import feign.{Client, Feign, Request, Response} +import feign.Request.Options +import feign.jackson.{JacksonDecoder, JacksonEncoder} +import feign.jaxrs.JAXRSContract +import okhttp3.OkHttpClient +import scala.reflect.ClassTag + +import org.apache.spark.status.api.v1.JacksonMessageWriter + +private[spark] object HttpClientUtil { + + def createClient[T: ClassTag]( + uris: Set[String], + maxRetriesPerServer: Int = 1, + sslSocketFactory: SSLSocketFactory = SSLContext.getDefault.getSocketFactory, + trustContext: X509TrustManager = null, + readTimeoutMillis: Int = 20000, + connectTimeoutMillis: Int = 20000): T = { + var httpClientBuilder = new OkHttpClient.Builder() + Option.apply(trustContext).foreach(context => { + httpClientBuilder = httpClientBuilder.sslSocketFactory(sslSocketFactory, context) + }) + val objectMapper = new ObjectMapper() + .registerModule(new DefaultScalaModule) + .setDateFormat(JacksonMessageWriter.makeISODateFormat) + objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false) + val target = new MultiServerFeignTarget[T](uris.toSeq, maxRetriesPerServer) + val baseHttpClient = new feign.okhttp.OkHttpClient(httpClientBuilder.build()) + val resetTargetHttpClient = new Client { + override def execute(request: Request, options: Options): Response = { + val response = baseHttpClient.execute(request, options) + if (response.status() / 100 == 2) { + target.reset() + } + response + } + } + Feign.builder() + .client(resetTargetHttpClient) + .contract(new JAXRSContract) + .encoder(new JacksonEncoder(objectMapper)) + .decoder(new JacksonDecoder(objectMapper)) + .options(new Options(connectTimeoutMillis, readTimeoutMillis)) + .retryer(target) + .target(target) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala new file mode 100644 index 0000000000000..f30be1535f81c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesFileUtils.scala @@ -0,0 +1,44 @@ +/* + * 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 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 getOnlySubmitterLocalFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, _ == "file") + } + + def isUriLocalFile(uri: String): Boolean = { + Option(Utils.resolveURI(uri).getScheme).getOrElse("file") == "file" + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala new file mode 100644 index 0000000000000..18eb9b7a12ca6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestApi.scala @@ -0,0 +1,38 @@ +/* + * 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 javax.ws.rs.{Consumes, GET, Path, POST, Produces} +import javax.ws.rs.core.MediaType + +import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KubernetesCreateSubmissionRequest, PingResponse} + +@Path("/v1/submissions/") +trait KubernetesSparkRestApi { + + @POST + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + @Path("/create") + def submitApplication(request: KubernetesCreateSubmissionRequest): CreateSubmissionResponse + + @GET + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.APPLICATION_JSON)) + @Path("/ping") + def ping(): PingResponse +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala new file mode 100644 index 0000000000000..5952acc0d5916 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkRestServer.scala @@ -0,0 +1,382 @@ +/* + * 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, StringReader} +import java.net.URI +import java.nio.file.Paths +import java.util.concurrent.CountDownLatch +import java.util.concurrent.atomic.AtomicInteger +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, ByteStreams, Files} +import org.apache.commons.codec.binary.Base64 +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SSLOptions} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.rest._ +import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} + +private case class KubernetesSparkRestServerArguments( + host: Option[String] = None, + port: Option[Int] = None, + useSsl: Boolean = false, + secretFile: Option[String] = None, + keyStoreFile: Option[String] = None, + keyStorePasswordFile: Option[String] = None, + keyStoreType: Option[String] = None, + keyPasswordFile: Option[String] = None) { + def validate(): KubernetesSparkRestServerArguments = { + require(host.isDefined, "Hostname not set via --hostname.") + require(port.isDefined, "Port not set via --port") + require(secretFile.isDefined, "Secret file not set via --secret-file") + this + } +} + +private object KubernetesSparkRestServerArguments { + def fromArgsArray(inputArgs: Array[String]): KubernetesSparkRestServerArguments = { + var args = inputArgs.toList + var resolvedArguments = KubernetesSparkRestServerArguments() + while (args.nonEmpty) { + resolvedArguments = args match { + case "--hostname" :: value :: tail => + args = tail + resolvedArguments.copy(host = Some(value)) + case "--port" :: value :: tail => + args = tail + resolvedArguments.copy(port = Some(value.toInt)) + case "--secret-file" :: value :: tail => + args = tail + resolvedArguments.copy(secretFile = Some(value)) + case "--use-ssl" :: value :: tail => + args = tail + resolvedArguments.copy(useSsl = value.toBoolean) + case "--keystore-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyStoreFile = Some(value)) + case "--keystore-password-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyStorePasswordFile = Some(value)) + case "--keystore-type" :: value :: tail => + args = tail + resolvedArguments.copy(keyStoreType = Some(value)) + case "--keystore-key-password-file" :: value :: tail => + args = tail + resolvedArguments.copy(keyPasswordFile = Some(value)) + // TODO polish usage message + case Nil => resolvedArguments + case unknown => throw new IllegalStateException(s"Unknown argument(s) found: $unknown") + } + } + resolvedArguments.validate() + } +} + +/** + * Runs in the driver pod and receives a request to run an application. Note that + * unlike the submission rest server in standalone mode, this server is expected + * to be used to run one application only, and then shut down once that application + * is complete. + */ +private[spark] class KubernetesSparkRestServer( + host: String, + port: Int, + conf: SparkConf, + expectedApplicationSecret: Array[Byte], + shutdownLock: CountDownLatch, + exitCode: AtomicInteger, + sslOptions: SSLOptions = new SSLOptions) + extends RestSubmissionServer(host, port, conf, sslOptions) { + + private val SERVLET_LOCK = new Object + private val javaExecutable = s"${System.getenv("JAVA_HOME")}/bin/java" + private val sparkHome = System.getenv("SPARK_HOME") + private val securityManager = new SecurityManager(conf) + override protected lazy val contextToServlet = Map[String, RestServlet]( + s"$baseContext/create/*" -> submitRequestServlet, + s"$baseContext/ping/*" -> pingServlet) + + private val pingServlet = new PingServlet + override protected val submitRequestServlet: SubmitRequestServlet + = new KubernetesSubmitRequestServlet + // TODO + override protected val statusRequestServlet: StatusRequestServlet = null + override protected val killRequestServlet: KillRequestServlet = null + + private class PingServlet extends RestServlet { + protected override def doGet( + request: HttpServletRequest, + response: HttpServletResponse): Unit = { + sendResponse(new PingResponse, response) + } + } + + private class KubernetesSubmitRequestServlet extends SubmitRequestServlet { + + private val waitForProcessCompleteExecutor = ThreadUtils + .newDaemonSingleThreadExecutor("wait-for-spark-app-complete") + private var startedApplication = false + + // TODO validating the secret should be done as part of a header of the request. + // Instead here we have to specify the secret in the body. + override protected def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + SERVLET_LOCK.synchronized { + if (startedApplication) { + throw new IllegalStateException("Application has already been submitted.") + } else { + requestMessage match { + case KubernetesCreateSubmissionRequest( + appResource, + mainClass, + appArgs, + sparkProperties, + secret, + uploadedJars, + uploadedFiles) => + val decodedSecret = Base64.decodeBase64(secret) + if (!expectedApplicationSecret.sameElements(decodedSecret)) { + responseServlet.setStatus(HttpServletResponse.SC_UNAUTHORIZED) + handleError("Unauthorized to submit application.") + } else { + val tempDir = Utils.createTempDir() + val resolvedAppResource = resolveAppResource(appResource, tempDir) + val writtenJars = writeUploadedJars(uploadedJars, tempDir) + val writtenFiles = writeUploadedFiles(uploadedFiles) + val resolvedSparkProperties = new mutable.HashMap[String, String] + resolvedSparkProperties ++= sparkProperties + val originalJars = sparkProperties.get("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty) + + // The driver at this point has handed us the value of spark.jars verbatim as + // specified in spark-submit. At this point, remove all jars that were local + // to the submitting user's disk, and replace them with the paths that were + // written to disk above. + val onlyContainerLocalOrRemoteJars = KubernetesFileUtils + .getNonSubmitterLocalFiles(originalJars) + val resolvedJars = (writtenJars ++ + onlyContainerLocalOrRemoteJars ++ + Array(resolvedAppResource.sparkJarPath)).toSet + if (resolvedJars.nonEmpty) { + resolvedSparkProperties("spark.jars") = resolvedJars.mkString(",") + } else { + resolvedSparkProperties.remove("spark.jars") + } + + // Determining the driver classpath is similar. It's the combination of: + // - Jars written from uploads + // - Jars in (spark.jars + mainAppResource) that has a "local" prefix + // - spark.driver.extraClasspath + // - Spark core jars from the installation + val sparkCoreJars = new File(sparkHome, "jars").listFiles().map(_.getAbsolutePath) + val driverExtraClasspath = sparkProperties + .get("spark.driver.extraClassPath") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val onlyContainerLocalJars = KubernetesFileUtils + .getOnlyContainerLocalFiles(originalJars) + val driverClasspath = driverExtraClasspath ++ + Seq(resolvedAppResource.localPath) ++ + writtenJars ++ + onlyContainerLocalJars ++ + sparkCoreJars + + // Resolve spark.files similarly to spark.jars. + val originalFiles = sparkProperties.get("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val onlyContainerLocalOrRemoteFiles = KubernetesFileUtils + .getNonSubmitterLocalFiles(originalFiles) + val resolvedFiles = writtenFiles ++ onlyContainerLocalOrRemoteFiles + if (resolvedFiles.nonEmpty) { + resolvedSparkProperties("spark.files") = resolvedFiles.mkString(",") + } else { + resolvedSparkProperties.remove("spark.files") + } + + val command = new ArrayBuffer[String] + command += javaExecutable + command += "-cp" + command += s"${driverClasspath.mkString(":")}" + for (prop <- resolvedSparkProperties) { + command += s"-D${prop._1}=${prop._2}" + } + val driverMemory = resolvedSparkProperties.getOrElse("spark.driver.memory", "1g") + command += s"-Xms$driverMemory" + command += s"-Xmx$driverMemory" + val extraJavaOpts = resolvedSparkProperties.get("spark.driver.extraJavaOptions") + .map(Utils.splitCommandString) + .getOrElse(Seq.empty) + command ++= extraJavaOpts + command += mainClass + command ++= appArgs + val pb = new ProcessBuilder(command: _*).inheritIO() + val process = pb.start() + ShutdownHookManager.addShutdownHook(() => { + logInfo("Received stop command, shutting down the running Spark application...") + process.destroy() + shutdownLock.countDown() + }) + waitForProcessCompleteExecutor.submit(new Runnable { + override def run(): Unit = { + // set the REST service's exit code to the exit code of the driver subprocess + exitCode.set(process.waitFor) + SERVLET_LOCK.synchronized { + logInfo("Spark application complete. Shutting down submission server...") + KubernetesSparkRestServer.this.stop + shutdownLock.countDown() + } + } + }) + startedApplication = true + val response = new CreateSubmissionResponse + response.success = true + response.submissionId = null + response.message = "success" + response.serverSparkVersion = sparkVersion + response + } + case unexpected => + responseServlet.setStatus(HttpServletResponse.SC_BAD_REQUEST) + handleError(s"Received message of unexpected type ${unexpected.messageType}.") + } + } + } + } + + private def writeUploadedJars(jars: TarGzippedData, rootTempDir: File): + Seq[String] = { + val resolvedDirectory = new File(rootTempDir, "jars") + if (!resolvedDirectory.mkdir()) { + throw new IllegalStateException(s"Failed to create jars dir at " + + resolvedDirectory.getAbsolutePath) + } + CompressionUtils.unpackAndWriteCompressedFiles(jars, resolvedDirectory) + } + + private def writeUploadedFiles(files: TarGzippedData): Seq[String] = { + val workingDir = Paths.get("").toFile.getAbsoluteFile + CompressionUtils.unpackAndWriteCompressedFiles(files, workingDir) + } + + + /** + * Retrieve the path on the driver container where the main app resource is, and what value it + * ought to have in the spark.jars property. The two may be different because for non-local + * dependencies, we have to fetch the resource (if it is not "local") but still want to use + * the full URI in spark.jars. + */ + private def resolveAppResource(appResource: AppResource, tempDir: File): + ResolvedAppResource = { + appResource match { + case UploadedAppResource(resourceContentsBase64, resourceName) => + val resourceFile = new File(tempDir, resourceName) + val resourceFilePath = resourceFile.getAbsolutePath + if (resourceFile.createNewFile()) { + Utils.tryWithResource(new StringReader(resourceContentsBase64)) { reader => + Utils.tryWithResource(new FileOutputStream(resourceFile)) { os => + Utils.tryWithResource(BaseEncoding.base64().decodingStream(reader)) { + decodingStream => + ByteStreams.copy(decodingStream, os) + } + } + } + ResolvedAppResource(resourceFile.getAbsolutePath, resourceFile.getAbsolutePath) + } else { + throw new IllegalStateException(s"Failed to write main app resource file" + + s" to $resourceFilePath") + } + case ContainerAppResource(resource) => + ResolvedAppResource(Utils.resolveURI(resource).getPath, resource) + case RemoteAppResource(resource) => + Utils.fetchFile(resource, tempDir, conf, + securityManager, SparkHadoopUtil.get.newConfiguration(conf), + System.currentTimeMillis(), useCache = false) + val fileName = Utils.decodeFileNameInURI(URI.create(resource)) + val downloadedFile = new File(tempDir, fileName) + val downloadedFilePath = downloadedFile.getAbsolutePath + if (!downloadedFile.isFile) { + throw new IllegalStateException(s"Main app resource is not a file or" + + s" does not exist at $downloadedFilePath") + } + ResolvedAppResource(downloadedFilePath, resource) + } + } + } + + private case class ResolvedAppResource(localPath: String, sparkJarPath: String) +} + +private[spark] object KubernetesSparkRestServer { + private val barrier = new CountDownLatch(1) + + def main(args: Array[String]): Unit = { + val parsedArguments = KubernetesSparkRestServerArguments.fromArgsArray(args) + val secretFile = new File(parsedArguments.secretFile.get) + if (!secretFile.isFile) { + throw new IllegalArgumentException(s"Secret file specified by --secret-file" + + " is not a file, or does not exist.") + } + val sslOptions = if (parsedArguments.useSsl) { + val keyStorePassword = parsedArguments + .keyStorePasswordFile + .map(new File(_)) + .map(Files.toString(_, Charsets.UTF_8)) + val keyPassword = parsedArguments + .keyPasswordFile + .map(new File(_)) + .map(Files.toString(_, Charsets.UTF_8)) + new SSLOptions( + enabled = true, + keyStore = parsedArguments.keyStoreFile.map(new File(_)), + keyStoreType = parsedArguments.keyStoreType, + keyStorePassword = keyStorePassword, + keyPassword = keyPassword) + } else { + new SSLOptions + } + val secretBytes = Files.toByteArray(secretFile) + val sparkConf = new SparkConf(true) + val exitCode = new AtomicInteger(0) + val server = new KubernetesSparkRestServer( + parsedArguments.host.get, + parsedArguments.port.get, + sparkConf, + secretBytes, + barrier, + exitCode, + sslOptions) + server.start() + ShutdownHookManager.addShutdownHook(() => { + try { + server.stop() + } finally { + barrier.countDown() + } + }) + barrier.await() + System.exit(exitCode.get()) + } +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala new file mode 100644 index 0000000000000..51313e00ce2da --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/MultiServerFeignTarget.scala @@ -0,0 +1,89 @@ +/* + * 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 feign.{Request, RequestTemplate, RetryableException, Retryer, Target} +import scala.reflect.ClassTag +import scala.util.Random + +import org.apache.spark.internal.Logging + +private[kubernetes] class MultiServerFeignTarget[T : ClassTag]( + private val servers: Seq[String], + private val maxRetriesPerServer: Int = 1, + private val delayBetweenRetriesMillis: Int = 1000) extends Target[T] with Retryer with Logging { + require(servers.nonEmpty, "Must provide at least one server URI.") + + private val threadLocalShuffledServers = new ThreadLocal[Seq[String]] { + override def initialValue(): Seq[String] = Random.shuffle(servers) + } + private val threadLocalCurrentAttempt = new ThreadLocal[Int] { + override def initialValue(): Int = 0 + } + + override def `type`(): Class[T] = { + implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] + } + + /** + * Cloning the target is done on every request, for use on the current + * thread - thus it's important that clone returns a "fresh" target. + */ + override def clone(): Retryer = { + reset() + this + } + + override def name(): String = { + s"${getClass.getSimpleName} with servers [${servers.mkString(",")}]" + } + + override def apply(requestTemplate: RequestTemplate): Request = { + if (!requestTemplate.url().startsWith("http")) { + requestTemplate.insert(0, url()) + } + requestTemplate.request() + } + + override def url(): String = threadLocalShuffledServers.get.head + + override def continueOrPropagate(e: RetryableException): Unit = { + threadLocalCurrentAttempt.set(threadLocalCurrentAttempt.get + 1) + val currentAttempt = threadLocalCurrentAttempt.get + if (threadLocalCurrentAttempt.get < maxRetriesPerServer) { + logWarning(s"Attempt $currentAttempt of $maxRetriesPerServer failed for" + + s" server ${url()}. Retrying request...", e) + Thread.sleep(delayBetweenRetriesMillis) + } else { + val previousUrl = url() + threadLocalShuffledServers.set(threadLocalShuffledServers.get.drop(1)) + if (threadLocalShuffledServers.get.isEmpty) { + logError(s"Failed request to all servers $maxRetriesPerServer times.", e) + throw e + } else { + logWarning(s"Failed request to $previousUrl $maxRetriesPerServer times." + + s" Trying to access ${url()} instead.", e) + threadLocalCurrentAttempt.set(0) + } + } + } + + def reset(): Unit = { + threadLocalShuffledServers.set(Random.shuffle(servers)) + threadLocalCurrentAttempt.set(0) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/NodePortUrisDriverServiceManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/NodePortUrisDriverServiceManager.scala new file mode 100644 index 0000000000000..fa8362677f38f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/NodePortUrisDriverServiceManager.scala @@ -0,0 +1,70 @@ +/* + * 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 io.fabric8.kubernetes.api.model.{Service, ServiceBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging + +/** + * Creates the service with an open NodePort. The URI to reach the submission server is thus + * at the address of any of the nodes through the service's node port. + */ +private[spark] class NodePortUrisDriverServiceManager extends DriverServiceManager with Logging { + + override def getServiceManagerType: String = NodePortUrisDriverServiceManager.TYPE + + override def customizeDriverService(driverServiceTemplate: ServiceBuilder): ServiceBuilder = { + driverServiceTemplate.editSpec().withType("NodePort").endSpec() + } + + override def getDriverServiceSubmissionServerUris(driverService: Service): Set[String] = { + val urlScheme = if (sparkConf.get(DRIVER_SUBMIT_SSL_ENABLED)) { + "https" + } else { + logWarning("Submitting application details, application secret, and local" + + " jars to the cluster over an insecure connection. You should configure SSL" + + " to secure this step.") + "http" + } + val servicePort = driverService.getSpec.getPorts.asScala + .filter(_.getName == SUBMISSION_SERVER_PORT_NAME) + .head.getNodePort + val nodeUrls = kubernetesClient.nodes.list.getItems.asScala + .filterNot(node => node.getSpec.getUnschedulable != null && + node.getSpec.getUnschedulable) + .flatMap(_.getStatus.getAddresses.asScala) + // The list contains hostnames, internal and external IP addresses. + // (https://kubernetes.io/docs/admin/node/#addresses) + // we want only external IP addresses and legacyHostIP addresses in our list + // legacyHostIPs are deprecated and will be removed in the future. + // (https://github.com/kubernetes/kubernetes/issues/9267) + .filter(address => address.getType == "ExternalIP" || address.getType == "LegacyHostIP") + .map(address => { + s"$urlScheme://${address.getAddress}:$servicePort" + }).toSet + require(nodeUrls.nonEmpty, "No nodes found to contact the driver!") + nodeUrls + } +} + +private[spark] object NodePortUrisDriverServiceManager { + val TYPE = "NodePort" +} 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..36f7149a832c3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -0,0 +1,42 @@ +/* + * 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.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} + +private[spark] class KubernetesClusterManager extends ExternalClusterManager { + + override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") + + override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = { + val scheduler = new TaskSchedulerImpl(sc) + sc.taskScheduler = scheduler + scheduler + } + + override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) + : SchedulerBackend = { + new KubernetesClusterSchedulerBackend(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc) + } + + 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..90907ff83ed84 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -0,0 +1,266 @@ +/* + * 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.util.UUID +import java.util.concurrent.Executors +import java.util.concurrent.atomic.{AtomicInteger, AtomicLong} + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, Pod, QuantityBuilder} +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.deploy.kubernetes.{Client, KubernetesClientBuilder} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.{ThreadUtils, Utils} + +private[spark] class KubernetesClusterSchedulerBackend( + scheduler: TaskSchedulerImpl, + val sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + + import KubernetesClusterSchedulerBackend._ + + private val EXECUTOR_MODIFICATION_LOCK = new Object + private val runningExecutorPods = new scala.collection.mutable.HashMap[String, Pod] + + private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) + 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 kubernetesDriverServiceName = conf + .get(KUBERNETES_DRIVER_SERVICE_NAME) + .getOrElse( + throw new SparkException("Must specify the service name the driver is running with")) + + private val kubernetesDriverPodName = conf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse( + throw new SparkException("Must specify the driver pod name")) + + 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.getOption("spark.executor.cores").getOrElse("1") + + private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) + + private val kubernetesClient = KubernetesClientBuilder + .buildFromWithinPod(kubernetesNamespace) + + private val driverPod = try { + kubernetesClient.pods().inNamespace(kubernetesNamespace). + withName(kubernetesDriverPodName).get() + } catch { + case throwable: Throwable => + logError(s"Executor cannot find driver pod.", throwable) + throw new SparkException(s"Executor cannot find driver pod", throwable) + } + + override val minRegisteredRatio = + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + 0.8 + } else { + super.minRegisteredRatio + } + + 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(1) + + 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() + if (!Utils.isDynamicAllocationEnabled(sc.conf)) { + doRequestTotalExecutors(initialExecutors) + } + } + + override def stop(): Unit = { + // 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 { + runningExecutorPods.values.foreach(kubernetesClient.pods().delete(_)) + } catch { + case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) + } + try { + kubernetesClient.services().withName(kubernetesDriverServiceName).delete() + } catch { + case e: Throwable => logError("Uncaught exception while shutting down driver service.", e) + } + try { + kubernetesClient.close() + } catch { + case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e) + } + super.stop() + } + + private def allocateNewExecutorPod(): (String, Pod) = { + val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString + val name = s"${applicationId()}-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 selectors = Map(SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId()).asJava + 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) + .build() + val requiredEnv = Seq( + (ENV_EXECUTOR_PORT, executorPort.toString), + (ENV_DRIVER_URL, driverUrl), + (ENV_EXECUTOR_CORES, executorCores), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, applicationId()), + (ENV_EXECUTOR_ID, executorId) + ).map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build()) + val requiredPorts = Seq( + (EXECUTOR_PORT_NAME, executorPort), + (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + .map(port => { + new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build() + }) + try { + (executorId, kubernetesClient.pods().createNew() + .withNewMetadata() + .withName(name) + .withLabels(selectors) + .withOwnerReferences() + .addNewOwnerReference() + .withController(true) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withHostname(hostname) + .addNewContainer() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy("IfNotPresent") + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .addToLimits("cpu", executorCpuQuantity) + .endResources() + .withEnv(requiredEnv.asJava) + .withPorts(requiredPorts.asJava) + .endContainer() + .endSpec() + .done()) + } catch { + case throwable: Throwable => + logError("Failed to allocate executor pod.", throwable) + throw throwable + } + } + + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] { + EXECUTOR_MODIFICATION_LOCK.synchronized { + if (requestedTotal > totalExpectedExecutors.get) { + logInfo(s"Requesting ${requestedTotal - totalExpectedExecutors.get}" + + s" additional executors, expecting total $requestedTotal and currently" + + s" expected ${totalExpectedExecutors.get}") + for (i <- 0 until (requestedTotal - totalExpectedExecutors.get)) { + runningExecutorPods += allocateNewExecutorPod() + } + } + totalExpectedExecutors.set(requestedTotal) + } + true + } + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { + EXECUTOR_MODIFICATION_LOCK.synchronized { + for (executor <- executorIds) { + runningExecutorPods.remove(executor) match { + case Some(pod) => kubernetesClient.pods().delete(pod) + case None => logWarning(s"Unable to remove pod for unknown executor $executor") + } + } + } + true + } +} + +private object KubernetesClusterSchedulerBackend { + private val DEFAULT_STATIC_PORT = 10000 + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) +} 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..e9f88e37a5f89 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -0,0 +1,138 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.1.0-SNAPSHOT + ../../../pom.xml + + + spark-docker-minimal-bundle_2.11 + Spark Project Docker Minimal Bundle + http://spark.apache.org/ + pom + + + 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 + + + driver-docker-dist + pre-integration-test + + single + + + + src/main/assembly/driver-assembly.xml + + posix + + + + executor-docker-dist + pre-integration-test + + single + + + + src/main/assembly/executor-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/driver-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml new file mode 100644 index 0000000000000..b5fcaa75f049c --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml @@ -0,0 +1,75 @@ + + + driver-docker-dist + + tar.gz + dir + + false + + + + ${project.parent.basedir}/sbin/ + + sbin + + **/* + + + + + ${project.parent.basedir}/bin/ + + bin + + **/* + + + + + ${project.parent.basedir}/conf/ + + conf + + **/* + + + + + src/main/docker/driver + + + + **/* + + + + + + 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/assembly/executor-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml new file mode 100644 index 0000000000000..d97ba56562a12 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml @@ -0,0 +1,84 @@ + + + executor-docker-dist + + tar.gz + dir + + false + + + + ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ + + ui-resources/org/apache/spark/ui/static + + **/* + + + + + ${project.parent.basedir}/sbin/ + + sbin + + **/* + + + + + ${project.parent.basedir}/bin/ + + bin + + **/* + + + + + ${project.parent.basedir}/conf/ + + conf + + **/* + + + + + src/main/docker/executor + + + + **/* + + + + + + 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/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile new file mode 100644 index 0000000000000..3bf6b50ff69c1 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile @@ -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. +# + +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-driver:latest -f dockerfiles/driver/Dockerfile . + +RUN apk upgrade --update +RUN apk add --update bash +RUN mkdir -p /opt/spark +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD examples /opt/spark/examples +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark + +CMD SSL_ARGS="" && \ + if ! [ -z ${SPARK_SUBMISSION_USE_SSL+x} ]; then SSL_ARGS="$SSL_ARGS --use-ssl $SPARK_SUBMISSION_USE_SSL"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-file $SPARK_SUBMISSION_KEYSTORE_FILE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_TYPE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-type $SPARK_SUBMISSION_KEYSTORE_TYPE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-password-file $SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE"; fi && \ + if ! [ -z ${SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE+x} ]; then SSL_ARGS="$SSL_ARGS --keystore-key-password-file $SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE"; fi && \ + exec bin/spark-class org.apache.spark.deploy.rest.kubernetes.KubernetesSparkRestServer \ + --hostname $HOSTNAME \ + --port $SPARK_SUBMISSION_SERVER_PORT \ + --secret-file $SPARK_SUBMISSION_SECRET_LOCATION \ + ${SSL_ARGS} 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..cd5ac466a1fa0 --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -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. +# + +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-executor:latest -f dockerfiles/executor/Dockerfile . + +RUN apk upgrade --update +RUN apk add --update bash +RUN mkdir -p /opt/spark +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD examples /opt/spark/examples +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark + +# TODO support spark.executor.extraClassPath +CMD exec ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp ${SPARK_HOME}/jars/\* 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 $HOSTNAME 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..b9c29b26eb648 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml @@ -0,0 +1,33 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.1.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs-helpers_2.11 + jar + Spark Project 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..16dd0c9322c13 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml @@ -0,0 +1,51 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.1.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests-spark-jobs_2.11 + jar + Spark Project 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..8b8d5e05f6479 --- /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,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.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("Usage: WordCount ") + } + // 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 + } + val spark = SparkSession.builder() + .appName("Test") + .getOrCreate() + 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..5c54d0e5e3aab --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -0,0 +1,300 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.1.0-k8s-0.1.0-SNAPSHOT + ../../../pom.xml + + + spark-kubernetes-integration-tests_2.11 + jar + Spark Project Kubernetes Integration Tests + + + + org.apache.spark + spark-kubernetes_${scala.binary.version} + ${project.version} + test + + + 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 + driver-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.bouncycastle + bcpkix-jdk15on + + + + + + + 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-driver + pre-integration-test + + copy + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/driver/examples/integration-tests-jars + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/driver/examples/integration-tests-jars + + + + + + copy-test-spark-jobs-to-docker-executor + pre-integration-test + + copy + + + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/executor/examples/integration-tests-jars + + + org.apache.spark + spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} + ${project.version} + jar + ${project.build.directory}/docker/executor/examples/integration-tests-jars + + + + + + unpack-docker-driver-bundle + pre-integration-test + + unpack + + + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + driver-docker-dist + tar.gz + true + ${project.build.directory}/docker/driver + + + + + + unpack-docker-executor-bundle + pre-integration-test + + unpack + + + + + org.apache.spark + spark-docker-minimal-bundle_${scala.binary.version} + ${project.version} + executor-docker-dist + tar.gz + true + ${project.build.directory}/docker/executor + + + + + + + + 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/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/ExternalUriProviderWatch.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ExternalUriProviderWatch.scala new file mode 100644 index 0000000000000..3199a8c385f95 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ExternalUriProviderWatch.scala @@ -0,0 +1,75 @@ +/* + * 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.atomic.AtomicBoolean + +import io.fabric8.kubernetes.api.model.Service +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube +import org.apache.spark.internal.Logging + +/** + * A slightly unrealistic implementation of external URI provision, but works + * for tests - essentially forces the service to revert back to being exposed + * on NodePort. + */ +private[spark] class ExternalUriProviderWatch(kubernetesClient: KubernetesClient) + extends Watcher[Service] with Logging { + + // Visible for testing + val annotationSet = new AtomicBoolean(false) + + override def eventReceived(action: Action, service: Service): Unit = { + if (action == Action.ADDED) { + service.getMetadata + .getAnnotations + .asScala + .get(ANNOTATION_PROVIDE_EXTERNAL_URI).foreach { _ => + if (!annotationSet.getAndSet(true)) { + val nodePortService = kubernetesClient.services().withName(service.getMetadata.getName) + .edit() + .editSpec() + .withType("NodePort") + .endSpec() + .done() + val submissionServerPort = nodePortService + .getSpec() + .getPorts + .asScala + .find(_.getName == SUBMISSION_SERVER_PORT_NAME) + .map(_.getNodePort) + .getOrElse(throw new IllegalStateException("Submission server port not found.")) + val resolvedNodePortUri = s"http://${Minikube.getMinikubeIp}:$submissionServerPort" + kubernetesClient.services().withName(service.getMetadata.getName).edit() + .editMetadata() + .addToAnnotations(ANNOTATION_RESOLVED_EXTERNAL_URI, resolvedNodePortUri) + .endMetadata() + .done() + } + } + } + } + + override def onClose(cause: KubernetesClientException): Unit = { + logWarning("External URI provider watch closed.", cause) + } +} 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..6aa1c1fee0d47 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -0,0 +1,428 @@ +/* + * 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.collect.ImmutableList +import com.google.common.io.Files +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{Config, KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +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, SparkException, SparkFunSuite} +import org.apache.spark.deploy.SparkSubmit +import org.apache.spark.deploy.kubernetes.Client +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube +import org.apache.spark.deploy.kubernetes.integrationtest.restapis.SparkRestApiV1 +import org.apache.spark.deploy.kubernetes.integrationtest.sslutil.SSLUtils +import org.apache.spark.deploy.rest.kubernetes.ExternalSuppliedUrisDriverServiceManager +import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} +import org.apache.spark.util.Utils + +private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { + + private val EXAMPLES_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs") + .toFile + .listFiles()(0) + + private val HELPER_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs-helpers") + .toFile + .listFiles()(0) + + private val TEST_EXISTENCE_FILE = Paths.get("test-data", "input.txt").toFile + private val TEST_EXISTENCE_FILE_CONTENTS = Files.toString(TEST_EXISTENCE_FILE, Charsets.UTF_8) + private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + private val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.SparkPiWithInfiniteWait" + private val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.FileExistenceTest" + private val NAMESPACE = UUID.randomUUID().toString.replaceAll("-", "") + private var minikubeKubernetesClient: KubernetesClient = _ + private var clientConfig: Config = _ + private var keyStoreFile: File = _ + private var trustStoreFile: File = _ + + override def beforeAll(): Unit = { + Minikube.startMinikube() + new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() + Minikube.getKubernetesClient.namespaces.createNew() + .withNewMetadata() + .withName(NAMESPACE) + .endMetadata() + .done() + minikubeKubernetesClient = Minikube.getKubernetesClient.inNamespace(NAMESPACE) + clientConfig = minikubeKubernetesClient.getConfiguration + val (keyStore, trustStore) = SSLUtils.generateKeyStoreTrustStorePair( + Minikube.getMinikubeIp, + "changeit", + "changeit", + "changeit") + keyStoreFile = keyStore + trustStoreFile = trustStore + } + + before { + Eventually.eventually(TIMEOUT, INTERVAL) { + val podsList = minikubeKubernetesClient.pods().list() + assert(podsList == null + || podsList.getItems == null + || podsList.getItems.isEmpty + ) + val servicesList = minikubeKubernetesClient.services().list() + assert(servicesList == null + || servicesList.getItems == null + || servicesList.getItems.isEmpty) + } + } + + after { + val pods = minikubeKubernetesClient.pods().list().getItems.asScala + pods.par.foreach(pod => { + minikubeKubernetesClient + .pods() + .withName(pod.getMetadata.getName) + .withGracePeriod(60) + .delete + }) + // spark-submit sets system properties so we have to clear them + new SparkConf(true).getAll.map(_._1).foreach { System.clearProperty } + } + + override def afterAll(): Unit = { + if (!System.getProperty("spark.docker.test.persistMinikube", "false").toBoolean) { + Minikube.deleteMinikube() + } + } + + private def getSparkMetricsService(sparkBaseAppName: String): SparkRestApiV1 = { + val serviceName = minikubeKubernetesClient.services() + .withLabel("spark-app-name", sparkBaseAppName) + .list() + .getItems + .get(0) + .getMetadata + .getName + Minikube.getService[SparkRestApiV1](serviceName, NAMESPACE, "spark-ui-port") + } + + private def expectationsForStaticAllocation(sparkMetricsService: SparkRestApiV1): Unit = { + val apps = Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService + .getApplications(ImmutableList.of(ApplicationStatus.RUNNING, ApplicationStatus.COMPLETED)) + assert(result.size == 1 + && !result.head.id.equalsIgnoreCase("appid") + && !result.head.id.equalsIgnoreCase("{appId}")) + result + } + Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService.getExecutors(apps.head.id) + assert(result.size == 2) + assert(result.count(exec => exec.id != "driver") == 1) + result + } + Eventually.eventually(TIMEOUT, INTERVAL) { + val result = sparkMetricsService.getStages( + apps.head.id, Seq(StageStatus.COMPLETE).asJava) + assert(result.size == 1) + result + } + } + + test("Run a simple example") { + // We'll make assertions based on spark rest api, so we need to turn on + // spark.ui.enabled explicitly since the scalatest-maven-plugin would set it + // to false by default. + val sparkConf = new SparkConf(true) + .setMaster(s"k8s://https://${Minikube.getMinikubeIp}:8443") + .set("spark.kubernetes.submit.caCertFile", clientConfig.getCaCertFile) + .set("spark.kubernetes.submit.clientKeyFile", clientConfig.getClientKeyFile) + .set("spark.kubernetes.submit.clientCertFile", clientConfig.getClientCertFile) + .set("spark.kubernetes.namespace", NAMESPACE) + .set("spark.kubernetes.driver.docker.image", "spark-driver:latest") + .set("spark.kubernetes.executor.docker.image", "spark-executor:latest") + .set("spark.jars", HELPER_JAR_FILE.getAbsolutePath) + .set("spark.executor.memory", "500m") + .set("spark.executor.cores", "1") + .set("spark.executors.instances", "1") + .set("spark.app.name", "spark-pi") + .set("spark.ui.enabled", "true") + .set("spark.testing", "false") + .set("spark.kubernetes.submit.waitAppCompletion", "false") + val mainAppResource = s"file://${EXAMPLES_JAR_FILE.getAbsolutePath}" + + new Client( + sparkConf = sparkConf, + mainClass = SPARK_PI_MAIN_CLASS, + mainAppResource = mainAppResource, + appArgs = Array.empty[String]).run() + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run using spark-submit") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", HELPER_JAR_FILE.getAbsolutePath, + "--class", SPARK_PI_MAIN_CLASS, + "--conf", "spark.ui.enabled=true", + "--conf", "spark.testing=false", + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + EXAMPLES_JAR_FILE.getAbsolutePath) + SparkSubmit.main(args) + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run using spark-submit with the examples jar on the docker image") { + val args = Array( + "--master", s"k8s://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", s"local:///opt/spark/examples/integration-tests-jars/${HELPER_JAR_FILE.getName}", + "--class", SPARK_PI_MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + s"local:///opt/spark/examples/integration-tests-jars/${EXAMPLES_JAR_FILE.getName}") + SparkSubmit.main(args) + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run with custom labels and annotations") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", HELPER_JAR_FILE.getAbsolutePath, + "--class", SPARK_PI_MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.driver.labels=label1=label1value,label2=label2value", + "--conf", "spark.kubernetes.driver.annotations=" + + "annotation1=annotation1value," + + "annotation2=annotation2value", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + EXAMPLES_JAR_FILE.getAbsolutePath) + SparkSubmit.main(args) + val driverPodMetadata = minikubeKubernetesClient + .pods + .withLabel("spark-app-name", "spark-pi") + .list() + .getItems + .get(0) + .getMetadata + val driverPodLabels = driverPodMetadata.getLabels + // We can't match all of the selectors directly since one of the selectors is based on the + // launch time. + assert(driverPodLabels.size === 5, "Unexpected number of pod labels.") + assert(driverPodLabels.get("spark-app-name") === "spark-pi", "Unexpected value for" + + " spark-app-name label.") + assert(driverPodLabels.get("spark-app-id").startsWith("spark-pi"), "Unexpected value for" + + " spark-app-id label (should be prefixed with the app name).") + assert(driverPodLabels.get("label1") === "label1value", "Unexpected value for label1") + assert(driverPodLabels.get("label2") === "label2value", "Unexpected value for label2") + val driverPodAnnotations = driverPodMetadata.getAnnotations + assert(driverPodAnnotations.size === 2, "Unexpected number of pod annotations.") + assert(driverPodAnnotations.get("annotation1") === "annotation1value", + "Unexpected value for annotation1") + assert(driverPodAnnotations.get("annotation2") === "annotation2value", + "Unexpected value for annotation2") + } + + test("Enable SSL on the driver submit server") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", HELPER_JAR_FILE.getAbsolutePath, + "--class", SPARK_PI_MAIN_CLASS, + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.ssl.kubernetes.submit.enabled=true", + "--conf", "spark.ssl.kubernetes.submit.keyStore=" + + s"file://${keyStoreFile.getAbsolutePath}", + "--conf", "spark.ssl.kubernetes.submit.keyStorePassword=changeit", + "--conf", "spark.ssl.kubernetes.submit.keyPassword=changeit", + "--conf", "spark.ssl.kubernetes.submit.trustStore=" + + s"file://${trustStoreFile.getAbsolutePath}", + "--conf", s"spark.ssl.kubernetes.driverlaunch.trustStorePassword=changeit", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + EXAMPLES_JAR_FILE.getAbsolutePath) + SparkSubmit.main(args) + } + + test("Added files should exist on the driver.") { + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-file-existence-test", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", HELPER_JAR_FILE.getAbsolutePath, + "--files", TEST_EXISTENCE_FILE.getAbsolutePath, + "--class", FILE_EXISTENCE_MAIN_CLASS, + "--conf", "spark.ui.enabled=false", + "--conf", "spark.testing=true", + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + EXAMPLES_JAR_FILE.getAbsolutePath, + TEST_EXISTENCE_FILE.getName, + TEST_EXISTENCE_FILE_CONTENTS) + val podCompletedFuture = SettableFuture.create[Boolean] + val watch = new Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + val containerStatuses = pod.getStatus.getContainerStatuses.asScala + val allSuccessful = containerStatuses.nonEmpty && containerStatuses + .forall(status => { + status.getState.getTerminated != null && status.getState.getTerminated.getExitCode == 0 + }) + if (allSuccessful) { + podCompletedFuture.set(true) + } else { + val failedContainers = containerStatuses.filter(container => { + container.getState.getTerminated != null && + container.getState.getTerminated.getExitCode != 0 + }) + if (failedContainers.nonEmpty) { + podCompletedFuture.setException(new SparkException( + "One or more containers in the driver failed with a nonzero exit code.")) + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + logWarning("Watch closed", e) + } + } + Utils.tryWithResource(minikubeKubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .watch(watch)) { _ => + SparkSubmit.main(args) + assert(podCompletedFuture.get, "Failed to run driver pod") + val driverPod = minikubeKubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .list() + .getItems + .get(0) + val podLog = minikubeKubernetesClient + .pods + .withName(driverPod.getMetadata.getName) + .getLog + assert(podLog.contains(s"File found at /opt/spark/${TEST_EXISTENCE_FILE.getName}" + + s" with correct contents."), "Job did not find the file as expected.") + } + } + + test("Use external URI provider") { + val externalUriProviderWatch = new ExternalUriProviderWatch(minikubeKubernetesClient) + Utils.tryWithResource(minikubeKubernetesClient.services() + .withLabel("spark-app-name", "spark-pi") + .watch(externalUriProviderWatch)) { _ => + val args = Array( + "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", + "--deploy-mode", "cluster", + "--kubernetes-namespace", NAMESPACE, + "--name", "spark-pi", + "--executor-memory", "512m", + "--executor-cores", "1", + "--num-executors", "1", + "--jars", HELPER_JAR_FILE.getAbsolutePath, + "--class", SPARK_PI_MAIN_CLASS, + "--conf", "spark.ui.enabled=true", + "--conf", "spark.testing=false", + "--conf", s"spark.kubernetes.submit.caCertFile=${clientConfig.getCaCertFile}", + "--conf", s"spark.kubernetes.submit.clientKeyFile=${clientConfig.getClientKeyFile}", + "--conf", s"spark.kubernetes.submit.clientCertFile=${clientConfig.getClientCertFile}", + "--conf", "spark.kubernetes.executor.docker.image=spark-executor:latest", + "--conf", "spark.kubernetes.driver.docker.image=spark-driver:latest", + "--conf", "spark.kubernetes.submit.waitAppCompletion=false", + "--conf", s"${DRIVER_SERVICE_MANAGER_TYPE.key}=${ExternalSuppliedUrisDriverServiceManager.TYPE}", + EXAMPLES_JAR_FILE.getAbsolutePath) + SparkSubmit.main(args) + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + assert(externalUriProviderWatch.annotationSet.get) + val driverService = minikubeKubernetesClient + .services() + .withLabel("spark-app-name", "spark-pi") + .list() + .getItems + .asScala(0) + assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_PROVIDE_EXTERNAL_URI), + "External URI request annotation was not set on the driver service.") + // Unfortunately we can't check the correctness of the actual value of the URI, as it depends + // on the driver submission port set on the driver service but we remove that port from the + // service once the submission is complete. + assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_RESOLVED_EXTERNAL_URI), + "Resolved URI annotation not set on driver service.") + } + } +} 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..1aa6a7b7e70c2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -0,0 +1,58 @@ +/* + * 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.net.URI +import java.nio.file.Paths + +import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider +import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates} +import org.apache.http.client.utils.URIBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Seconds, Span} + +private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, String]) { + + 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() } + dockerClient.build(Paths.get("target", "docker", "driver"), "spark-driver") + dockerClient.build(Paths.get("target", "docker", "executor"), "spark-executor") + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.scala new file mode 100644 index 0000000000000..07274bf962dde --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/minikube/Minikube.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.integrationtest.minikube + +import java.io.{BufferedReader, InputStreamReader} +import java.nio.file.Paths +import java.util.concurrent.TimeUnit +import java.util.regex.Pattern +import javax.net.ssl.X509TrustManager + +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} +import io.fabric8.kubernetes.client.internal.SSLUtils +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import org.apache.spark.deploy.rest.kubernetes.HttpClientUtil +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 getService[T: ClassTag]( + serviceName: String, + namespace: String, + servicePortName: String, + servicePath: String = ""): T = synchronized { + val kubernetesMaster = s"https://$getMinikubeIp:8443" + val url = s"${ + Array[String]( + kubernetesMaster, + "api", "v1", "proxy", + "namespaces", namespace, + "services", serviceName).mkString("/")}" + + s":$servicePortName$servicePath" + 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() + val sslContext = SSLUtils.sslContext(kubernetesConf) + val trustManager = SSLUtils.trustManagers(kubernetesConf)(0).asInstanceOf[X509TrustManager] + HttpClientUtil.createClient[T](Set(url), 5, sslContext.getSocketFactory, trustManager) + } + + 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.") + } + } + val fullCommand = Array(MINIKUBE_EXECUTABLE_DEST.getAbsolutePath, action) ++ args + 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(MINIKUBE_STARTUP_TIMEOUT_SECONDS, TimeUnit.SECONDS), + s"Timed out while executing $action on minikube.") + assert(proc.exitValue == 0, s"Failed to execute minikube $action ${args.mkString(" ")}") + outputLines.toSeq + } +} + +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/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/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala new file mode 100644 index 0000000000000..bde7b43226660 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/sslutil/SSLUtils.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.integrationtest.sslutil + +import java.io.{File, FileOutputStream} +import java.math.BigInteger +import java.nio.file.Files +import java.security.{KeyPairGenerator, KeyStore, SecureRandom} +import java.util.{Calendar, Random} +import javax.security.auth.x500.X500Principal + +import org.bouncycastle.asn1.x509.{Extension, GeneralName, GeneralNames} +import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3CertificateBuilder} +import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder + +import org.apache.spark.util.Utils + +private[spark] object SSLUtils { + + def generateKeyStoreTrustStorePair( + ipAddress: String, + keyStorePassword: String, + keyPassword: String, + trustStorePassword: String): (File, File) = { + val keyPairGenerator = KeyPairGenerator.getInstance("RSA") + keyPairGenerator.initialize(512) + val keyPair = keyPairGenerator.generateKeyPair() + 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) + val jcaCertificate = new JcaX509CertificateConverter().getCertificate(bcCertificate) + val keyStore = KeyStore.getInstance("JKS") + keyStore.load(null, null) + keyStore.setKeyEntry("key", keyPair.getPrivate, + keyPassword.toCharArray, Array(jcaCertificate)) + 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", jcaCertificate) + val trustStoreFile = new File(tempDir, "trustStore.jks") + Utils.tryWithResource(new FileOutputStream(trustStoreFile)) { + trustStore.store(_, trustStorePassword.toCharArray) + } + (keyStoreFile, trustStoreFile) + } + +} 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..818f33868ef7a 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.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 37e7dccd2e27d..d1b8982b2d464 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.1.0-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..3dc6539e178c3 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.1.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 7bf4fc0df45e8..ce3879131e736 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.1.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 06569e6ee2231..715aa4bbf6373 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.1.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 35d53b30191a5..d978d6ef6fdd4 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.1.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 38374b5ae5a3b..424965419c10b 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.1.0-SNAPSHOT ../pom.xml