")
+ }
+ sparkConfWithCredentialLocations
+ }
+
+ private def safeFileConfToBase64(
+ conf: String,
+ fileNotFoundFormatString: String): Option[String] = {
+ submissionSparkConf.getOption(conf)
+ .map(new File(_))
+ .map { file =>
+ require(file.isFile, String.format(fileNotFoundFormatString, file.getAbsolutePath))
+ BaseEncoding.base64().encode(Files.toByteArray(file))
+ }
+ }
+
+ private def resolveSecretLocation(
+ mountedUserSpecified: Option[String],
+ valueMountedFromSubmitter: Option[String],
+ mountedCanonicalLocation: String): Option[String] = {
+ mountedUserSpecified.orElse(valueMountedFromSubmitter.map( _ => {
+ mountedCanonicalLocation
+ }))
+ }
+
+ private def resolveSecretData(
+ mountedUserSpecified: Option[String],
+ valueMountedFromSubmitter: Option[String],
+ secretName: String): Map[String, String] = {
+ mountedUserSpecified.map { _ => Map.empty[String, String]}
+ .getOrElse {
+ valueMountedFromSubmitter.map { valueBase64 =>
+ Map(secretName -> valueBase64)
+ }.getOrElse(Map.empty[String, String])
+ }
+ }
+
+ private implicit def augmentSparkConf(sparkConf: SparkConf): OptionSettableSparkConf = {
+ new OptionSettableSparkConf(sparkConf)
+ }
+}
+
+private class OptionSettableSparkConf(sparkConf: SparkConf) {
+ def setOption(configEntry: String, option: Option[String]): SparkConf = {
+ option.map( opt => {
+ sparkConf.set(configEntry, opt)
+ }).getOrElse(sparkConf)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala
new file mode 100644
index 0000000000000..29cad18c484c0
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata}
+
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder}
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
+
+/**
+ * Configures the init-container that bootstraps dependencies into the driver pod.
+ */
+private[spark] class InitContainerBootstrapStep(
+ initContainerConfigurationSteps: Seq[InitContainerConfigurationStep],
+ initContainerConfigMapName: String,
+ initContainerConfigMapKey: String)
+ extends DriverConfigurationStep {
+
+ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
+ var currentInitContainerSpec = InitContainerSpec(
+ initContainerProperties = Map.empty[String, String],
+ additionalDriverSparkConf = Map.empty[String, String],
+ initContainer = new ContainerBuilder().build(),
+ driverContainer = driverSpec.driverContainer,
+ podToInitialize = driverSpec.driverPod,
+ initContainerDependentResources = Seq.empty[HasMetadata])
+ for (nextStep <- initContainerConfigurationSteps) {
+ currentInitContainerSpec = nextStep.configureInitContainer(currentInitContainerSpec)
+ }
+ val configMap = PropertiesConfigMapFromScalaMapBuilder.buildConfigMap(
+ initContainerConfigMapName,
+ initContainerConfigMapKey,
+ currentInitContainerSpec.initContainerProperties)
+ val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone()
+ .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, initContainerConfigMapName)
+ .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, initContainerConfigMapKey)
+ .setAll(currentInitContainerSpec.additionalDriverSparkConf)
+ val resolvedDriverPod = InitContainerUtil.appendInitContainer(
+ currentInitContainerSpec.podToInitialize, currentInitContainerSpec.initContainer)
+ driverSpec.copy(
+ driverPod = resolvedDriverPod,
+ driverContainer = currentInitContainerSpec.driverContainer,
+ driverSparkConf = resolvedDriverSparkConf,
+ otherKubernetesResources =
+ driverSpec.otherKubernetesResources ++
+ currentInitContainerSpec.initContainerDependentResources ++
+ Seq(configMap))
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala
new file mode 100644
index 0000000000000..3ec4b6c4df10f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder}
+
+import org.apache.spark.SparkConf
+
+/**
+ * Represents the components and characteristics of a Spark driver. The driver can be considered
+ * as being comprised of the driver pod itself, any other Kubernetes resources that the driver
+ * pod depends on, and the SparkConf that should be supplied to the Spark application. The driver
+ * container should be operated on via the specific field of this case class as opposed to trying
+ * to edit the container directly on the pod. The driver container should be attached at the
+ * end of executing all submission steps.
+ */
+private[spark] case class KubernetesDriverSpec(
+ driverPod: Pod,
+ driverContainer: Container,
+ otherKubernetesResources: Seq[HasMetadata],
+ driverSparkConf: SparkConf)
+
+private[spark] object KubernetesDriverSpec {
+ def initialSpec(initialSparkConf: SparkConf): KubernetesDriverSpec = {
+ KubernetesDriverSpec(
+ // Set new metadata and a new spec so that submission steps can use
+ // PodBuilder#editMetadata() and/or PodBuilder#editSpec() safely.
+ new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build(),
+ new ContainerBuilder().build(),
+ Seq.empty[HasMetadata],
+ initialSparkConf.clone())
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala
new file mode 100644
index 0000000000000..024d643ddf9fd
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import io.fabric8.kubernetes.api.model.ContainerBuilder
+
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils
+
+private[spark] class PythonStep(
+ primaryPyFile: String,
+ otherPyFiles: Seq[String],
+ filesDownloadPath: String) extends DriverConfigurationStep {
+
+ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
+ val resolvedOtherPyFilesString = if (otherPyFiles.isEmpty) {
+ "null"
+ } else {
+ KubernetesFileUtils.resolveFilePaths(otherPyFiles, filesDownloadPath).mkString(",")
+ }
+ val withPythonPrimaryFileContainer = new ContainerBuilder(driverSpec.driverContainer)
+ .addNewEnv()
+ .withName(ENV_PYSPARK_PRIMARY)
+ .withValue(KubernetesFileUtils.resolveFilePath(primaryPyFile, filesDownloadPath))
+ .endEnv()
+ .addNewEnv()
+ .withName(ENV_PYSPARK_FILES)
+ .withValue(resolvedOtherPyFilesString)
+ .endEnv()
+ driverSpec.copy(driverContainer = withPythonPrimaryFileContainer.build())
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala
new file mode 100644
index 0000000000000..60bf27beacaaf
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils
+
+private[spark] class BaseInitContainerConfigurationStep(
+ sparkJars: Seq[String],
+ sparkFiles: Seq[String],
+ jarsDownloadPath: String,
+ filesDownloadPath: String,
+ configMapName: String,
+ configMapKey: String,
+ podAndInitContainerBootstrap: SparkPodInitContainerBootstrap)
+ extends InitContainerConfigurationStep {
+
+ override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
+ val remoteJarsToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkJars)
+ val remoteFilesToDownload = KubernetesFileUtils.getOnlyRemoteFiles(sparkFiles)
+ val remoteJarsConf = if (remoteJarsToDownload.nonEmpty) {
+ Map(INIT_CONTAINER_REMOTE_JARS.key -> remoteJarsToDownload.mkString(","))
+ } else {
+ Map.empty[String, String]
+ }
+ val remoteFilesConf = if (remoteFilesToDownload.nonEmpty) {
+ Map(INIT_CONTAINER_REMOTE_FILES.key -> remoteFilesToDownload.mkString(","))
+ } else {
+ Map.empty[String, String]
+ }
+ val baseInitContainerConfig = Map[String, String](
+ INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key -> jarsDownloadPath,
+ INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> filesDownloadPath) ++
+ remoteJarsConf ++
+ remoteFilesConf
+ val bootstrappedPodAndInitContainer =
+ podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes(
+ PodWithDetachedInitContainer(
+ initContainerSpec.podToInitialize,
+ initContainerSpec.initContainer,
+ initContainerSpec.driverContainer))
+ initContainerSpec.copy(
+ initContainer = bootstrappedPodAndInitContainer.initContainer,
+ driverContainer = bootstrappedPodAndInitContainer.mainContainer,
+ podToInitialize = bootstrappedPodAndInitContainer.pod,
+ initContainerProperties = baseInitContainerConfig)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala
new file mode 100644
index 0000000000000..7b7622c3d4f8b
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+/**
+ * Represents a step in preparing the init-container for the driver and executors.
+ */
+private[spark] trait InitContainerConfigurationStep {
+
+ def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala
new file mode 100644
index 0000000000000..e4ea5235af18f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl
+import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl}
+import org.apache.spark.util.Utils
+
+/**
+ * Returns the complete ordered list of steps required to configure the init-container.
+ */
+private[spark] class InitContainerConfigurationStepsOrchestrator(
+ namespace: String,
+ kubernetesResourceNamePrefix: String,
+ sparkJars: Seq[String],
+ sparkFiles: Seq[String],
+ jarsDownloadPath: String,
+ filesDownloadPath: String,
+ dockerImagePullPolicy: String,
+ driverLabels: Map[String, String],
+ initContainerConfigMapName: String,
+ initContainerConfigMapKey: String,
+ submissionSparkConf: SparkConf) {
+
+ private val submittedResourcesSecretName = s"$kubernetesResourceNamePrefix-init-secret"
+ private val resourceStagingServerUri = submissionSparkConf.get(RESOURCE_STAGING_SERVER_URI)
+ private val resourceStagingServerInternalUri =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_URI)
+ private val initContainerImage = submissionSparkConf.get(INIT_CONTAINER_DOCKER_IMAGE)
+ private val downloadTimeoutMinutes = submissionSparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
+ private val maybeResourceStagingServerInternalTrustStore =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_FILE)
+ .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE))
+ private val maybeResourceStagingServerInternalTrustStorePassword =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_PASSWORD)
+ .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD))
+ private val maybeResourceStagingServerInternalTrustStoreType =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_TRUSTSTORE_TYPE)
+ .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE))
+ private val maybeResourceStagingServerInternalClientCert =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_CLIENT_CERT_PEM)
+ .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM))
+ private val resourceStagingServerInternalSslEnabled =
+ submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED)
+ .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED))
+ .getOrElse(false)
+ OptionRequirements.requireNandDefined(
+ maybeResourceStagingServerInternalClientCert,
+ maybeResourceStagingServerInternalTrustStore,
+ "Cannot provide both a certificate file and a trustStore file for init-containers to" +
+ " use for contacting the resource staging server over TLS.")
+
+ require(maybeResourceStagingServerInternalTrustStore.forall { trustStore =>
+ Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match {
+ case "file" | "local" => true
+ case _ => false
+ }
+ }, "TrustStore URI used for contacting the resource staging server from init containers must" +
+ " have no scheme, or scheme file://, or scheme local://.")
+
+ require(maybeResourceStagingServerInternalClientCert.forall { trustStore =>
+ Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") match {
+ case "file" | "local" => true
+ case _ => false
+ }
+ }, "Client cert file URI used for contacting the resource staging server from init containers" +
+ " must have no scheme, or scheme file://, or scheme local://.")
+
+ def getAllConfigurationSteps(): Seq[InitContainerConfigurationStep] = {
+ val initContainerBootstrap = new SparkPodInitContainerBootstrapImpl(
+ initContainerImage,
+ dockerImagePullPolicy,
+ jarsDownloadPath,
+ filesDownloadPath,
+ downloadTimeoutMinutes,
+ initContainerConfigMapName,
+ initContainerConfigMapKey)
+ val baseInitContainerStep = new BaseInitContainerConfigurationStep(
+ sparkJars,
+ sparkFiles,
+ jarsDownloadPath,
+ filesDownloadPath,
+ initContainerConfigMapName,
+ initContainerConfigMapKey,
+ initContainerBootstrap)
+ val submittedResourcesInitContainerStep = resourceStagingServerUri.map {
+ stagingServerUri =>
+ val mountSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl(
+ submittedResourcesSecretName,
+ INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH)
+ val submittedDependencyUploader = new SubmittedDependencyUploaderImpl(
+ driverLabels,
+ namespace,
+ stagingServerUri,
+ sparkJars,
+ sparkFiles,
+ new ResourceStagingServerSslOptionsProviderImpl(submissionSparkConf).getSslOptions,
+ RetrofitClientFactoryImpl)
+ new SubmittedResourcesInitContainerConfigurationStep(
+ submittedResourcesSecretName,
+ resourceStagingServerInternalUri.getOrElse(stagingServerUri),
+ INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH,
+ resourceStagingServerInternalSslEnabled,
+ maybeResourceStagingServerInternalTrustStore,
+ maybeResourceStagingServerInternalClientCert,
+ maybeResourceStagingServerInternalTrustStorePassword,
+ maybeResourceStagingServerInternalTrustStoreType,
+ submittedDependencyUploader,
+ mountSecretPlugin)
+ }
+ Seq(baseInitContainerStep) ++ submittedResourcesInitContainerStep.toSeq
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala
new file mode 100644
index 0000000000000..5b5ac3c1f17c2
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod}
+
+/**
+ * Represents a given configuration of the init-container, informing the main
+ * InitContainerBootstrapStep of how the driver should be configured. This includes:
+ *
+ * - What properties should be set on the init-container,
+ * - What Spark properties should be set on the driver's SparkConf given this init-container,
+ * - The spec of the init container itself,
+ * - The spec of the main container so that it can be modified to share volumes with the
+ * init-container
+ * - The spec of the pod EXCEPT for the addition of the given init-container (e.g. volumes
+ * the init-container needs or modifications to a main container that shares data with the
+ * init-container),
+ * - Any Kubernetes resources that need to be created for the init-container's function.
+ */
+private[spark] case class InitContainerSpec(
+ initContainerProperties: Map[String, String],
+ additionalDriverSparkConf: Map[String, String],
+ initContainer: Container,
+ driverContainer: Container,
+ podToInitialize: Pod,
+ initContainerDependentResources: Seq[HasMetadata])
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala
new file mode 100644
index 0000000000000..7aa27a1de6811
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.{BaseEncoding, Files}
+import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader
+import org.apache.spark.internal.config.OptionalConfigEntry
+import org.apache.spark.util.Utils
+
+private[spark] class SubmittedResourcesInitContainerConfigurationStep(
+ submittedResourcesSecretName: String,
+ internalResourceStagingServerUri: String,
+ initContainerSecretMountPath: String,
+ resourceStagingServerSslEnabled: Boolean,
+ maybeInternalTrustStoreUri: Option[String],
+ maybeInternalClientCertUri: Option[String],
+ maybeInternalTrustStorePassword: Option[String],
+ maybeInternalTrustStoreType: Option[String],
+ submittedDependencyUploader: SubmittedDependencyUploader,
+ submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin)
+ extends InitContainerConfigurationStep {
+
+ override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
+ val jarsIdAndSecret = submittedDependencyUploader.uploadJars()
+ val filesIdAndSecret = submittedDependencyUploader.uploadFiles()
+
+ val submittedResourcesInitContainerProperties = Map[String, String](
+ RESOURCE_STAGING_SERVER_URI.key -> internalResourceStagingServerUri,
+ INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsIdAndSecret.resourceId,
+ INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key ->
+ s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY",
+ INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesIdAndSecret.resourceId,
+ INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key ->
+ s"$initContainerSecretMountPath/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY",
+ RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> resourceStagingServerSslEnabled.toString) ++
+ resolveSecretPath(
+ maybeInternalTrustStoreUri,
+ INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE,
+ "TrustStore URI") ++
+ resolveSecretPath(
+ maybeInternalClientCertUri,
+ INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY,
+ RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM,
+ "Client certificate URI") ++
+ maybeInternalTrustStorePassword.map { password =>
+ (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password)
+ }.toMap ++
+ maybeInternalTrustStoreType.map { storeType =>
+ (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType)
+ }.toMap
+ val initContainerSecret = createResourceStagingServerSecret(
+ jarsIdAndSecret.resourceSecret, filesIdAndSecret.resourceSecret)
+ val additionalDriverSparkConf =
+ Map(
+ EXECUTOR_INIT_CONTAINER_SECRET.key -> initContainerSecret.getMetadata.getName,
+ EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> initContainerSecretMountPath)
+ val initContainerWithSecretVolumeMount = submittedResourcesSecretPlugin
+ .mountResourceStagingServerSecretIntoInitContainer(initContainerSpec.initContainer)
+ val podWithSecretVolume = submittedResourcesSecretPlugin
+ .addResourceStagingServerSecretVolumeToPod(initContainerSpec.podToInitialize)
+ initContainerSpec.copy(
+ initContainer = initContainerWithSecretVolumeMount,
+ podToInitialize = podWithSecretVolume,
+ initContainerDependentResources =
+ initContainerSpec.initContainerDependentResources ++ Seq(initContainerSecret),
+ initContainerProperties =
+ initContainerSpec.initContainerProperties ++ submittedResourcesInitContainerProperties,
+ additionalDriverSparkConf = additionalDriverSparkConf)
+ }
+
+ private def createResourceStagingServerSecret(
+ jarsResourceSecret: String, filesResourceSecret: String): Secret = {
+ val trustStoreBase64 = convertFileToBase64IfSubmitterLocal(
+ INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, maybeInternalTrustStoreUri)
+ val clientCertBase64 = convertFileToBase64IfSubmitterLocal(
+ INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY, maybeInternalClientCertUri)
+ val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8))
+ val filesSecretBase64 = BaseEncoding.base64().encode(
+ filesResourceSecret.getBytes(Charsets.UTF_8))
+ val secretData = Map(
+ INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY -> jarsSecretBase64,
+ INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY -> filesSecretBase64) ++
+ trustStoreBase64 ++
+ clientCertBase64
+ val kubernetesSecret = new SecretBuilder()
+ .withNewMetadata()
+ .withName(submittedResourcesSecretName)
+ .endMetadata()
+ .addToData(secretData.asJava)
+ .build()
+ kubernetesSecret
+ }
+
+ private def convertFileToBase64IfSubmitterLocal(secretKey: String, secretUri: Option[String])
+ : Map[String, String] = {
+ secretUri.filter { trustStore =>
+ Option(Utils.resolveURI(trustStore).getScheme).getOrElse("file") == "file"
+ }.map { uri =>
+ val file = new File(Utils.resolveURI(uri).getPath)
+ require(file.isFile, "Dependency server trustStore provided at" +
+ file.getAbsolutePath + " does not exist or is not a file.")
+ (secretKey, BaseEncoding.base64().encode(Files.toByteArray(file)))
+ }.toMap
+ }
+
+ private def resolveSecretPath(
+ maybeUri: Option[String],
+ secretKey: String,
+ configEntry: OptionalConfigEntry[String],
+ uriType: String): Map[String, String] = {
+ maybeUri.map(Utils.resolveURI).map { uri =>
+ val resolvedPath = Option(uri.getScheme).getOrElse("file") match {
+ case "file" => s"$initContainerSecretMountPath/$secretKey"
+ case "local" => uri.getPath
+ case invalid => throw new SparkException(s"$uriType has invalid scheme $invalid must be" +
+ s" local://, file://, or empty.")
+ }
+ (configEntry.key, resolvedPath)
+ }.toMap
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala
new file mode 100644
index 0000000000000..d050e0a41a15a
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.File
+
+// Extracted for testing so that unit tests don't have to depend on Utils.fetchFile
+private[spark] trait FileFetcher {
+ def fetchFile(uri: String, targetDir: File): Unit
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala
new file mode 100644
index 0000000000000..0e274678ad6f0
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.File
+import java.util.concurrent.TimeUnit
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import com.google.common.util.concurrent.SettableFuture
+import okhttp3.ResponseBody
+import retrofit2.{Call, Callback, Response}
+import scala.concurrent.{ExecutionContext, Future}
+import scala.concurrent.duration.Duration
+
+import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SSLOptions}
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.deploy.kubernetes.{CompressionUtils, KubernetesCredentials}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+private trait WaitableCallback[T] extends Callback[T] {
+ private val complete = SettableFuture.create[Boolean]
+
+ override final def onFailure(call: Call[T], t: Throwable): Unit = complete.setException(t)
+
+ override final def onResponse(call: Call[T], response: Response[T]): Unit = {
+ require(response.code() >= 200 && response.code() < 300, Option(response.errorBody())
+ .map(_.string())
+ .getOrElse(s"Error executing HTTP request, but error body was not provided."))
+ handleResponse(response.body())
+ complete.set(true)
+ }
+
+ protected def handleResponse(body: T): Unit
+
+ final def waitForCompletion(time: Long, timeUnit: TimeUnit): Unit = {
+ complete.get(time, timeUnit)
+ }
+}
+
+private class DownloadTarGzCallback(downloadDir: File) extends WaitableCallback[ResponseBody] {
+
+ override def handleResponse(responseBody: ResponseBody): Unit = {
+ Utils.tryWithResource(responseBody.byteStream()) { responseStream =>
+ CompressionUtils.unpackTarStreamToDirectory(responseStream, downloadDir)
+ }
+ }
+}
+/**
+ * Process that fetches files from a resource staging server and/or arbitrary remote locations.
+ *
+ * The init-container can handle fetching files from any of those sources, but not all of the
+ * sources need to be specified. This allows for composing multiple instances of this container
+ * with different configurations for different download sources, or using the same container to
+ * download everything at once.
+ */
+private[spark] class KubernetesSparkDependencyDownloadInitContainer(
+ sparkConf: SparkConf,
+ retrofitClientFactory: RetrofitClientFactory,
+ fileFetcher: FileFetcher,
+ resourceStagingServerSslOptions: SSLOptions) extends Logging {
+
+ private implicit val downloadExecutor = ExecutionContext.fromExecutorService(
+ ThreadUtils.newDaemonCachedThreadPool("download-executor"))
+ private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI)
+
+ private val maybeDownloadJarsResourceIdentifier = sparkConf
+ .get(INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER)
+ private val downloadJarsSecretLocation = new File(
+ sparkConf.get(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION))
+ private val maybeDownloadFilesResourceIdentifier = sparkConf
+ .get(INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER)
+ private val downloadFilesSecretLocation = new File(
+ sparkConf.get(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION))
+
+ private val jarsDownloadDir = new File(
+ sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION))
+ private val filesDownloadDir = new File(
+ sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION))
+
+ private val remoteJars = sparkConf.get(INIT_CONTAINER_REMOTE_JARS)
+ private val remoteFiles = sparkConf.get(INIT_CONTAINER_REMOTE_FILES)
+
+ private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT)
+
+ def run(): Unit = {
+ val resourceStagingServerJarsDownload = Future[Unit] {
+ downloadResourcesFromStagingServer(
+ maybeDownloadJarsResourceIdentifier,
+ downloadJarsSecretLocation,
+ jarsDownloadDir,
+ "Starting to download jars from resource staging server...",
+ "Finished downloading jars from resource staging server.",
+ s"Application jars download secret provided at" +
+ s" ${downloadJarsSecretLocation.getAbsolutePath} does not exist or is not a file.",
+ s"Application jars download directory provided at" +
+ s" ${jarsDownloadDir.getAbsolutePath} does not exist or is not a directory.")
+ }
+ val resourceStagingServerFilesDownload = Future[Unit] {
+ downloadResourcesFromStagingServer(
+ maybeDownloadFilesResourceIdentifier,
+ downloadFilesSecretLocation,
+ filesDownloadDir,
+ "Starting to download files from resource staging server...",
+ "Finished downloading files from resource staging server.",
+ s"Application files download secret provided at" +
+ s" ${downloadFilesSecretLocation.getAbsolutePath} does not exist or is not a file.",
+ s"Application files download directory provided at" +
+ s" ${filesDownloadDir.getAbsolutePath} does not exist or is not" +
+ s" a directory.")
+ }
+ val remoteJarsDownload = Future[Unit] {
+ downloadFiles(remoteJars,
+ jarsDownloadDir,
+ s"Remote jars download directory specified at $jarsDownloadDir does not exist" +
+ s" or is not a directory.")
+ }
+ val remoteFilesDownload = Future[Unit] {
+ downloadFiles(remoteFiles,
+ filesDownloadDir,
+ s"Remote files download directory specified at $filesDownloadDir does not exist" +
+ s" or is not a directory.")
+ }
+ waitForFutures(
+ resourceStagingServerJarsDownload,
+ resourceStagingServerFilesDownload,
+ remoteJarsDownload,
+ remoteFilesDownload)
+ }
+
+ private def downloadResourcesFromStagingServer(
+ maybeResourceId: Option[String],
+ resourceSecretLocation: File,
+ resourceDownloadDir: File,
+ downloadStartMessage: String,
+ downloadFinishedMessage: String,
+ errMessageOnSecretNotAFile: String,
+ errMessageOnDownloadDirNotADirectory: String): Unit = {
+ maybeResourceStagingServerUri.foreach { resourceStagingServerUri =>
+ maybeResourceId.foreach { resourceId =>
+ require(resourceSecretLocation.isFile, errMessageOnSecretNotAFile)
+ require(resourceDownloadDir.isDirectory, errMessageOnDownloadDirNotADirectory)
+ val service = retrofitClientFactory.createRetrofitClient(
+ resourceStagingServerUri,
+ classOf[ResourceStagingServiceRetrofit],
+ resourceStagingServerSslOptions)
+ val resourceSecret = Files.toString(resourceSecretLocation, Charsets.UTF_8)
+ val downloadResourceCallback = new DownloadTarGzCallback(resourceDownloadDir)
+ logInfo(downloadStartMessage)
+ service.downloadResources(resourceId, resourceSecret).enqueue(downloadResourceCallback)
+ downloadResourceCallback.waitForCompletion(downloadTimeoutMinutes, TimeUnit.MINUTES)
+ logInfo(downloadFinishedMessage)
+ }
+ }
+ }
+
+ private def downloadFiles(
+ filesCommaSeparated: Option[String],
+ downloadDir: File,
+ errMessageOnDestinationNotADirectory: String): Unit = {
+ if (filesCommaSeparated.isDefined) {
+ require(downloadDir.isDirectory, errMessageOnDestinationNotADirectory)
+ }
+ filesCommaSeparated.map(_.split(",")).toSeq.flatten.foreach { file =>
+ fileFetcher.fetchFile(file, downloadDir)
+ }
+ }
+
+ private def waitForFutures(futures: Future[_]*) {
+ futures.foreach {
+ ThreadUtils.awaitResult(_, Duration.create(downloadTimeoutMinutes, TimeUnit.MINUTES))
+ }
+ }
+}
+
+private class FileFetcherImpl(sparkConf: SparkConf, securityManager: SparkSecurityManager)
+ extends FileFetcher {
+ def fetchFile(uri: String, targetDir: File): Unit = {
+ Utils.fetchFile(
+ url = uri,
+ targetDir = targetDir,
+ conf = sparkConf,
+ securityMgr = securityManager,
+ hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf),
+ timestamp = System.currentTimeMillis(),
+ useCache = false)
+ }
+}
+
+object KubernetesSparkDependencyDownloadInitContainer extends Logging {
+ def main(args: Array[String]): Unit = {
+ logInfo("Starting init-container to download Spark application dependencies.")
+ val sparkConf = if (args.nonEmpty) {
+ SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0)))
+ } else {
+ new SparkConf(true)
+ }
+ val securityManager = new SparkSecurityManager(sparkConf)
+ val resourceStagingServerSslOptions =
+ new ResourceStagingServerSslOptionsProviderImpl(sparkConf).getSslOptions
+ val fileFetcher = new FileFetcherImpl(sparkConf, securityManager)
+ new KubernetesSparkDependencyDownloadInitContainer(
+ sparkConf,
+ RetrofitClientFactoryImpl,
+ fileFetcher,
+ resourceStagingServerSslOptions).run()
+ logInfo("Finished downloading application dependencies.")
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala
new file mode 100644
index 0000000000000..17f90118e150d
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala
@@ -0,0 +1,139 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{File, FileInputStream, FileOutputStream, InputStreamReader}
+import java.security.{KeyStore, PrivateKey}
+import java.security.cert.Certificate
+import java.util.UUID
+
+import com.google.common.base.Charsets
+import org.bouncycastle.asn1.pkcs.PrivateKeyInfo
+import org.bouncycastle.cert.X509CertificateHolder
+import org.bouncycastle.cert.jcajce.JcaX509CertificateConverter
+import org.bouncycastle.openssl.{PEMKeyPair, PEMParser}
+import org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter
+import scala.collection.mutable
+
+import org.apache.spark.SparkException
+import org.apache.spark.util.Utils
+
+private[spark] object PemsToKeyStoreConverter {
+
+ /**
+ * Loads the given key-cert pair into a temporary keystore file. Returns the File pointing
+ * to where the keyStore was written to disk.
+ */
+ def convertPemsToTempKeyStoreFile(
+ keyPemFile: File,
+ certPemFile: File,
+ keyAlias: String,
+ keyStorePassword: String,
+ keyPassword: String,
+ keyStoreType: Option[String]): File = {
+ require(keyPemFile.isFile, s"Key PEM file provided at ${keyPemFile.getAbsolutePath}" +
+ " does not exist or is not a file.")
+ require(certPemFile.isFile, s"Cert PEM file provided at ${certPemFile.getAbsolutePath}" +
+ " does not exist or is not a file.")
+ val privateKey = parsePrivateKeyFromPemFile(keyPemFile)
+ val certificates = parseCertificatesFromPemFile(certPemFile)
+ val resolvedKeyStoreType = keyStoreType.getOrElse(KeyStore.getDefaultType)
+ val keyStore = KeyStore.getInstance(resolvedKeyStoreType)
+ keyStore.load(null, null)
+ keyStore.setKeyEntry(
+ keyAlias,
+ privateKey,
+ keyPassword.toCharArray,
+ certificates)
+ val keyStoreDir = Utils.createTempDir("temp-keystores")
+ val keyStoreFile = new File(keyStoreDir, s"keystore-${UUID.randomUUID()}.$resolvedKeyStoreType")
+ Utils.tryWithResource(new FileOutputStream(keyStoreFile)) { storeStream =>
+ keyStore.store(storeStream, keyStorePassword.toCharArray)
+ }
+ keyStoreFile
+ }
+
+ def convertCertPemToTrustStore(
+ certPemFile: File,
+ trustStoreType: Option[String]): KeyStore = {
+ require(certPemFile.isFile, s"Cert PEM file provided at ${certPemFile.getAbsolutePath}" +
+ " does not exist or is not a file.")
+ val trustStore = KeyStore.getInstance(trustStoreType.getOrElse(KeyStore.getDefaultType))
+ trustStore.load(null, null)
+ parseCertificatesFromPemFile(certPemFile).zipWithIndex.foreach { case (cert, index) =>
+ trustStore.setCertificateEntry(s"certificate-$index", cert)
+ }
+ trustStore
+ }
+
+ def convertCertPemToTempTrustStoreFile(
+ certPemFile: File,
+ trustStorePassword: String,
+ trustStoreType: Option[String]): File = {
+ val trustStore = convertCertPemToTrustStore(certPemFile, trustStoreType)
+ val tempTrustStoreDir = Utils.createTempDir(namePrefix = "temp-trustStore")
+ val tempTrustStoreFile = new File(tempTrustStoreDir,
+ s"trustStore.${trustStoreType.getOrElse(KeyStore.getDefaultType)}")
+ Utils.tryWithResource(new FileOutputStream(tempTrustStoreFile)) {
+ trustStore.store(_, trustStorePassword.toCharArray)
+ }
+ tempTrustStoreFile
+ }
+
+ private def withPemParsedFromFile[T](pemFile: File)(f: (PEMParser => T)): T = {
+ Utils.tryWithResource(new FileInputStream(pemFile)) { pemStream =>
+ Utils.tryWithResource(new InputStreamReader(pemStream, Charsets.UTF_8)) { pemReader =>
+ Utils.tryWithResource(new PEMParser(pemReader))(f)
+ }
+ }
+ }
+
+ private def parsePrivateKeyFromPemFile(keyPemFile: File): PrivateKey = {
+ withPemParsedFromFile(keyPemFile) { keyPemParser =>
+ val converter = new JcaPEMKeyConverter
+ keyPemParser.readObject() match {
+ case privateKey: PrivateKeyInfo =>
+ converter.getPrivateKey(privateKey)
+ case keyPair: PEMKeyPair =>
+ converter.getPrivateKey(keyPair.getPrivateKeyInfo)
+ case _ =>
+ throw new SparkException(s"Key file provided at ${keyPemFile.getAbsolutePath}" +
+ s" is not a key pair or private key PEM file.")
+ }
+ }
+ }
+
+ private def parseCertificatesFromPemFile(certPemFile: File): Array[Certificate] = {
+ withPemParsedFromFile(certPemFile) { certPemParser =>
+ val certificates = mutable.Buffer[Certificate]()
+ var pemObject = certPemParser.readObject()
+ while (pemObject != null) {
+ pemObject match {
+ case certificate: X509CertificateHolder =>
+ val converter = new JcaX509CertificateConverter
+ certificates += converter.getCertificate(certificate)
+ case _ =>
+ }
+ pemObject = certPemParser.readObject()
+ }
+ if (certificates.isEmpty) {
+ throw new SparkException(s"No certificates found in ${certPemFile.getAbsolutePath}")
+ }
+ certificates.toArray
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala
new file mode 100644
index 0000000000000..0b97317eba8b1
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala
@@ -0,0 +1,142 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.File
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import io.fabric8.kubernetes.client.Config
+import org.eclipse.jetty.http.HttpVersion
+import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector, SslConnectionFactory}
+import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
+import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorScheduler}
+import org.glassfish.jersey.media.multipart.MultiPartFeature
+import org.glassfish.jersey.server.ResourceConfig
+import org.glassfish.jersey.servlet.ServletContainer
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{SystemClock, ThreadUtils, Utils}
+
+private[spark] class ResourceStagingServer(
+ port: Int,
+ serviceInstance: ResourceStagingService,
+ sslOptionsProvider: ResourceStagingServerSslOptionsProvider) extends Logging {
+
+ private var jettyServer: Option[Server] = None
+
+ def start(): Unit = synchronized {
+ val threadPool = new QueuedThreadPool
+ val contextHandler = new ServletContextHandler()
+ val jsonProvider = new JacksonJaxbJsonProvider()
+ jsonProvider.setMapper(new ObjectMapper().registerModule(new DefaultScalaModule))
+ val resourceConfig = new ResourceConfig().registerInstances(
+ serviceInstance,
+ jsonProvider,
+ new MultiPartFeature)
+ val servletHolder = new ServletHolder("main", new ServletContainer(resourceConfig))
+ contextHandler.setContextPath("/api/")
+ contextHandler.addServlet(servletHolder, "/*")
+ threadPool.setDaemon(true)
+ val resolvedConnectionFactories = sslOptionsProvider.getSslOptions
+ .createJettySslContextFactory()
+ .map(sslFactory => {
+ val sslConnectionFactory = new SslConnectionFactory(
+ sslFactory, HttpVersion.HTTP_1_1.asString())
+ val rawHttpConfiguration = new HttpConfiguration()
+ rawHttpConfiguration.setSecureScheme("https")
+ rawHttpConfiguration.setSecurePort(port)
+ val rawHttpConnectionFactory = new HttpConnectionFactory(rawHttpConfiguration)
+ Array(sslConnectionFactory, rawHttpConnectionFactory)
+ }).getOrElse(Array(new HttpConnectionFactory()))
+ val server = new Server(threadPool)
+ val connector = new ServerConnector(
+ server,
+ null,
+ // Call this full constructor to set this, which forces daemon threads:
+ new ScheduledExecutorScheduler("DependencyServer-Executor", true),
+ null,
+ -1,
+ -1,
+ resolvedConnectionFactories: _*)
+ connector.setPort(port)
+ server.addConnector(connector)
+ server.setHandler(contextHandler)
+ server.start()
+ jettyServer = Some(server)
+ logInfo(s"Resource staging server started on port $port.")
+ }
+
+ def join(): Unit = jettyServer.foreach(_.join())
+
+ def stop(): Unit = synchronized {
+ jettyServer.foreach(_.stop())
+ jettyServer = None
+ }
+}
+
+object ResourceStagingServer {
+ def main(args: Array[String]): Unit = {
+ val sparkConf = if (args.nonEmpty) {
+ SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0)))
+ } else {
+ new SparkConf(true)
+ }
+ val apiServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_API_SERVER_URL)
+ val initialAccessExpirationMs = sparkConf.get(
+ RESOURCE_STAGING_SERVER_INITIAL_ACCESS_EXPIRATION_TIMEOUT)
+ val dependenciesRootDir = Utils.createTempDir(namePrefix = "local-application-dependencies")
+ val useServiceAccountCredentials = sparkConf.get(
+ RESOURCE_STAGING_SERVER_USE_SERVICE_ACCOUNT_CREDENTIALS)
+ // Namespace doesn't matter because we list resources from various namespaces
+ val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient(
+ apiServerUri,
+ None,
+ APISERVER_AUTH_RESOURCE_STAGING_SERVER_CONF_PREFIX,
+ sparkConf,
+ Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH))
+ .filter( _ => useServiceAccountCredentials),
+ Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))
+ .filter( _ => useServiceAccountCredentials))
+
+ val stagedResourcesStore = new StagedResourcesStoreImpl(dependenciesRootDir)
+ val stagedResourcesCleaner = new StagedResourcesCleanerImpl(
+ stagedResourcesStore,
+ kubernetesClient,
+ ThreadUtils.newDaemonSingleThreadScheduledExecutor("resource-expiration"),
+ new SystemClock(),
+ initialAccessExpirationMs)
+ stagedResourcesCleaner.start()
+ val serviceInstance = new ResourceStagingServiceImpl(
+ stagedResourcesStore, stagedResourcesCleaner)
+ val sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf)
+ val server = new ResourceStagingServer(
+ port = sparkConf.get(RESOURCE_STAGING_SERVER_PORT),
+ serviceInstance = serviceInstance,
+ sslOptionsProvider = sslOptionsProvider)
+ server.start()
+ try {
+ server.join()
+ } finally {
+ server.stop()
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala
new file mode 100644
index 0000000000000..cb1e65421c013
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.File
+import java.security.SecureRandom
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import org.apache.commons.lang3.RandomStringUtils
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions}
+import org.apache.spark.deploy.kubernetes.OptionRequirements
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.internal.Logging
+
+private[spark] trait ResourceStagingServerSslOptionsProvider {
+ def getSslOptions: SSLOptions
+}
+
+private[spark] class ResourceStagingServerSslOptionsProviderImpl(sparkConf: SparkConf)
+ extends ResourceStagingServerSslOptionsProvider with Logging {
+
+ private val SECURE_RANDOM = new SecureRandom()
+
+ def getSslOptions: SSLOptions = {
+ val baseSslOptions = new SecurityManager(sparkConf)
+ .getSSLOptions(RESOURCE_STAGING_SERVER_SSL_NAMESPACE)
+ val maybeKeyPem = sparkConf.get(RESOURCE_STAGING_SERVER_KEY_PEM)
+ val maybeServerCertPem = sparkConf.get(RESOURCE_STAGING_SERVER_CERT_PEM)
+ val maybeKeyStorePasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE)
+ val maybeKeyPasswordFile = sparkConf.get(RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE)
+ val maybeClientCertPem = sparkConf.get(RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM)
+
+ logSslConfigurations(
+ baseSslOptions,
+ maybeKeyPem,
+ maybeServerCertPem,
+ maybeKeyStorePasswordFile,
+ maybeKeyPasswordFile,
+ maybeClientCertPem)
+
+ OptionRequirements.requireNandDefined(
+ baseSslOptions.keyStore,
+ maybeKeyPem,
+ "Shouldn't provide both key PEM and keyStore files for TLS.")
+ OptionRequirements.requireNandDefined(
+ baseSslOptions.keyStore,
+ maybeServerCertPem,
+ "Shouldn't provide both certificate PEM and keyStore files for TLS.")
+ OptionRequirements.requireNandDefined(
+ baseSslOptions.keyStorePassword,
+ maybeKeyStorePasswordFile,
+ "Shouldn't provide both the keyStore password value and the keyStore password file.")
+ OptionRequirements.requireNandDefined(
+ baseSslOptions.keyPassword,
+ maybeKeyPasswordFile,
+ "Shouldn't provide both a keyStore key password value and a keyStore key password file.")
+ OptionRequirements.requireBothOrNeitherDefined(
+ maybeKeyPem,
+ maybeServerCertPem,
+ "When providing a certificate PEM file, the key PEM file must also be provided.",
+ "When providing a key PEM file, the certificate PEM file must also be provided.")
+ OptionRequirements.requireNandDefined(baseSslOptions.trustStore, maybeClientCertPem,
+ "Shouldn't provide both the trustStore and a client certificate PEM file.")
+
+ val resolvedKeyStorePassword = baseSslOptions.keyStorePassword
+ .orElse(maybeKeyStorePasswordFile.map { keyStorePasswordFile =>
+ safeFileToString(keyStorePasswordFile, "KeyStore password file")
+ })
+ .orElse(maybeKeyPem.map { _ => randomPassword()})
+ val resolvedKeyStoreKeyPassword = baseSslOptions.keyPassword
+ .orElse(maybeKeyPasswordFile.map { keyPasswordFile =>
+ safeFileToString(keyPasswordFile, "KeyStore key password file")
+ })
+ .orElse(maybeKeyPem.map { _ => randomPassword()})
+ val resolvedKeyStore = baseSslOptions.keyStore.orElse {
+ for {
+ keyPem <- maybeKeyPem
+ certPem <- maybeServerCertPem
+ keyStorePassword <- resolvedKeyStorePassword
+ keyPassword <- resolvedKeyStoreKeyPassword
+ } yield {
+ val keyPemFile = new File(keyPem)
+ val certPemFile = new File(certPem)
+ PemsToKeyStoreConverter.convertPemsToTempKeyStoreFile(
+ keyPemFile,
+ certPemFile,
+ "key",
+ keyStorePassword,
+ keyPassword,
+ baseSslOptions.keyStoreType)
+ }
+ }
+ val resolvedTrustStorePassword = baseSslOptions.trustStorePassword
+ .orElse(maybeClientCertPem.map( _ => "defaultTrustStorePassword"))
+ val resolvedTrustStore = baseSslOptions.trustStore.orElse {
+ for {
+ clientCertPem <- maybeClientCertPem
+ trustStorePassword <- resolvedTrustStorePassword
+ } yield {
+ val certPemFile = new File(clientCertPem)
+ PemsToKeyStoreConverter.convertCertPemToTempTrustStoreFile(
+ certPemFile,
+ trustStorePassword,
+ baseSslOptions.trustStoreType)
+ }
+ }
+ baseSslOptions.copy(
+ keyStore = resolvedKeyStore,
+ keyStorePassword = resolvedKeyStorePassword,
+ keyPassword = resolvedKeyStoreKeyPassword,
+ trustStore = resolvedTrustStore)
+ }
+
+ private def logSslConfigurations(
+ baseSslOptions: SSLOptions,
+ maybeKeyPem: Option[String],
+ maybeServerCertPem: Option[String],
+ maybeKeyStorePasswordFile: Option[String],
+ maybeKeyPasswordFile: Option[String],
+ maybeClientCertPem: Option[String]) = {
+ logDebug("The following SSL configurations were provided for the resource staging server:")
+ logDebug(s"KeyStore File: ${baseSslOptions.keyStore.map(_.getAbsolutePath).getOrElse("N/A")}")
+ logDebug("KeyStore Password: " +
+ baseSslOptions.keyStorePassword.map(_ => "").getOrElse("N/A"))
+ logDebug(s"KeyStore Password File: ${maybeKeyStorePasswordFile.getOrElse("N/A")}")
+ logDebug("Key Password: " +
+ baseSslOptions.keyPassword.map(_ => "").getOrElse("N/A"))
+ logDebug(s"Key Password File: ${maybeKeyPasswordFile.getOrElse("N/A")}")
+ logDebug(s"KeyStore Type: ${baseSslOptions.keyStoreType.getOrElse("N/A")}")
+ logDebug(s"Key PEM: ${maybeKeyPem.getOrElse("N/A")}")
+ logDebug(s"Server-side certificate PEM: ${maybeServerCertPem.getOrElse("N/A")}")
+ logDebug(s"Client-side certificate PEM: ${maybeClientCertPem.getOrElse("N/A")}")
+ }
+
+ private def safeFileToString(filePath: String, fileType: String): String = {
+ val file = new File(filePath)
+ if (!file.isFile) {
+ throw new SparkException(s"$fileType provided at ${file.getAbsolutePath} does not exist or"
+ + s" is not a file.")
+ }
+ Files.toString(file, Charsets.UTF_8)
+ }
+
+ private def randomPassword(): String = {
+ RandomStringUtils.random(1024, 0, Integer.MAX_VALUE, false, false, null, SECURE_RANDOM)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala
new file mode 100644
index 0000000000000..b9d283a99ade9
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.InputStream
+import javax.ws.rs.{Consumes, GET, HeaderParam, Path, PathParam, POST, Produces}
+import javax.ws.rs.core.{MediaType, StreamingOutput}
+
+import org.glassfish.jersey.media.multipart.FormDataParam
+
+import org.apache.spark.deploy.kubernetes.KubernetesCredentials
+import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret
+
+/**
+ * Service that receives application data that can be retrieved later on. This is primarily used
+ * in the context of Spark, but the concept is generic enough to be used for arbitrary applications.
+ * The use case is to have a place for Kubernetes application submitters to bootstrap dynamic,
+ * heavyweight application data for pods. Application submitters may have data stored on their
+ * local disks that they want to provide to the pods they create through the API server. ConfigMaps
+ * are one way to provide this data, but the data in ConfigMaps are stored in etcd which cannot
+ * maintain data in the hundreds of megabytes in size.
+ *
+ * The general use case is for an application submitter to ship the dependencies to the server via
+ * {@link uploadResources}; the application submitter will then receive a unique secure token.
+ * The application submitter then ought to convert the token into a secret, and use this secret in
+ * a pod that fetches the uploaded dependencies via {@link downloadResources}. An application can
+ * provide multiple resource bundles simply by hitting the upload endpoint multiple times and
+ * downloading each bundle with the appropriate secret.
+ */
+@Path("/v0")
+private[spark] trait ResourceStagingService {
+
+ /**
+ * Register a resource with the dependency service, so that pods with the given labels can
+ * retrieve them when they run.
+ *
+ * @param resources Application resources to upload, compacted together in tar + gzip format.
+ * The tarball should contain the files laid out in a flat hierarchy, without
+ * any directories. We take a stream here to avoid holding these entirely in
+ * memory.
+ * @param resourcesOwner A description of the "owner" of a resource. A resource owner is a
+ * Kubernetes API object in a given namespace, with a specific set of
+ * labels. When there are no resources of the owner's type in the given
+ * namespace with the given labels, the resources are cleaned up. The owner
+ * bundle also includes any Kubernetes credentials that are required for
+ * resource staging server to watch the object's state over time.
+ * @return A unique token that should be provided when retrieving these dependencies later.
+ */
+ @POST
+ @Consumes(Array(MediaType.MULTIPART_FORM_DATA, MediaType.APPLICATION_JSON, MediaType.TEXT_PLAIN))
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ @Path("/resources")
+ def uploadResources(
+ @FormDataParam("resources") resources: InputStream,
+ @FormDataParam("resourcesOwner") resourcesOwner: StagedResourcesOwner)
+ : SubmittedResourceIdAndSecret
+
+ /**
+ * Download an application's resources. The resources are provided as a stream, where the stream's
+ * underlying data matches the stream that was uploaded in uploadResources.
+ */
+ @GET
+ @Consumes(Array(MediaType.APPLICATION_JSON))
+ @Produces(Array(MediaType.APPLICATION_OCTET_STREAM))
+ @Path("/resources/{resourceId}")
+ def downloadResources(
+ @PathParam("resourceId") resourceId: String,
+ @HeaderParam("Authorization") resourceSecret: String): StreamingOutput
+
+ /**
+ * Health check.
+ */
+ @GET
+ @Consumes(Array(MediaType.APPLICATION_JSON))
+ @Produces(Array(MediaType.TEXT_PLAIN))
+ @Path("/ping")
+ def ping(): String
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala
new file mode 100644
index 0000000000000..7bc21c21619e1
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{File, FileOutputStream, InputStream, OutputStream}
+import java.security.SecureRandom
+import java.util.UUID
+import javax.ws.rs.{NotAuthorizedException, NotFoundException}
+import javax.ws.rs.core.StreamingOutput
+
+import com.google.common.io.{BaseEncoding, ByteStreams, Files}
+import scala.collection.concurrent.TrieMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.deploy.kubernetes.KubernetesCredentials
+import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+private[spark] class ResourceStagingServiceImpl(
+ stagedResourcesStore: StagedResourcesStore,
+ stagedResourcesCleaner: StagedResourcesCleaner)
+ extends ResourceStagingService with Logging {
+
+ override def uploadResources(
+ resources: InputStream,
+ resourcesOwner: StagedResourcesOwner): SubmittedResourceIdAndSecret = {
+ val stagedResources = stagedResourcesStore.addResources(
+ resourcesOwner.ownerNamespace, resources)
+ stagedResourcesCleaner.registerResourceForCleaning(
+ stagedResources.resourceId, resourcesOwner)
+ SubmittedResourceIdAndSecret(stagedResources.resourceId, stagedResources.resourceSecret)
+ }
+
+ override def downloadResources(resourceId: String, resourceSecret: String): StreamingOutput = {
+ val resource = stagedResourcesStore.getResources(resourceId)
+ .getOrElse(throw new NotFoundException(s"No resource bundle found with id $resourceId"))
+ if (!resource.resourceSecret.equals(resourceSecret)) {
+ throw new NotAuthorizedException(s"Unauthorized to download resource with id $resourceId")
+ }
+ stagedResourcesCleaner.markResourceAsUsed(resourceId)
+ new StreamingOutput {
+ override def write(outputStream: OutputStream) = {
+ Files.copy(resource.resourcesFile, outputStream)
+ }
+ }
+ }
+
+ override def ping(): String = "pong"
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala
new file mode 100644
index 0000000000000..5fbf0f9c43970
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import okhttp3.{RequestBody, ResponseBody}
+import retrofit2.Call
+import retrofit2.http.{Multipart, Path, Streaming}
+
+import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret
+
+/**
+ * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on
+ * how to use this service, see the aforementioned JAX-RS based interface.
+ */
+private[spark] trait ResourceStagingServiceRetrofit {
+
+ @Multipart
+ @retrofit2.http.POST("api/v0/resources/")
+ def uploadResources(
+ @retrofit2.http.Part("resources") resources: RequestBody,
+ @retrofit2.http.Part("resourcesOwner") resourcesOwner: RequestBody)
+ : Call[SubmittedResourceIdAndSecret]
+
+ @Streaming
+ @retrofit2.http.GET("api/v0/resources/{resourceId}")
+ def downloadResources(
+ @Path("resourceId") resourceId: String,
+ @retrofit2.http.Header("Authorization") resourceSecret: String): Call[ResponseBody]
+
+ @retrofit2.http.GET("/api/ping")
+ def ping(): String
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala
new file mode 100644
index 0000000000000..5046cb479054c
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.FileInputStream
+import java.net.{InetSocketAddress, URI}
+import java.security.{KeyStore, SecureRandom}
+import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager}
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import io.fabric8.kubernetes.client.Config
+import okhttp3.{Dispatcher, OkHttpClient}
+import retrofit2.Retrofit
+import retrofit2.converter.jackson.JacksonConverterFactory
+import retrofit2.converter.scalars.ScalarsConverterFactory
+
+import org.apache.spark.SSLOptions
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+private[spark] trait RetrofitClientFactory {
+ def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T
+}
+
+private[spark] object RetrofitClientFactoryImpl extends RetrofitClientFactory with Logging {
+
+ private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule)
+ private val SECURE_RANDOM = new SecureRandom()
+
+ def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T = {
+ val dispatcher = new Dispatcher(ThreadUtils.newDaemonCachedThreadPool(s"http-client-$baseUrl"))
+ val serviceUri = URI.create(baseUrl)
+ val maybeAllProxy = Option.apply(System.getProperty(Config.KUBERNETES_ALL_PROXY))
+ val serviceUriScheme = serviceUri.getScheme
+ val maybeHttpProxy = (if (serviceUriScheme.equalsIgnoreCase("https")) {
+ Option.apply(System.getProperty(Config.KUBERNETES_HTTPS_PROXY))
+ } else if (serviceUriScheme.equalsIgnoreCase("http")) {
+ Option.apply(System.getProperty(Config.KUBERNETES_HTTP_PROXY))
+ } else {
+ maybeAllProxy
+ }).map(uriStringToProxy)
+ val maybeNoProxy = Option.apply(System.getProperty(Config.KUBERNETES_NO_PROXY))
+ .map(_.split(","))
+ .toSeq
+ .flatten
+ val resolvedProxy = maybeNoProxy.find(_ == serviceUri.getHost)
+ .map( _ => java.net.Proxy.NO_PROXY)
+ .orElse(maybeHttpProxy)
+ .getOrElse(java.net.Proxy.NO_PROXY)
+ val okHttpClientBuilder = new OkHttpClient.Builder()
+ .dispatcher(dispatcher)
+ .proxy(resolvedProxy)
+ logDebug(s"Proxying to $baseUrl through address ${resolvedProxy.address()} with proxy of" +
+ s" type ${resolvedProxy.`type`()}")
+ sslOptions.trustStore.foreach { trustStoreFile =>
+ require(trustStoreFile.isFile, s"TrustStore provided at ${trustStoreFile.getAbsolutePath}"
+ + " does not exist, or is not a file.")
+ val trustStoreType = sslOptions.trustStoreType.getOrElse(KeyStore.getDefaultType)
+ val trustStore = KeyStore.getInstance(trustStoreType)
+ val trustStorePassword = sslOptions.trustStorePassword.map(_.toCharArray).orNull
+ Utils.tryWithResource(new FileInputStream(trustStoreFile)) {
+ trustStore.load(_, trustStorePassword)
+ }
+ val trustManagerFactory = TrustManagerFactory.getInstance(
+ TrustManagerFactory.getDefaultAlgorithm)
+ trustManagerFactory.init(trustStore)
+ val trustManagers = trustManagerFactory.getTrustManagers
+ val sslContext = SSLContext.getInstance("TLSv1.2")
+ sslContext.init(null, trustManagers, SECURE_RANDOM)
+ okHttpClientBuilder.sslSocketFactory(sslContext.getSocketFactory,
+ trustManagers(0).asInstanceOf[X509TrustManager])
+ }
+ val resolvedBaseUrl = if (!baseUrl.endsWith("/")) {
+ s"$baseUrl/"
+ } else {
+ baseUrl
+ }
+ new Retrofit.Builder()
+ .baseUrl(resolvedBaseUrl)
+ .addConverterFactory(ScalarsConverterFactory.create())
+ .addConverterFactory(JacksonConverterFactory.create(OBJECT_MAPPER))
+ .client(okHttpClientBuilder.build())
+ .build()
+ .create(serviceType)
+ }
+
+ private def uriStringToProxy(uriString: String): java.net.Proxy = {
+ val uriObject = URI.create(uriString)
+ new java.net.Proxy(java.net.Proxy.Type.HTTP,
+ new InetSocketAddress(uriObject.getHost, uriObject.getPort))
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala
new file mode 100644
index 0000000000000..9e2b8a780df29
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{File, FileInputStream}
+import java.util.Properties
+
+import com.google.common.collect.Maps
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkConf
+import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider}
+import org.apache.spark.util.Utils
+
+private[spark] object SparkConfPropertiesParser {
+
+ def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = {
+ val sparkConf = new SparkConf(true)
+ if (!propertiesFile.isFile) {
+ throw new IllegalArgumentException(s"Server properties file given at" +
+ s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.")
+ }
+ val properties = new Properties
+ Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load)
+ val propertiesMap = Maps.fromProperties(properties)
+ val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap))
+ propertiesMap.asScala.keys.foreach { key =>
+ configReader.get(key).foreach(sparkConf.set(key, _))
+ }
+ sparkConf
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala
new file mode 100644
index 0000000000000..81f394800f803
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.File
+
+case class StagedResources(
+ resourceId: String,
+ resourceSecret: String,
+ resourcesFile: File)
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala
new file mode 100644
index 0000000000000..5d9db728483fa
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+
+import io.fabric8.kubernetes.client.KubernetesClient
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Clock
+
+private[spark] trait StagedResourcesCleaner {
+
+ def start(): Unit
+
+ def registerResourceForCleaning(
+ resourceId: String, stagedResourceOwner: StagedResourcesOwner): Unit
+
+ def markResourceAsUsed(resourceId: String): Unit
+}
+
+private class StagedResourcesCleanerImpl(
+ stagedResourcesStore: StagedResourcesStore,
+ kubernetesClient: KubernetesClient,
+ cleanupExecutorService: ScheduledExecutorService,
+ clock: Clock,
+ initialAccessExpirationMs: Long)
+ extends StagedResourcesCleaner {
+
+ private val CLEANUP_INTERVAL_MS = 30000
+ private val RESOURCE_LOCK = new Object()
+ private val activeResources = mutable.Map.empty[String, MonitoredResource]
+ private val unusedResources = mutable.Map.empty[String, UnusedMonitoredResource]
+
+ override def start(): Unit = {
+ cleanupExecutorService.scheduleAtFixedRate(
+ new CleanupRunnable(),
+ CLEANUP_INTERVAL_MS,
+ CLEANUP_INTERVAL_MS,
+ TimeUnit.MILLISECONDS)
+ }
+
+ override def registerResourceForCleaning(
+ resourceId: String, stagedResourceOwner: StagedResourcesOwner): Unit = {
+ RESOURCE_LOCK.synchronized {
+ unusedResources(resourceId) = UnusedMonitoredResource(
+ clock.getTimeMillis() + initialAccessExpirationMs,
+ MonitoredResource(resourceId, stagedResourceOwner))
+
+ }
+ }
+
+ override def markResourceAsUsed(resourceId: String): Unit = RESOURCE_LOCK.synchronized {
+ val resource = unusedResources.remove(resourceId)
+ resource.foreach { res =>
+ activeResources(resourceId) = res.resource
+ }
+ }
+
+ private class CleanupRunnable extends Runnable with Logging {
+
+ override def run(): Unit = {
+ // Make a copy so we can iterate through this while modifying
+ val activeResourcesCopy = RESOURCE_LOCK.synchronized {
+ Map.apply(activeResources.toSeq: _*)
+ }
+ for ((resourceId, resource) <- activeResourcesCopy) {
+ val namespace = kubernetesClient.namespaces()
+ .withName(resource.stagedResourceOwner.ownerNamespace)
+ .get()
+ if (namespace == null) {
+ logInfo(s"Resource files with id $resourceId is being removed. The owner's namespace" +
+ s" ${resource.stagedResourceOwner.ownerNamespace} was not found.")
+ stagedResourcesStore.removeResources(resourceId)
+ RESOURCE_LOCK.synchronized {
+ activeResources.remove(resourceId)
+ }
+ } else {
+ val metadataOperation = resource.stagedResourceOwner.ownerType match {
+ case StagedResourcesOwnerType.Pod =>
+ kubernetesClient.pods().inNamespace(resource.stagedResourceOwner.ownerNamespace)
+ case _ =>
+ throw new SparkException(s"Unsupported resource owner type for cleanup:" +
+ s" ${resource.stagedResourceOwner.ownerType}")
+ }
+ if (metadataOperation
+ .withLabels(resource.stagedResourceOwner.ownerLabels.asJava)
+ .list()
+ .getItems
+ .isEmpty) {
+ logInfo(s"Resource files with id $resourceId is being removed. Owners of the" +
+ s" resource with namespace: ${resource.stagedResourceOwner.ownerNamespace}," +
+ s" type: ${resource.stagedResourceOwner.ownerType}, and labels:" +
+ s" ${resource.stagedResourceOwner.ownerLabels} was not found on the API server.")
+ stagedResourcesStore.removeResources(resourceId)
+ RESOURCE_LOCK.synchronized {
+ activeResources.remove(resourceId)
+ }
+ }
+ }
+ }
+
+ // Make a copy so we can iterate through this while modifying
+ val unusedResourcesCopy = RESOURCE_LOCK.synchronized {
+ Map.apply(unusedResources.toSeq: _*)
+ }
+
+ for ((resourceId, resource) <- unusedResourcesCopy) {
+ if (resource.expiresAt < clock.getTimeMillis()) {
+ RESOURCE_LOCK.synchronized {
+ // Check for existence again here (via foreach) because in between the time we starting
+ // iterating over the unused resources copy, we might have already marked the resource
+ // as active in-between, and likely shouldn't remove the resources in such a case.
+ unusedResources.remove(resourceId).foreach { _ =>
+ logInfo(s"Resources with id $resourceId was not accessed after being added to" +
+ s" the staging server at least $initialAccessExpirationMs ms ago. The resource" +
+ s" will be deleted.")
+ stagedResourcesStore.removeResources(resourceId)
+ }
+ }
+ }
+ }
+ }
+ }
+
+ private case class MonitoredResource(
+ resourceId: String,
+ stagedResourceOwner: StagedResourcesOwner)
+
+ private case class UnusedMonitoredResource(expiresAt: Long, resource: MonitoredResource)
+}
+
+
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala
new file mode 100644
index 0000000000000..4061bc36764d7
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import com.fasterxml.jackson.core.`type`.TypeReference
+import com.fasterxml.jackson.module.scala.JsonScalaEnumeration
+
+object StagedResourcesOwnerType extends Enumeration {
+ type OwnerType = Value
+ // In more generic scenarios, we might want to be watching Deployments, etc.
+ val Pod = Value
+}
+
+class StagedResourcesOwnerTypeReference extends TypeReference[StagedResourcesOwnerType.type]
+
+case class StagedResourcesOwner(
+ ownerNamespace: String,
+ ownerLabels: Map[String, String],
+ @JsonScalaEnumeration(classOf[StagedResourcesOwnerTypeReference])
+ ownerType: StagedResourcesOwnerType.OwnerType)
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala
new file mode 100644
index 0000000000000..0c0d428e035dc
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{File, FileOutputStream, InputStream, IOException}
+import java.security.SecureRandom
+import java.util.UUID
+
+import com.google.common.io.{BaseEncoding, ByteStreams}
+import org.apache.commons.io.FileUtils
+import scala.collection.concurrent.TrieMap
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+
+private[spark] trait StagedResourcesStore {
+
+ /**
+ * Store the given stream on disk and return its resource ID and secret.
+ */
+ def addResources(
+ podNamespace: String,
+ resources: InputStream): StagedResources
+
+ /**
+ * Retrieve a resource bundle with the given id. Returns empty if no resources match this id.
+ */
+ def getResources(resourceId: String): Option[StagedResources]
+
+ def removeResources(resourceId: String): Unit
+}
+
+private[spark] class StagedResourcesStoreImpl(dependenciesRootDir: File)
+ extends StagedResourcesStore with Logging {
+
+ private val SECURE_RANDOM = new SecureRandom()
+ private val stagedResources = TrieMap.empty[String, StagedResources]
+
+ override def addResources(
+ podNamespace: String,
+ resources: InputStream): StagedResources = {
+ val resourceId = UUID.randomUUID().toString
+ val secretBytes = new Array[Byte](1024)
+ SECURE_RANDOM.nextBytes(secretBytes)
+ val resourceSecret = resourceId + "-" + BaseEncoding.base64().encode(secretBytes)
+
+ val namespaceDir = new File(dependenciesRootDir, podNamespace)
+ val resourcesDir = new File(namespaceDir, resourceId)
+ try {
+ if (!resourcesDir.exists()) {
+ if (!resourcesDir.mkdirs()) {
+ throw new SparkException("Failed to create dependencies directory for application" +
+ s" at ${resourcesDir.getAbsolutePath}")
+ }
+ }
+ // TODO encrypt the written data with the secret.
+ val resourcesFile = new File(resourcesDir, "resources.data")
+ Utils.tryWithResource(new FileOutputStream(resourcesFile)) {
+ ByteStreams.copy(resources, _)
+ }
+ val resourceBundle = StagedResources(resourceId, resourceSecret, resourcesFile)
+ stagedResources(resourceId) = resourceBundle
+ resourceBundle
+ } catch {
+ case e: Throwable =>
+ if (!resourcesDir.delete()) {
+ logWarning(s"Failed to delete application directory $resourcesDir.")
+ }
+ stagedResources.remove(resourceId)
+ throw e
+ }
+ }
+
+ override def getResources(resourceId: String): Option[StagedResources] = {
+ stagedResources.get(resourceId)
+ }
+
+ override def removeResources(resourceId: String): Unit = {
+ stagedResources.remove(resourceId)
+ .map(_.resourcesFile.getParentFile)
+ .foreach { resourcesDirectory =>
+ try {
+ FileUtils.deleteDirectory(resourcesDirectory)
+ } catch {
+ case e: IOException =>
+ logWarning(s"Failed to delete resources directory" +
+ s" at ${resourcesDirectory.getAbsolutePath}", e)
+ }
+ }
+ }
+}
+
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala
new file mode 100644
index 0000000000000..fa0ecca3b4ee6
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala
@@ -0,0 +1,101 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import java.io.File
+
+import io.fabric8.kubernetes.client.Config
+
+import org.apache.spark.SparkContext
+import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl}
+
+private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging {
+
+ override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s")
+
+ override def createTaskScheduler(sc: SparkContext, masterURL: String): TaskScheduler = {
+ val scheduler = new KubernetesTaskSchedulerImpl(sc)
+ sc.taskScheduler = scheduler
+ scheduler
+ }
+
+ override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler)
+ : SchedulerBackend = {
+ val sparkConf = sc.getConf
+ val maybeConfigMap = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP)
+ val maybeConfigMapKey = sparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY)
+
+ val maybeExecutorInitContainerSecretName =
+ sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET)
+ val maybeExecutorInitContainerSecretMount =
+ sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR)
+ val executorInitContainerSecretVolumePlugin = for {
+ initContainerSecretName <- maybeExecutorInitContainerSecretName
+ initContainerSecretMountPath <- maybeExecutorInitContainerSecretMount
+ } yield {
+ new InitContainerResourceStagingServerSecretPluginImpl(
+ initContainerSecretName,
+ initContainerSecretMountPath)
+ }
+ // Only set up the bootstrap if they've provided both the config map key and the config map
+ // name. Note that we generally expect both to have been set from spark-submit V2, but for
+ // testing developers may simply run the driver JVM locally, but the config map won't be set
+ // then.
+ val bootStrap = for {
+ configMap <- maybeConfigMap
+ configMapKey <- maybeConfigMapKey
+ } yield {
+ new SparkPodInitContainerBootstrapImpl(
+ sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE),
+ sparkConf.get(DOCKER_IMAGE_PULL_POLICY),
+ sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION),
+ sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION),
+ sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT),
+ configMap,
+ configMapKey)
+ }
+ if (maybeConfigMap.isEmpty) {
+ logWarning("The executor's init-container config map was not specified. Executors will" +
+ " therefore not attempt to fetch remote or submitted dependencies.")
+ }
+ if (maybeConfigMapKey.isEmpty) {
+ logWarning("The executor's init-container config map key was not specified. Executors will" +
+ " therefore not attempt to fetch remote or submitted dependencies.")
+ }
+ val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient(
+ KUBERNETES_MASTER_INTERNAL_URL,
+ Some(sparkConf.get(KUBERNETES_NAMESPACE)),
+ APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX,
+ sparkConf,
+ Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)),
+ Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH)))
+ new KubernetesClusterSchedulerBackend(
+ sc.taskScheduler.asInstanceOf[TaskSchedulerImpl],
+ sc,
+ bootStrap,
+ executorInitContainerSecretVolumePlugin,
+ kubernetesClient)
+ }
+
+ override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = {
+ scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala
new file mode 100644
index 0000000000000..c993bff8df962
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala
@@ -0,0 +1,802 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import java.io.Closeable
+import java.net.InetAddress
+import java.util.Collections
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
+import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference}
+
+import scala.collection.{concurrent, mutable}
+import scala.collection.JavaConverters._
+import scala.concurrent.{ExecutionContext, Future}
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import io.fabric8.kubernetes.api.model._
+import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import org.apache.commons.io.FilenameUtils
+
+import org.apache.spark.{SparkContext, SparkEnv, SparkException}
+import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.InitContainerUtil
+import org.apache.spark.network.netty.SparkTransportConf
+import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient
+import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv}
+import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl}
+import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig}
+import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+private[spark] class KubernetesClusterSchedulerBackend(
+ scheduler: TaskSchedulerImpl,
+ val sc: SparkContext,
+ executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap],
+ executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin],
+ kubernetesClient: KubernetesClient)
+ extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) {
+
+ import KubernetesClusterSchedulerBackend._
+
+ private val RUNNING_EXECUTOR_PODS_LOCK = new Object
+ // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK.
+ private val runningExecutorsToPods = new mutable.HashMap[String, Pod]
+ // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK.
+ private val runningPodsToExecutors = new mutable.HashMap[String, String]
+ // TODO(varun): Get rid of this lock object by my making the underlying map a concurrent hash map.
+ private val EXECUTOR_PODS_BY_IPS_LOCK = new Object
+ // Indexed by executor IP addrs and guarded by EXECUTOR_PODS_BY_IPS_LOCK
+ private val executorPodsByIPs = new mutable.HashMap[String, Pod]
+ private val failedPods: concurrent.Map[String, ExecutorExited] = new
+ ConcurrentHashMap[String, ExecutorExited]().asScala
+ private val executorsToRemove = Collections.newSetFromMap[String](
+ new ConcurrentHashMap[String, java.lang.Boolean]()).asScala
+
+ private val executorExtraClasspath = conf.get(
+ org.apache.spark.internal.config.EXECUTOR_CLASS_PATH)
+ private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION)
+
+ private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf(
+ conf,
+ KUBERNETES_EXECUTOR_LABEL_PREFIX,
+ KUBERNETES_EXECUTOR_LABELS,
+ "executor label")
+ require(
+ !executorLabels.contains(SPARK_APP_ID_LABEL),
+ s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is" +
+ s" reserved for Spark.")
+ require(
+ !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL),
+ s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" +
+ s" Spark.")
+
+ private val executorAnnotations =
+ ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf(
+ conf,
+ KUBERNETES_EXECUTOR_ANNOTATION_PREFIX,
+ KUBERNETES_EXECUTOR_ANNOTATIONS,
+ "executor annotation")
+ private val nodeSelector =
+ ConfigurationUtils.parsePrefixedKeyValuePairs(
+ conf,
+ KUBERNETES_NODE_SELECTOR_PREFIX,
+ "node-selector")
+ private var shufflePodCache: Option[ShufflePodCache] = None
+ private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE)
+ private val dockerImagePullPolicy = conf.get(DOCKER_IMAGE_PULL_POLICY)
+ private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE)
+ private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT)
+ private val blockmanagerPort = conf
+ .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT)
+
+ private val kubernetesDriverPodName = conf
+ .get(KUBERNETES_DRIVER_POD_NAME)
+ .getOrElse(
+ throw new SparkException("Must specify the driver pod name"))
+ private val executorPodNamePrefix = conf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX)
+
+ private val executorMemoryMb = conf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY)
+ private val executorMemoryString = conf.get(
+ org.apache.spark.internal.config.EXECUTOR_MEMORY.key,
+ org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString)
+
+ private val memoryOverheadMb = conf
+ .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD)
+ .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMb).toInt,
+ MEMORY_OVERHEAD_MIN))
+ private val executorMemoryWithOverhead = executorMemoryMb + memoryOverheadMb
+
+ private val executorCores = conf.getDouble("spark.executor.cores", 1d)
+ private val executorLimitCores = conf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key)
+
+ private implicit val requestExecutorContext = ExecutionContext.fromExecutorService(
+ ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests"))
+
+ private val driverPod = try {
+ kubernetesClient.pods().inNamespace(kubernetesNamespace).
+ withName(kubernetesDriverPodName).get()
+ } catch {
+ case throwable: Throwable =>
+ logError(s"Executor cannot find driver pod.", throwable)
+ throw new SparkException(s"Executor cannot find driver pod", throwable)
+ }
+
+ private val shuffleServiceConfig: Option[ShuffleServiceConfig] =
+ if (Utils.isDynamicAllocationEnabled(sc.conf)) {
+ val shuffleNamespace = conf.get(KUBERNETES_SHUFFLE_NAMESPACE)
+ val parsedShuffleLabels = ConfigurationUtils.parseKeyValuePairs(
+ conf.get(KUBERNETES_SHUFFLE_LABELS), KUBERNETES_SHUFFLE_LABELS.key,
+ "shuffle-labels")
+ if (parsedShuffleLabels.isEmpty) {
+ throw new SparkException(s"Dynamic allocation enabled " +
+ s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified")
+ }
+
+ val shuffleDirs = conf.get(KUBERNETES_SHUFFLE_DIR).map {
+ _.split(",")
+ }.getOrElse(Utils.getConfiguredLocalDirs(conf))
+ Some(
+ ShuffleServiceConfig(shuffleNamespace,
+ parsedShuffleLabels,
+ shuffleDirs))
+ } else {
+ None
+ }
+
+ // A client for talking to the external shuffle service
+ private val kubernetesExternalShuffleClient: Option[KubernetesExternalShuffleClient] = {
+ if (Utils.isDynamicAllocationEnabled(sc.conf)) {
+ Some(getShuffleClient())
+ } else {
+ None
+ }
+ }
+
+ override val minRegisteredRatio =
+ if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) {
+ 0.8
+ } else {
+ super.minRegisteredRatio
+ }
+
+ private val executorWatchResource = new AtomicReference[Closeable]
+ protected var totalExpectedExecutors = new AtomicInteger(0)
+
+
+ private val driverUrl = RpcEndpointAddress(
+ sc.getConf.get("spark.driver.host"),
+ sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT),
+ CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString
+
+ private val initialExecutors = getInitialTargetExecutorNumber()
+
+ private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY)
+ require(podAllocationInterval > 0, s"Allocation batch delay " +
+ s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " +
+ s"is ${podAllocationInterval}, should be a positive integer")
+
+ private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE)
+ require(podAllocationSize > 0, s"Allocation batch size " +
+ s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " +
+ s"is ${podAllocationSize}, should be a positive integer")
+
+ private val allocator = ThreadUtils
+ .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator")
+
+ private val allocatorRunnable: Runnable = new Runnable {
+
+ // Number of times we are allowed check for the loss reason for an executor before we give up
+ // and assume the executor failed for good, and attribute it to a framework fault.
+ private val MAX_EXECUTOR_LOST_REASON_CHECKS = 10
+ private val executorsToRecover = new mutable.HashSet[String]
+ // Maintains a map of executor id to count of checks performed to learn the loss reason
+ // for an executor.
+ private val executorReasonChecks = new mutable.HashMap[String, Int]
+
+ override def run(): Unit = {
+ removeFailedExecutors()
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) {
+ logDebug("Waiting for pending executors before scaling")
+ } else if (totalExpectedExecutors.get() <= runningExecutorsToPods.size) {
+ logDebug("Maximum allowed executor limit reached. Not scaling up further.")
+ } else {
+ val nodeToLocalTaskCount = getNodesWithLocalTaskCounts
+ for (i <- 0 until math.min(
+ totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) {
+ val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount)
+ runningExecutorsToPods.put(executorId, pod)
+ runningPodsToExecutors.put(pod.getMetadata.getName, executorId)
+ logInfo(
+ s"Requesting a new executor, total executors is now ${runningExecutorsToPods.size}")
+ }
+ }
+ }
+ }
+
+ def removeFailedExecutors(): Unit = {
+ val localRunningExecutorsToPods = RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ runningExecutorsToPods.toMap
+ }
+ executorsToRemove.foreach { case (executorId) =>
+ localRunningExecutorsToPods.get(executorId).map { pod: Pod =>
+ failedPods.get(pod.getMetadata.getName).map { executorExited: ExecutorExited =>
+ logDebug(s"Removing executor $executorId with loss reason " + executorExited.message)
+ removeExecutor(executorId, executorExited)
+ if (!executorExited.exitCausedByApp) {
+ executorsToRecover.add(executorId)
+ }
+ }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId))
+ }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId))
+
+ executorsToRecover.foreach(executorId => {
+ executorsToRemove -= executorId
+ executorReasonChecks -= executorId
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ runningExecutorsToPods.remove(executorId).map { pod: Pod =>
+ kubernetesClient.pods().delete(pod)
+ runningPodsToExecutors.remove(pod.getMetadata.getName)
+ }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId"))
+ }
+ })
+ executorsToRecover.clear()
+ }
+ }
+
+ def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = {
+ val reasonCheckCount = executorReasonChecks.getOrElse(executorId, 0)
+ if (reasonCheckCount > MAX_EXECUTOR_LOST_REASON_CHECKS) {
+ removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons"))
+ executorsToRecover.add(executorId)
+ executorReasonChecks -= executorId
+ } else {
+ executorReasonChecks.put(executorId, reasonCheckCount + 1)
+ }
+ }
+ }
+
+ private val objectMapper = new ObjectMapper().registerModule(DefaultScalaModule)
+
+ private def getShuffleClient(): KubernetesExternalShuffleClient = {
+ new KubernetesExternalShuffleClient(
+ SparkTransportConf.fromSparkConf(conf, "shuffle"),
+ sc.env.securityManager,
+ sc.env.securityManager.isAuthenticationEnabled(),
+ sc.env.securityManager.isSaslEncryptionEnabled())
+ }
+
+ private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = {
+ if (Utils.isDynamicAllocationEnabled(conf)) {
+ val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0)
+ val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf)
+ val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", 1)
+ require(initialNumExecutors >= minNumExecutors && initialNumExecutors <= maxNumExecutors,
+ s"initial executor number $initialNumExecutors must between min executor number " +
+ s"$minNumExecutors and max executor number $maxNumExecutors")
+
+ initialNumExecutors
+ } else {
+ conf.getInt("spark.executor.instances", defaultNumExecutors)
+ }
+
+ }
+
+ override def applicationId(): String = conf.get("spark.app.id", super.applicationId())
+
+ override def sufficientResourcesRegistered(): Boolean = {
+ totalRegisteredExecutors.get() >= initialExecutors * minRegisteredRatio
+ }
+
+ override def start(): Unit = {
+ super.start()
+ executorWatchResource.set(
+ kubernetesClient
+ .pods()
+ .withLabel(SPARK_APP_ID_LABEL, applicationId())
+ .watch(new ExecutorPodsWatcher()))
+
+ allocator.scheduleWithFixedDelay(
+ allocatorRunnable, 0, podAllocationInterval, TimeUnit.SECONDS)
+
+ if (!Utils.isDynamicAllocationEnabled(sc.conf)) {
+ doRequestTotalExecutors(initialExecutors)
+ } else {
+ shufflePodCache = shuffleServiceConfig
+ .map { config => new ShufflePodCache(
+ kubernetesClient, config.shuffleNamespace, config.shuffleLabels) }
+ shufflePodCache.foreach(_.start())
+ kubernetesExternalShuffleClient.foreach(_.init(applicationId()))
+ }
+ }
+
+ override def stop(): Unit = {
+ // stop allocation of new resources and caches.
+ allocator.shutdown()
+ shufflePodCache.foreach(_.stop())
+ kubernetesExternalShuffleClient.foreach(_.close())
+
+ // send stop message to executors so they shut down cleanly
+ super.stop()
+
+ // then delete the executor pods
+ // TODO investigate why Utils.tryLogNonFatalError() doesn't work in this context.
+ // When using Utils.tryLogNonFatalError some of the code fails but without any logs or
+ // indication as to why.
+ try {
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ runningExecutorsToPods.values.foreach(kubernetesClient.pods().delete(_))
+ runningExecutorsToPods.clear()
+ runningPodsToExecutors.clear()
+ }
+ EXECUTOR_PODS_BY_IPS_LOCK.synchronized {
+ executorPodsByIPs.clear()
+ }
+ val resource = executorWatchResource.getAndSet(null)
+ if (resource != null) {
+ resource.close()
+ }
+ } catch {
+ case e: Throwable => logError("Uncaught exception while shutting down controllers.", e)
+ }
+ try {
+ logInfo("Closing kubernetes client")
+ kubernetesClient.close()
+ } catch {
+ case e: Throwable => logError("Uncaught exception closing Kubernetes client.", e)
+ }
+ }
+
+ /**
+ * @return A map of K8s cluster nodes to the number of tasks that could benefit from data
+ * locality if an executor launches on the cluster node.
+ */
+ private def getNodesWithLocalTaskCounts() : Map[String, Int] = {
+ val executorPodsWithIPs = EXECUTOR_PODS_BY_IPS_LOCK.synchronized {
+ executorPodsByIPs.values.toList // toList makes a defensive copy.
+ }
+ val nodeToLocalTaskCount = mutable.Map[String, Int]() ++
+ KubernetesClusterSchedulerBackend.this.synchronized {
+ hostToLocalTaskCount
+ }
+ for (pod <- executorPodsWithIPs) {
+ // Remove cluster nodes that are running our executors already.
+ // TODO: This prefers spreading out executors across nodes. In case users want
+ // consolidating executors on fewer nodes, introduce a flag. See the spark.deploy.spreadOut
+ // flag that Spark standalone has: https://spark.apache.org/docs/latest/spark-standalone.html
+ nodeToLocalTaskCount.remove(pod.getSpec.getNodeName).nonEmpty ||
+ nodeToLocalTaskCount.remove(pod.getStatus.getHostIP).nonEmpty ||
+ nodeToLocalTaskCount.remove(
+ InetAddress.getByName(pod.getStatus.getHostIP).getCanonicalHostName).nonEmpty
+ }
+ nodeToLocalTaskCount.toMap[String, Int]
+ }
+
+ private def addNodeAffinityAnnotationIfUseful(
+ baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = {
+ def scaleToRange(value: Int, baseMin: Double, baseMax: Double,
+ rangeMin: Double, rangeMax: Double): Int =
+ (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt
+
+ if (nodeToTaskCount.nonEmpty) {
+ val taskTotal = nodeToTaskCount.foldLeft(0)(_ + _._2)
+ // Normalize to node affinity weights in 1 to 100 range.
+ val nodeToWeight = nodeToTaskCount.map{
+ case (node, taskCount) =>
+ (node, scaleToRange(taskCount, 1, taskTotal, rangeMin = 1, rangeMax = 100))}
+ val weightToNodes = nodeToWeight.groupBy(_._2).mapValues(_.keys)
+ // @see https://kubernetes.io/docs/concepts/configuration/assign-pod-node
+ val nodeAffinityJson = objectMapper.writeValueAsString(SchedulerAffinity(NodeAffinity(
+ preferredDuringSchedulingIgnoredDuringExecution =
+ for ((weight, nodes) <- weightToNodes) yield
+ WeightedPreference(weight,
+ Preference(Array(MatchExpression("kubernetes.io/hostname", "In", nodes))))
+ )))
+ // TODO: Use non-annotation syntax when we switch to K8s version 1.6.
+ logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson")
+ new PodBuilder(baseExecutorPod).editMetadata()
+ .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson)
+ .endMetadata()
+ .build()
+ } else {
+ baseExecutorPod
+ }
+ }
+
+ /**
+ * Allocates a new executor pod
+ *
+ * @param nodeToLocalTaskCount A map of K8s cluster nodes to the number of tasks that could
+ * benefit from data locality if an executor launches on the cluster
+ * node.
+ * @return A tuple of the new executor name and the Pod data structure.
+ */
+ private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = {
+ val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString
+ val name = s"$executorPodNamePrefix-exec-$executorId"
+
+ // hostname must be no longer than 63 characters, so take the last 63 characters of the pod
+ // name as the hostname. This preserves uniqueness since the end of name contains
+ // executorId and applicationId
+ val hostname = name.substring(Math.max(0, name.length - 63))
+ val resolvedExecutorLabels = Map(
+ SPARK_EXECUTOR_ID_LABEL -> executorId,
+ SPARK_APP_ID_LABEL -> applicationId(),
+ SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++
+ executorLabels
+ val executorMemoryQuantity = new QuantityBuilder(false)
+ .withAmount(s"${executorMemoryMb}M")
+ .build()
+ val executorMemoryLimitQuantity = new QuantityBuilder(false)
+ .withAmount(s"${executorMemoryWithOverhead}M")
+ .build()
+ val executorCpuQuantity = new QuantityBuilder(false)
+ .withAmount(executorCores.toString)
+ .build()
+ val executorExtraClasspathEnv = executorExtraClasspath.map { cp =>
+ new EnvVarBuilder()
+ .withName(ENV_EXECUTOR_EXTRA_CLASSPATH)
+ .withValue(cp)
+ .build()
+ }
+ val requiredEnv = Seq(
+ (ENV_EXECUTOR_PORT, executorPort.toString),
+ (ENV_DRIVER_URL, driverUrl),
+ // Executor backend expects integral value for executor cores, so round it up to an int.
+ (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString),
+ (ENV_EXECUTOR_MEMORY, executorMemoryString),
+ (ENV_APPLICATION_ID, applicationId()),
+ (ENV_EXECUTOR_ID, executorId),
+ (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*"))
+ .map(env => new EnvVarBuilder()
+ .withName(env._1)
+ .withValue(env._2)
+ .build()
+ ) ++ Seq(
+ new EnvVarBuilder()
+ .withName(ENV_EXECUTOR_POD_IP)
+ .withValueFrom(new EnvVarSourceBuilder()
+ .withNewFieldRef("v1", "status.podIP")
+ .build())
+ .build()
+ )
+ val requiredPorts = Seq(
+ (EXECUTOR_PORT_NAME, executorPort),
+ (BLOCK_MANAGER_PORT_NAME, blockmanagerPort))
+ .map(port => {
+ new ContainerPortBuilder()
+ .withName(port._1)
+ .withContainerPort(port._2)
+ .build()
+ })
+
+ val executorContainer = new ContainerBuilder()
+ .withName(s"executor")
+ .withImage(executorDockerImage)
+ .withImagePullPolicy(dockerImagePullPolicy)
+ .withNewResources()
+ .addToRequests("memory", executorMemoryQuantity)
+ .addToLimits("memory", executorMemoryLimitQuantity)
+ .addToRequests("cpu", executorCpuQuantity)
+ .endResources()
+ .addAllToEnv(requiredEnv.asJava)
+ .addToEnv(executorExtraClasspathEnv.toSeq: _*)
+ .withPorts(requiredPorts.asJava)
+ .build()
+
+ val executorPod = new PodBuilder()
+ .withNewMetadata()
+ .withName(name)
+ .withLabels(resolvedExecutorLabels.asJava)
+ .withAnnotations(executorAnnotations.asJava)
+ .withOwnerReferences()
+ .addNewOwnerReference()
+ .withController(true)
+ .withApiVersion(driverPod.getApiVersion)
+ .withKind(driverPod.getKind)
+ .withName(driverPod.getMetadata.getName)
+ .withUid(driverPod.getMetadata.getUid)
+ .endOwnerReference()
+ .endMetadata()
+ .withNewSpec()
+ .withHostname(hostname)
+ .withRestartPolicy("Never")
+ .withNodeSelector(nodeSelector.asJava)
+ .endSpec()
+ .build()
+
+ val containerWithExecutorLimitCores = executorLimitCores.map {
+ limitCores =>
+ val executorCpuLimitQuantity = new QuantityBuilder(false)
+ .withAmount(limitCores)
+ .build()
+ new ContainerBuilder(executorContainer)
+ .editResources()
+ .addToLimits("cpu", executorCpuLimitQuantity)
+ .endResources()
+ .build()
+ }.getOrElse(executorContainer)
+
+ val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config =>
+ config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) =>
+ new ContainerBuilder(container)
+ .addNewVolumeMount()
+ .withName(FilenameUtils.getBaseName(dir))
+ .withMountPath(dir)
+ .endVolumeMount()
+ .build()
+ }
+ }.getOrElse(containerWithExecutorLimitCores)
+ val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config =>
+ config.shuffleDirs.foldLeft(executorPod) { (builder, dir) =>
+ new PodBuilder(builder)
+ .editSpec()
+ .addNewVolume()
+ .withName(FilenameUtils.getBaseName(dir))
+ .withNewHostPath()
+ .withPath(dir)
+ .endHostPath()
+ .endVolume()
+ .endSpec()
+ .build()
+ }
+ }.getOrElse(executorPod)
+ val (executorPodWithInitContainer, initBootstrappedExecutorContainer) =
+ executorInitContainerBootstrap.map { bootstrap =>
+ val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes(
+ PodWithDetachedInitContainer(
+ withMaybeShuffleConfigPod,
+ new ContainerBuilder().build(),
+ withMaybeShuffleConfigExecutorContainer))
+
+ val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin =>
+ plugin.mountResourceStagingServerSecretIntoInitContainer(
+ podWithDetachedInitContainer.initContainer)
+ }.getOrElse(podWithDetachedInitContainer.initContainer)
+
+ val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer(
+ podWithDetachedInitContainer.pod, resolvedInitContainer)
+
+ val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin =>
+ plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer)
+ }.getOrElse(podWithAttachedInitContainer)
+
+ (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer)
+ }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer))
+
+ val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful(
+ executorPodWithInitContainer, nodeToLocalTaskCount)
+ val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity)
+ .editSpec()
+ .addToContainers(initBootstrappedExecutorContainer)
+ .endSpec()
+ .build()
+ try {
+ (executorId, kubernetesClient.pods.create(resolvedExecutorPod))
+ } catch {
+ case throwable: Throwable =>
+ logError("Failed to allocate executor pod.", throwable)
+ throw throwable
+ }
+ }
+
+ override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future[Boolean] {
+ totalExpectedExecutors.set(requestedTotal)
+ true
+ }
+
+ override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] {
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ for (executor <- executorIds) {
+ runningExecutorsToPods.remove(executor) match {
+ case Some(pod) =>
+ kubernetesClient.pods().delete(pod)
+ runningPodsToExecutors.remove(pod.getMetadata.getName)
+ case None => logWarning(s"Unable to remove pod for unknown executor $executor")
+ }
+ }
+ }
+ true
+ }
+
+ def getExecutorPodByIP(podIP: String): Option[Pod] = {
+ EXECUTOR_PODS_BY_IPS_LOCK.synchronized {
+ executorPodsByIPs.get(podIP)
+ }
+ }
+
+ private class ExecutorPodsWatcher extends Watcher[Pod] {
+
+ private val DEFAULT_CONTAINER_FAILURE_EXIT_STATUS = -1
+
+ override def eventReceived(action: Action, pod: Pod): Unit = {
+ if (action == Action.MODIFIED && pod.getStatus.getPhase == "Running"
+ && pod.getMetadata.getDeletionTimestamp == null) {
+ val podIP = pod.getStatus.getPodIP
+ val clusterNodeName = pod.getSpec.getNodeName
+ logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.")
+ EXECUTOR_PODS_BY_IPS_LOCK.synchronized {
+ executorPodsByIPs += ((podIP, pod))
+ }
+ } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) ||
+ action == Action.DELETED || action == Action.ERROR) {
+ val podName = pod.getMetadata.getName
+ val podIP = pod.getStatus.getPodIP
+ logDebug(s"Executor pod $podName at IP $podIP was at $action.")
+ if (podIP != null) {
+ EXECUTOR_PODS_BY_IPS_LOCK.synchronized {
+ executorPodsByIPs -= podIP
+ }
+ }
+ if (action == Action.ERROR) {
+ logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason)
+ handleErroredPod(pod)
+ } else if (action == Action.DELETED) {
+ logInfo(s"Received delete pod $podName event. Reason: " + pod.getStatus.getReason)
+ handleDeletedPod(pod)
+ }
+ }
+ }
+
+ override def onClose(cause: KubernetesClientException): Unit = {
+ logDebug("Executor pod watch closed.", cause)
+ }
+
+ def getExecutorExitStatus(pod: Pod): Int = {
+ val containerStatuses = pod.getStatus.getContainerStatuses
+ if (!containerStatuses.isEmpty) {
+ // we assume the first container represents the pod status. This assumption may not hold
+ // true in the future. Revisit this if side-car containers start running inside executor
+ // pods.
+ getExecutorExitStatus(containerStatuses.get(0))
+ } else DEFAULT_CONTAINER_FAILURE_EXIT_STATUS
+ }
+
+ def getExecutorExitStatus(containerStatus: ContainerStatus): Int = {
+ Option(containerStatus.getState).map(containerState =>
+ Option(containerState.getTerminated).map(containerStateTerminated =>
+ containerStateTerminated.getExitCode.intValue()).getOrElse(UNKNOWN_EXIT_CODE)
+ ).getOrElse(UNKNOWN_EXIT_CODE)
+ }
+
+ def isPodAlreadyReleased(pod: Pod): Boolean = {
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ !runningPodsToExecutors.contains(pod.getMetadata.getName)
+ }
+ }
+
+ def handleErroredPod(pod: Pod): Unit = {
+ val alreadyReleased = isPodAlreadyReleased(pod)
+ val containerExitStatus = getExecutorExitStatus(pod)
+ // container was probably actively killed by the driver.
+ val exitReason = if (alreadyReleased) {
+ ExecutorExited(containerExitStatus, exitCausedByApp = false,
+ s"Container in pod " + pod.getMetadata.getName +
+ " exited from explicit termination request.")
+ } else {
+ val containerExitReason = containerExitStatus match {
+ case VMEM_EXCEEDED_EXIT_CODE | PMEM_EXCEEDED_EXIT_CODE =>
+ memLimitExceededLogMessage(pod.getStatus.getReason)
+ case _ =>
+ // Here we can't be sure that that exit was caused by the application but this seems
+ // to be the right default since we know the pod was not explicitly deleted by
+ // the user.
+ "Pod exited with following container exit status code " + containerExitStatus
+ }
+ ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason)
+ }
+ failedPods.put(pod.getMetadata.getName, exitReason)
+ }
+
+ def handleDeletedPod(pod: Pod): Unit = {
+ val exitReason = ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false,
+ "Pod " + pod.getMetadata.getName + " deleted or lost.")
+ failedPods.put(pod.getMetadata.getName, exitReason)
+ }
+ }
+
+ override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = {
+ new KubernetesDriverEndpoint(rpcEnv, properties)
+ }
+
+ private class KubernetesDriverEndpoint(
+ rpcEnv: RpcEnv,
+ sparkProperties: Seq[(String, String)])
+ extends DriverEndpoint(rpcEnv, sparkProperties) {
+ private val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337)
+
+ override def onDisconnected(rpcAddress: RpcAddress): Unit = {
+ addressToExecutorId.get(rpcAddress).foreach { executorId =>
+ if (disableExecutor(executorId)) {
+ executorsToRemove.add(executorId)
+ }
+ }
+ }
+
+ override def receiveAndReply(
+ context: RpcCallContext): PartialFunction[Any, Unit] = {
+ new PartialFunction[Any, Unit]() {
+ override def isDefinedAt(msg: Any): Boolean = {
+ msg match {
+ case RetrieveSparkAppConfig(executorId) =>
+ Utils.isDynamicAllocationEnabled(sc.conf)
+ case _ => false
+ }
+ }
+
+ override def apply(msg: Any): Unit = {
+ msg match {
+ case RetrieveSparkAppConfig(executorId) =>
+ RUNNING_EXECUTOR_PODS_LOCK.synchronized {
+ var resolvedProperties = sparkProperties
+ val runningExecutorPod = kubernetesClient
+ .pods()
+ .withName(runningExecutorsToPods(executorId).getMetadata.getName)
+ .get()
+ val nodeName = runningExecutorPod.getSpec.getNodeName
+ val shufflePodIp = shufflePodCache.get.getShufflePodForExecutor(nodeName)
+
+ // Inform the shuffle pod about this application so it can watch.
+ kubernetesExternalShuffleClient.foreach(
+ _.registerDriverWithShuffleService(shufflePodIp, externalShufflePort))
+
+ resolvedProperties = resolvedProperties ++ Seq(
+ (SPARK_SHUFFLE_SERVICE_HOST.key, shufflePodIp))
+
+ val reply = SparkAppConfig(
+ resolvedProperties,
+ SparkEnv.get.securityManager.getIOEncryptionKey())
+ context.reply(reply)
+ }
+ }
+ }
+ }.orElse(super.receiveAndReply(context))
+ }
+ }
+}
+case class ShuffleServiceConfig(
+ shuffleNamespace: String,
+ shuffleLabels: Map[String, String],
+ shuffleDirs: Seq[String])
+
+private object KubernetesClusterSchedulerBackend {
+ private val DEFAULT_STATIC_PORT = 10000
+ private val EXECUTOR_ID_COUNTER = new AtomicLong(0L)
+ private val VMEM_EXCEEDED_EXIT_CODE = -103
+ private val PMEM_EXCEEDED_EXIT_CODE = -104
+ private val UNKNOWN_EXIT_CODE = -111
+
+ def memLimitExceededLogMessage(diagnostics: String): String = {
+ s"Pod/Container killed for exceeding memory limits. $diagnostics" +
+ " Consider boosting spark executor memory overhead."
+ }
+}
+
+/**
+ * These case classes model K8s node affinity syntax for
+ * preferredDuringSchedulingIgnoredDuringExecution.
+ * @see https://kubernetes.io/docs/concepts/configuration/assign-pod-node
+ */
+case class SchedulerAffinity(nodeAffinity: NodeAffinity)
+case class NodeAffinity(preferredDuringSchedulingIgnoredDuringExecution:
+ Iterable[WeightedPreference])
+case class WeightedPreference(weight: Int, preference: Preference)
+case class Preference(matchExpressions: Array[MatchExpression])
+case class MatchExpression(key: String, operator: String, values: Iterable[String])
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala
new file mode 100644
index 0000000000000..a5e126480b83d
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import org.apache.spark.SparkContext
+import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager}
+
+private[spark] class KubernetesTaskSchedulerImpl(sc: SparkContext) extends TaskSchedulerImpl(sc) {
+
+ override def createTaskSetManager(taskSet: TaskSet, maxTaskFailures: Int): TaskSetManager = {
+ new KubernetesTaskSetManager(this, taskSet, maxTaskFailures)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala
new file mode 100644
index 0000000000000..17710fada2876
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import java.net.InetAddress
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager}
+
+private[spark] class KubernetesTaskSetManager(
+ sched: TaskSchedulerImpl,
+ taskSet: TaskSet,
+ maxTaskFailures: Int,
+ inetAddressUtil: InetAddressUtil = new InetAddressUtil)
+ extends TaskSetManager(sched, taskSet, maxTaskFailures) {
+
+ private val conf = sched.sc.conf
+
+ /**
+ * Overrides the lookup to use not only the executor pod IP, but also the cluster node
+ * name and host IP address that the pod is running on. The base class may have populated
+ * the lookup target map with HDFS datanode locations if this task set reads HDFS data.
+ * Those datanode locations are based on cluster node names or host IP addresses. Using
+ * only executor pod IPs may not match them.
+ */
+ override def getPendingTasksForHost(executorIP: String): ArrayBuffer[Int] = {
+ val pendingTasksExecutorIP = super.getPendingTasksForHost(executorIP)
+ if (pendingTasksExecutorIP.nonEmpty) {
+ pendingTasksExecutorIP
+ } else {
+ val backend = sched.backend.asInstanceOf[KubernetesClusterSchedulerBackend]
+ val pod = backend.getExecutorPodByIP(executorIP)
+ if (pod.nonEmpty) {
+ val clusterNodeName = pod.get.getSpec.getNodeName
+ val pendingTasksClusterNodeName = super.getPendingTasksForHost(clusterNodeName)
+ if (pendingTasksClusterNodeName.nonEmpty) {
+ logDebug(s"Got preferred task list $pendingTasksClusterNodeName for executor host " +
+ s"$executorIP using cluster node name $clusterNodeName")
+ pendingTasksClusterNodeName
+ } else {
+ val clusterNodeIP = pod.get.getStatus.getHostIP
+ val pendingTasksClusterNodeIP = super.getPendingTasksForHost(clusterNodeIP)
+ if (pendingTasksClusterNodeIP.nonEmpty) {
+ logDebug(s"Got preferred task list $pendingTasksClusterNodeIP for executor host " +
+ s"$executorIP using cluster node IP $clusterNodeIP")
+ pendingTasksClusterNodeIP
+ } else {
+ if (conf.get(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED)) {
+ val clusterNodeFullName = inetAddressUtil.getFullHostName(clusterNodeIP)
+ val pendingTasksClusterNodeFullName = super.getPendingTasksForHost(
+ clusterNodeFullName)
+ if (pendingTasksClusterNodeFullName.nonEmpty) {
+ logDebug(s"Got preferred task list $pendingTasksClusterNodeFullName " +
+ s"for executor host $executorIP using cluster node full name " +
+ s"$clusterNodeFullName")
+ }
+ pendingTasksClusterNodeFullName
+ } else {
+ pendingTasksExecutorIP // Empty
+ }
+ }
+ }
+ } else {
+ pendingTasksExecutorIP // Empty
+ }
+ }
+ }
+}
+
+// To support mocks in unit tests.
+private[kubernetes] class InetAddressUtil {
+
+ // NOTE: This does issue a network call to DNS. Caching is done internally by the InetAddress
+ // class for both hits and misses.
+ def getFullHostName(ipAddress: String): String = {
+ InetAddress.getByName(ipAddress).getCanonicalHostName
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala
new file mode 100644
index 0000000000000..15e02664589eb
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import scala.collection.JavaConverters._
+
+import io.fabric8.kubernetes.api.model.Pod
+import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import io.fabric8.kubernetes.client.internal.readiness.Readiness
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+
+private[spark] class ShufflePodCache (
+ client: KubernetesClient,
+ dsNamespace: String,
+ dsLabels: Map[String, String]) extends Logging {
+
+ private var shufflePodCache = scala.collection.mutable.Map[String, String]()
+ private var watcher: Watch = _
+
+ def start(): Unit = {
+ // seed the initial cache.
+ val pods = client.pods()
+ .inNamespace(dsNamespace).withLabels(dsLabels.asJava).list()
+ pods.getItems.asScala.foreach {
+ pod =>
+ if (Readiness.isReady(pod)) {
+ addShufflePodToCache(pod)
+ } else {
+ logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " +
+ s"on node ${pod.getSpec.getNodeName}")
+ }
+ }
+
+ watcher = client
+ .pods()
+ .inNamespace(dsNamespace)
+ .withLabels(dsLabels.asJava)
+ .watch(new Watcher[Pod] {
+ override def eventReceived(action: Watcher.Action, p: Pod): Unit = {
+ action match {
+ case Action.DELETED | Action.ERROR =>
+ shufflePodCache.remove(p.getSpec.getNodeName)
+ case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) =>
+ addShufflePodToCache(p)
+ }
+ }
+ override def onClose(e: KubernetesClientException): Unit = {}
+ })
+ }
+
+ private def addShufflePodToCache(pod: Pod): Unit = {
+ if (shufflePodCache.contains(pod.getSpec.getNodeName)) {
+ val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get
+ logError(s"Ambiguous specification of shuffle service pod. " +
+ s"Found multiple matching pods: ${pod.getMetadata.getName}, " +
+ s"${registeredPodName} on ${pod.getSpec.getNodeName}")
+
+ throw new SparkException(s"Ambiguous specification of shuffle service pod. " +
+ s"Found multiple matching pods: ${pod.getMetadata.getName}, " +
+ s"${registeredPodName} on ${pod.getSpec.getNodeName}")
+ } else {
+ shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP
+ }
+ }
+
+ def stop(): Unit = {
+ watcher.close()
+ }
+
+ def getShufflePodForExecutor(executorNode: String): String = {
+ shufflePodCache.get(executorNode)
+ .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode"))
+ }
+}
+
diff --git a/resource-managers/kubernetes/core/src/test/resources/log4j.properties b/resource-managers/kubernetes/core/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000..ad95fadb7c0c0
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/unit-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala
new file mode 100644
index 0000000000000..597bcdb416fc0
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes
+
+import io.fabric8.kubernetes.api.model._
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.SparkFunSuite
+
+class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{
+ private val INIT_CONTAINER_SECRET_NAME = "init-secret"
+ private val INIT_CONTAINER_SECRET_MOUNT = "/tmp/secret"
+
+ private val initContainerSecretPlugin = new InitContainerResourceStagingServerSecretPluginImpl(
+ INIT_CONTAINER_SECRET_NAME,
+ INIT_CONTAINER_SECRET_MOUNT)
+
+ test("Volume Mount into InitContainer") {
+ val returnedCont = initContainerSecretPlugin.mountResourceStagingServerSecretIntoInitContainer(
+ new ContainerBuilder().withName("init-container").build())
+ assert(returnedCont.getName === "init-container")
+ assert(returnedCont.getVolumeMounts.asScala.map(
+ vm => (vm.getName, vm.getMountPath)) ===
+ List((INIT_CONTAINER_SECRET_VOLUME_NAME, INIT_CONTAINER_SECRET_MOUNT)))
+ }
+
+ test("Add Volume with Secret to Pod") {
+ val returnedPod = initContainerSecretPlugin.addResourceStagingServerSecretVolumeToPod(
+ basePod().build)
+ assert(returnedPod.getMetadata.getName === "spark-pod")
+ val volume = returnedPod.getSpec.getVolumes.asScala.head
+ assert(volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME)
+ assert(volume.getSecret.getSecretName === INIT_CONTAINER_SECRET_NAME)
+ }
+ private def basePod(): PodBuilder = {
+ new PodBuilder()
+ .withNewMetadata()
+ .withName("spark-pod")
+ .endMetadata()
+ .withNewSpec()
+ .endSpec()
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala
new file mode 100644
index 0000000000000..8de0f56f007dc
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes
+
+import java.io.{File, FileOutputStream, OutputStreamWriter}
+import java.math.BigInteger
+import java.nio.file.Files
+import java.security.{KeyPair, KeyPairGenerator, KeyStore, SecureRandom}
+import java.security.cert.X509Certificate
+import java.util.{Calendar, Random}
+import javax.security.auth.x500.X500Principal
+
+import com.google.common.base.Charsets
+import org.bouncycastle.asn1.x509.{Extension, GeneralName, GeneralNames}
+import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3CertificateBuilder}
+import org.bouncycastle.openssl.jcajce.JcaPEMWriter
+import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder
+
+import org.apache.spark.deploy.kubernetes.submit.{KeyAndCertPem, KeyStoreAndTrustStore}
+import org.apache.spark.util.Utils
+
+private[spark] object SSLUtils {
+
+ def generateKeyStoreTrustStorePair(
+ ipAddress: String,
+ keyStorePassword: String,
+ keyPassword: String,
+ trustStorePassword: String): KeyStoreAndTrustStore = {
+ val keyPairGenerator = KeyPairGenerator.getInstance("RSA")
+ keyPairGenerator.initialize(512)
+ val keyPair = keyPairGenerator.generateKeyPair()
+ val certificate = generateCertificate(ipAddress, keyPair)
+ val keyStore = KeyStore.getInstance("JKS")
+ keyStore.load(null, null)
+ keyStore.setKeyEntry("key", keyPair.getPrivate,
+ keyPassword.toCharArray, Array(certificate))
+ val tempDir = Files.createTempDirectory("temp-ssl-stores").toFile
+ tempDir.deleteOnExit()
+ val keyStoreFile = new File(tempDir, "keyStore.jks")
+ Utils.tryWithResource(new FileOutputStream(keyStoreFile)) {
+ keyStore.store(_, keyStorePassword.toCharArray)
+ }
+ val trustStore = KeyStore.getInstance("JKS")
+ trustStore.load(null, null)
+ trustStore.setCertificateEntry("key", certificate)
+ val trustStoreFile = new File(tempDir, "trustStore.jks")
+ Utils.tryWithResource(new FileOutputStream(trustStoreFile)) {
+ trustStore.store(_, trustStorePassword.toCharArray)
+ }
+ KeyStoreAndTrustStore(keyStoreFile, trustStoreFile)
+ }
+
+ def generateKeyCertPemPair(ipAddress: String): KeyAndCertPem = {
+ val keyPairGenerator = KeyPairGenerator.getInstance("RSA")
+ keyPairGenerator.initialize(512)
+ val keyPair = keyPairGenerator.generateKeyPair()
+ val certificate = generateCertificate(ipAddress, keyPair)
+ val tempDir = Files.createTempDirectory("temp-ssl-pems").toFile
+ tempDir.deleteOnExit()
+ val keyPemFile = new File(tempDir, "key.pem")
+ val certPemFile = new File(tempDir, "cert.pem")
+ Utils.tryWithResource(new FileOutputStream(keyPemFile)) { keyPemStream =>
+ Utils.tryWithResource(
+ new OutputStreamWriter(keyPemStream, Charsets.UTF_8)) { streamWriter =>
+ Utils.tryWithResource(
+ new JcaPEMWriter(streamWriter)) { pemWriter =>
+ pemWriter.writeObject(keyPair.getPrivate)
+ }
+ }
+ }
+ Utils.tryWithResource(new FileOutputStream(certPemFile)) { keyPemStream =>
+ Utils.tryWithResource(
+ new OutputStreamWriter(keyPemStream, Charsets.UTF_8)) { streamWriter =>
+ Utils.tryWithResource(
+ new JcaPEMWriter(streamWriter)) { pemWriter =>
+ pemWriter.writeObject(certificate)
+ }
+ }
+ }
+ KeyAndCertPem(keyPemFile, certPemFile)
+ }
+
+ private def generateCertificate(ipAddress: String, keyPair: KeyPair): X509Certificate = {
+ val selfPrincipal = new X500Principal(s"cn=$ipAddress")
+ val currentDate = Calendar.getInstance
+ val validForOneHundredYears = Calendar.getInstance
+ validForOneHundredYears.add(Calendar.YEAR, 100)
+ val certificateBuilder = new JcaX509v3CertificateBuilder(
+ selfPrincipal,
+ new BigInteger(4096, new Random()),
+ currentDate.getTime,
+ validForOneHundredYears.getTime,
+ selfPrincipal,
+ keyPair.getPublic)
+ certificateBuilder.addExtension(Extension.subjectAlternativeName, false,
+ new GeneralNames(new GeneralName(GeneralName.iPAddress, ipAddress)))
+ val signer = new JcaContentSignerBuilder("SHA1WithRSA")
+ .setSecureRandom(new SecureRandom())
+ .build(keyPair.getPrivate)
+ val bcCertificate = certificateBuilder.build(signer)
+ new JcaX509CertificateConverter().getCertificate(bcCertificate)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala
new file mode 100644
index 0000000000000..d5f25983f5080
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes
+
+import io.fabric8.kubernetes.api.model._
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.SparkFunSuite
+
+class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter {
+ private val INIT_CONTAINER_IMAGE = "spark-init:latest"
+ private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent"
+ private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars"
+ private val FILES_DOWNLOAD_PATH = "/var/data/spark-files"
+ private val DOWNLOAD_TIMEOUT_MINUTES = 5
+ private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map"
+ private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key"
+ private val MAIN_CONTAINER_NAME = "spark-main"
+
+ private val sparkPodInit = new SparkPodInitContainerBootstrapImpl(
+ INIT_CONTAINER_IMAGE,
+ DOCKER_IMAGE_PULL_POLICY,
+ JARS_DOWNLOAD_PATH,
+ FILES_DOWNLOAD_PATH,
+ DOWNLOAD_TIMEOUT_MINUTES,
+ INIT_CONTAINER_CONFIG_MAP_NAME,
+ INIT_CONTAINER_CONFIG_MAP_KEY)
+ private val expectedSharedVolumeMap = Map(
+ JARS_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME,
+ FILES_DOWNLOAD_PATH -> INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
+
+ test("InitContainer: Volume mounts, args, and builder specs") {
+ val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes(
+ PodWithDetachedInitContainer(
+ pod = basePod().build(),
+ initContainer = new Container(),
+ mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build()))
+ val initContainer: Container = returnedPodWithCont.initContainer
+ val volumes = initContainer.getVolumeMounts.asScala
+ assert(volumes.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap
+ ++ Map("/etc/spark-init" -> "spark-init-properties"))
+ assert(initContainer.getName === "spark-init")
+ assert(initContainer.getImage === INIT_CONTAINER_IMAGE)
+ assert(initContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY)
+ assert(initContainer.getArgs.asScala.head === INIT_CONTAINER_PROPERTIES_FILE_PATH)
+ }
+ test("Main: Volume mounts and env") {
+ val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes(
+ PodWithDetachedInitContainer(
+ pod = basePod().build(),
+ initContainer = new Container(),
+ mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build()))
+ val mainContainer: Container = returnedPodWithCont.mainContainer
+ assert(mainContainer.getName === MAIN_CONTAINER_NAME)
+ val volumeMounts = mainContainer.getVolumeMounts.asScala
+ assert(volumeMounts.map(vm => (vm.getMountPath, vm.getName)).toMap === expectedSharedVolumeMap)
+ assert(mainContainer.getEnv.asScala.map(e => (e.getName, e.getValue)).toMap ===
+ Map(ENV_MOUNTED_FILES_DIR -> FILES_DOWNLOAD_PATH))
+ }
+ test("Pod: Volume Mounts") {
+ val returnedPodWithCont = sparkPodInit.bootstrapInitContainerAndVolumes(
+ PodWithDetachedInitContainer(
+ pod = basePod().build(),
+ initContainer = new Container(),
+ mainContainer = new ContainerBuilder().withName(MAIN_CONTAINER_NAME).build()))
+ val returnedPod = returnedPodWithCont.pod
+ assert(returnedPod.getMetadata.getName === "spark-pod")
+ val volumes = returnedPod.getSpec.getVolumes.asScala.toList
+ assert(volumes.head.getName === INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
+ assert(volumes.head.getConfigMap.getName === INIT_CONTAINER_CONFIG_MAP_NAME)
+ assert(volumes.head.getConfigMap.getItems.asScala.map(
+ i => (i.getKey, i.getPath)) ===
+ List((INIT_CONTAINER_CONFIG_MAP_KEY, INIT_CONTAINER_PROPERTIES_FILE_NAME)))
+ assert(volumes(1).getName === INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
+ assert(volumes(1).getEmptyDir === new EmptyDirVolumeSource())
+ assert(volumes(2).getName === INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
+ assert(volumes(2).getEmptyDir === new EmptyDirVolumeSource())
+ }
+
+ private def basePod(): PodBuilder = {
+ new PodBuilder()
+ .withNewMetadata()
+ .withName("spark-pod")
+ .endMetadata()
+ .withNewSpec()
+ .endSpec()
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala
new file mode 100644
index 0000000000000..965ee75c248b8
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit
+
+import com.google.common.collect.Iterables
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder}
+import io.fabric8.kubernetes.client.{KubernetesClient, Watch}
+import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, NamespaceVisitFromServerGetWatchDeleteRecreateWaitApplicable, PodResource, Resource}
+import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations}
+import org.mockito.Mockito.{doReturn, verify, when}
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.scalatest.BeforeAndAfter
+import org.scalatest.mock.MockitoSugar._
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec}
+
+class ClientSuite extends SparkFunSuite with BeforeAndAfter {
+
+ private val DRIVER_POD_UID = "pod-id"
+ private val DRIVER_POD_API_VERSION = "v1"
+ private val DRIVER_POD_KIND = "pod"
+
+ private type ResourceList = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[
+ HasMetadata, Boolean]
+ private type Pods = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
+
+ @Mock
+ private var kubernetesClient: KubernetesClient = _
+
+ @Mock
+ private var podOperations: Pods = _
+
+ @Mock
+ private var namedPods: PodResource[Pod, DoneablePod] = _
+
+ @Mock
+ private var loggingPodStatusWatcher: LoggingPodStatusWatcher = _
+
+ @Mock
+ private var resourceList: ResourceList = _
+
+ private val submissionSteps = Seq(FirstTestConfigurationStep, SecondTestConfigurationStep)
+ private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _
+ private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _
+
+ before {
+ MockitoAnnotations.initMocks(this)
+ when(kubernetesClient.pods()).thenReturn(podOperations)
+ when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods)
+
+ createdPodArgumentCaptor = ArgumentCaptor.forClass(classOf[Pod])
+ createdResourcesArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata])
+ when(podOperations.create(createdPodArgumentCaptor.capture())).thenAnswer(new Answer[Pod] {
+ override def answer(invocation: InvocationOnMock): Pod = {
+ new PodBuilder(invocation.getArgumentAt(0, classOf[Pod]))
+ .editMetadata()
+ .withUid(DRIVER_POD_UID)
+ .endMetadata()
+ .withApiVersion(DRIVER_POD_API_VERSION)
+ .withKind(DRIVER_POD_KIND)
+ .build()
+ }
+ })
+ when(podOperations.withName(FirstTestConfigurationStep.podName)).thenReturn(namedPods)
+ when(namedPods.watch(loggingPodStatusWatcher)).thenReturn(mock[Watch])
+ doReturn(resourceList)
+ .when(kubernetesClient)
+ .resourceList(createdResourcesArgumentCaptor.capture())
+ }
+
+ test("The client should configure the pod with the submission steps.") {
+ val submissionClient = new Client(
+ submissionSteps,
+ new SparkConf(false),
+ kubernetesClient,
+ false,
+ "spark",
+ loggingPodStatusWatcher)
+ submissionClient.run()
+ val createdPod = createdPodArgumentCaptor.getValue
+ assert(createdPod.getMetadata.getName === FirstTestConfigurationStep.podName)
+ assert(createdPod.getMetadata.getLabels.asScala ===
+ Map(FirstTestConfigurationStep.labelKey -> FirstTestConfigurationStep.labelValue))
+ assert(createdPod.getMetadata.getAnnotations.asScala ===
+ Map(SecondTestConfigurationStep.annotationKey ->
+ SecondTestConfigurationStep.annotationValue))
+ assert(createdPod.getSpec.getContainers.size() === 1)
+ assert(createdPod.getSpec.getContainers.get(0).getName ===
+ SecondTestConfigurationStep.containerName)
+ }
+
+ test("The client should create the secondary Kubernetes resources.") {
+ val submissionClient = new Client(
+ submissionSteps,
+ new SparkConf(false),
+ kubernetesClient,
+ false,
+ "spark",
+ loggingPodStatusWatcher)
+ submissionClient.run()
+ val createdPod = createdPodArgumentCaptor.getValue
+ val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues
+ assert(otherCreatedResources.size === 1)
+ val createdResource = Iterables.getOnlyElement(otherCreatedResources).asInstanceOf[Secret]
+ assert(createdResource.getMetadata.getName === FirstTestConfigurationStep.secretName)
+ assert(createdResource.getData.asScala ===
+ Map(FirstTestConfigurationStep.secretKey -> FirstTestConfigurationStep.secretData))
+ val ownerReference = Iterables.getOnlyElement(createdResource.getMetadata.getOwnerReferences)
+ assert(ownerReference.getName === createdPod.getMetadata.getName)
+ assert(ownerReference.getKind === DRIVER_POD_KIND)
+ assert(ownerReference.getUid === DRIVER_POD_UID)
+ assert(ownerReference.getApiVersion === DRIVER_POD_API_VERSION)
+ }
+
+ test("The client should attach the driver container with the appropriate JVM options.") {
+ val sparkConf = new SparkConf(false)
+ .set("spark.logConf", "true")
+ .set(
+ org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS,
+ "-XX:+|-HeapDumpOnOutOfMemoryError")
+ val submissionClient = new Client(
+ submissionSteps,
+ sparkConf,
+ kubernetesClient,
+ false,
+ "spark",
+ loggingPodStatusWatcher)
+ submissionClient.run()
+ val createdPod = createdPodArgumentCaptor.getValue
+ val driverContainer = Iterables.getOnlyElement(createdPod.getSpec.getContainers)
+ assert(driverContainer.getName === SecondTestConfigurationStep.containerName)
+ val driverJvmOptsEnv = Iterables.getOnlyElement(driverContainer.getEnv)
+ assert(driverJvmOptsEnv.getName === ENV_DRIVER_JAVA_OPTS)
+ val driverJvmOpts = driverJvmOptsEnv.getValue.split(" ").toSet
+ assert(driverJvmOpts.contains("-Dspark.logConf=true"))
+ assert(driverJvmOpts.contains(
+ s"-D${SecondTestConfigurationStep.sparkConfKey}=" +
+ SecondTestConfigurationStep.sparkConfValue))
+ assert(driverJvmOpts.contains(
+ "-XX:+|-HeapDumpOnOutOfMemoryError"))
+ }
+
+ test("Waiting for app completion should stall on the watcher") {
+ val submissionClient = new Client(
+ submissionSteps,
+ new SparkConf(false),
+ kubernetesClient,
+ true,
+ "spark",
+ loggingPodStatusWatcher)
+ submissionClient.run()
+ verify(loggingPodStatusWatcher).awaitCompletion()
+ }
+
+}
+
+private object FirstTestConfigurationStep extends DriverConfigurationStep {
+
+ val podName = "test-pod"
+ val secretName = "test-secret"
+ val labelKey = "first-submit"
+ val labelValue = "true"
+ val secretKey = "secretKey"
+ val secretData = "secretData"
+
+ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
+ val modifiedPod = new PodBuilder(driverSpec.driverPod)
+ .editMetadata()
+ .withName(podName)
+ .addToLabels(labelKey, labelValue)
+ .endMetadata()
+ .build()
+ val additionalResource = new SecretBuilder()
+ .withNewMetadata()
+ .withName(secretName)
+ .endMetadata()
+ .addToData(secretKey, secretData)
+ .build()
+ driverSpec.copy(
+ driverPod = modifiedPod,
+ otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(additionalResource))
+ }
+}
+
+private object SecondTestConfigurationStep extends DriverConfigurationStep {
+
+ val annotationKey = "second-submit"
+ val annotationValue = "submitted"
+ val sparkConfKey = "spark.custom-conf"
+ val sparkConfValue = "custom-conf-value"
+ val containerName = "driverContainer"
+
+ override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = {
+ val modifiedPod = new PodBuilder(driverSpec.driverPod)
+ .editMetadata()
+ .addToAnnotations(annotationKey, annotationValue)
+ .endMetadata()
+ .build()
+ val resolvedSparkConf = driverSpec.driverSparkConf.clone().set(sparkConfKey, sparkConfValue)
+ val modifiedContainer = new ContainerBuilder(driverSpec.driverContainer)
+ .withName(containerName)
+ .build()
+ driverSpec.copy(
+ driverPod = modifiedPod,
+ driverSparkConf = resolvedSparkConf,
+ driverContainer = modifiedContainer)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala
new file mode 100644
index 0000000000000..e4f221ad99cc5
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, PythonStep}
+
+private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite {
+
+ private val NAMESPACE = "default"
+ private val APP_ID = "spark-app-id"
+ private val LAUNCH_TIME = 975256L
+ private val APP_NAME = "spark"
+ private val MAIN_CLASS = "org.apache.spark.examples.SparkPi"
+ private val APP_ARGS = Array("arg1", "arg2")
+ private val ADDITIONAL_PYTHON_FILES = Seq("local:///var/apps/python/py1.py")
+
+ test("Base submission steps without an init-container or python files.") {
+ val sparkConf = new SparkConf(false)
+ .set("spark.jars", "local:///var/apps/jars/jar1.jar")
+ val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
+ val orchestrator = new DriverConfigurationStepsOrchestrator(
+ NAMESPACE,
+ APP_ID,
+ LAUNCH_TIME,
+ mainAppResource,
+ APP_NAME,
+ MAIN_CLASS,
+ APP_ARGS,
+ ADDITIONAL_PYTHON_FILES,
+ sparkConf)
+ val steps = orchestrator.getAllConfigurationSteps()
+ assert(steps.size === 3)
+ assert(steps(0).isInstanceOf[BaseDriverConfigurationStep])
+ assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep])
+ assert(steps(2).isInstanceOf[DependencyResolutionStep])
+ }
+
+ test("Submission steps with an init-container.") {
+ val sparkConf = new SparkConf(false)
+ .set("spark.jars", "hdfs://localhost:9000/var/apps/jars/jar1.jar")
+ val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar")
+ val orchestrator = new DriverConfigurationStepsOrchestrator(
+ NAMESPACE,
+ APP_ID,
+ LAUNCH_TIME,
+ mainAppResource,
+ APP_NAME,
+ MAIN_CLASS,
+ APP_ARGS,
+ ADDITIONAL_PYTHON_FILES,
+ sparkConf)
+ val steps = orchestrator.getAllConfigurationSteps()
+ assert(steps.size === 4)
+ assert(steps(0).isInstanceOf[BaseDriverConfigurationStep])
+ assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep])
+ assert(steps(2).isInstanceOf[DependencyResolutionStep])
+ assert(steps(3).isInstanceOf[InitContainerBootstrapStep])
+ }
+
+ test("Submission steps with python files.") {
+ val sparkConf = new SparkConf(false)
+ val mainAppResource = PythonMainAppResource("local:///var/apps/python/main.py")
+ val orchestrator = new DriverConfigurationStepsOrchestrator(
+ NAMESPACE,
+ APP_ID,
+ LAUNCH_TIME,
+ mainAppResource,
+ APP_NAME,
+ MAIN_CLASS,
+ APP_ARGS,
+ ADDITIONAL_PYTHON_FILES,
+ sparkConf)
+ val steps = orchestrator.getAllConfigurationSteps()
+ assert(steps.size === 4)
+ assert(steps(0).isInstanceOf[BaseDriverConfigurationStep])
+ assert(steps(1).isInstanceOf[DriverKubernetesCredentialsStep])
+ assert(steps(2).isInstanceOf[DependencyResolutionStep])
+ assert(steps(3).isInstanceOf[PythonStep])
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala
new file mode 100644
index 0000000000000..0de1955884c8e
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit
+
+import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService
+import org.apache.spark.network.netty.SparkTransportConf
+import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient
+
+private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite {
+
+ private val SPARK_CONF = new SparkConf()
+ .set(KUBERNETES_SHUFFLE_USE_SERVICE_ACCOUNT_CREDENTIALS, false)
+
+ test("Run kubernetes shuffle service.") {
+ val shuffleService = new KubernetesExternalShuffleService(
+ SPARK_CONF,
+ new SecurityManager(SPARK_CONF))
+
+ val shuffleClient = new KubernetesExternalShuffleClient(
+ SparkTransportConf.fromSparkConf(SPARK_CONF, "shuffle"),
+ new SecurityManager(SPARK_CONF),
+ false,
+ false)
+
+ shuffleService.start()
+ shuffleClient.init("newapp")
+
+ // verifies that we can connect to the shuffle service and send
+ // it a message.
+ shuffleClient.registerDriverWithShuffleService("localhost", 7337)
+ shuffleService.stop()
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala
new file mode 100644
index 0000000000000..5240128743b76
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit
+
+import java.io.File
+
+case class KeyAndCertPem(keyPem: File, certPem: File)
+
+case class KeyStoreAndTrustStore(keyStore: File, trustStore: File)
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala
new file mode 100644
index 0000000000000..96fa92c254297
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit
+
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File}
+import java.util.UUID
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import com.google.common.base.Charsets
+import com.google.common.io.{BaseEncoding, Files}
+import okhttp3.RequestBody
+import okio.Okio
+import org.mockito.{ArgumentCaptor, Mockito}
+import org.scalatest.BeforeAndAfter
+import org.scalatest.mock.MockitoSugar._
+import retrofit2.{Call, Response}
+
+import org.apache.spark.{SparkFunSuite, SSLOptions}
+import org.apache.spark.deploy.kubernetes.CompressionUtils
+import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner}
+import org.apache.spark.util.Utils
+
+private[spark] class SubmittedDependencyUploaderSuite extends SparkFunSuite with BeforeAndAfter {
+ import SubmittedDependencyUploaderSuite.createTempFile
+
+ private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule)
+ private val BASE_64 = BaseEncoding.base64()
+ private val APP_ID = "app-id"
+ private val LABELS = Map("label1" -> "label1value", "label2" -> "label2value")
+ private val NAMESPACE = "namespace"
+ private val STAGING_SERVER_URI = "http://localhost:8000"
+ private val LOCAL_JARS = Seq(createTempFile("jar"), createTempFile("jar"))
+ private val JARS = Seq("hdfs://localhost:9000/jars/jar1.jar",
+ s"file://${LOCAL_JARS.head}",
+ LOCAL_JARS(1))
+ private val LOCAL_FILES = Seq(createTempFile("txt"))
+ private val FILES = Seq("hdfs://localhost:9000/files/file1.txt",
+ LOCAL_FILES.head)
+ private val TRUSTSTORE_FILE = new File(createTempFile(".jks"))
+ private val TRUSTSTORE_PASSWORD = "trustStorePassword"
+ private val TRUSTSTORE_TYPE = "jks"
+ private val STAGING_SERVER_SSL_OPTIONS = SSLOptions(
+ enabled = true,
+ trustStore = Some(TRUSTSTORE_FILE),
+ trustStorePassword = Some(TRUSTSTORE_PASSWORD),
+ trustStoreType = Some(TRUSTSTORE_TYPE))
+ private val CLIENT_KEY_FILE = createTempFile("pem")
+ private val CLIENT_CERT_FILE = createTempFile("pem")
+ private val OAUTH_TOKEN = "token"
+ private var retrofitClientFactory: RetrofitClientFactory = _
+ private var retrofitClient: ResourceStagingServiceRetrofit = _
+ private var resourcesOwnerCaptor: ArgumentCaptor[RequestBody] = _
+ private var resourcesDataCaptor: ArgumentCaptor[RequestBody] = _
+
+ private var dependencyUploaderUnderTest: SubmittedDependencyUploader = _
+
+ before {
+ resourcesOwnerCaptor = ArgumentCaptor.forClass(classOf[RequestBody])
+ resourcesDataCaptor = ArgumentCaptor.forClass(classOf[RequestBody])
+ retrofitClientFactory = mock[RetrofitClientFactory]
+ retrofitClient = mock[ResourceStagingServiceRetrofit]
+ Mockito.when(
+ retrofitClientFactory.createRetrofitClient(
+ STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS))
+ .thenReturn(retrofitClient)
+ val responseCall = mock[Call[SubmittedResourceIdAndSecret]]
+ Mockito.when(responseCall.execute()).thenReturn(
+ Response.success(SubmittedResourceIdAndSecret("resourceId", "resourceSecret")))
+ Mockito.when(retrofitClient.uploadResources(
+ resourcesDataCaptor.capture(), resourcesOwnerCaptor.capture()))
+ .thenReturn(responseCall)
+ dependencyUploaderUnderTest = new SubmittedDependencyUploaderImpl(
+ LABELS,
+ NAMESPACE,
+ STAGING_SERVER_URI,
+ JARS,
+ FILES,
+ STAGING_SERVER_SSL_OPTIONS,
+ retrofitClientFactory)
+ }
+
+ test("Uploading jars should contact the staging server with the appropriate parameters") {
+ dependencyUploaderUnderTest.uploadJars()
+ testUploadSendsCorrectFiles(LOCAL_JARS)
+ }
+
+ test("Uploading files should contact the staging server with the appropriate parameters") {
+ dependencyUploaderUnderTest.uploadFiles()
+ testUploadSendsCorrectFiles(LOCAL_FILES)
+ }
+
+ private def testUploadSendsCorrectFiles(expectedFiles: Seq[String]) = {
+ val resourceOwnerString = new String(
+ requestBodyBytes(resourcesOwnerCaptor.getValue), Charsets.UTF_8)
+ val resourceOwner = OBJECT_MAPPER.readValue(resourceOwnerString, classOf[StagedResourcesOwner])
+ assert(resourceOwner.ownerLabels === LABELS)
+ assert(resourceOwner.ownerNamespace === NAMESPACE)
+ val unpackedFilesDir = Utils.createTempDir(namePrefix = "test-unpacked-files")
+ val compressedBytesInput = new ByteArrayInputStream(
+ requestBodyBytes(resourcesDataCaptor.getValue()))
+ CompressionUtils.unpackTarStreamToDirectory(compressedBytesInput, unpackedFilesDir)
+ val writtenFiles = unpackedFilesDir.listFiles
+ assert(writtenFiles.size === expectedFiles.size)
+
+ expectedFiles.map(new File(_)).foreach { expectedFile =>
+ val maybeWrittenFile = writtenFiles.find(_.getName == expectedFile.getName)
+ assert(maybeWrittenFile.isDefined)
+ maybeWrittenFile.foreach { writtenFile =>
+ val writtenFileBytes = Files.toByteArray(writtenFile)
+ val expectedFileBytes = Files.toByteArray(expectedFile)
+ assert(expectedFileBytes.toSeq === writtenFileBytes.toSeq)
+ }
+ }
+ }
+
+ private def requestBodyBytes(requestBody: RequestBody): Array[Byte] = {
+ Utils.tryWithResource(new ByteArrayOutputStream()) { outputStream =>
+ Utils.tryWithResource(Okio.sink(outputStream)) { sink =>
+ Utils.tryWithResource(Okio.buffer(sink)) { bufferedSink =>
+ try {
+ requestBody.writeTo(bufferedSink)
+ } finally {
+ bufferedSink.flush()
+ }
+ }
+ }
+ outputStream.toByteArray
+ }
+ }
+}
+
+private object SubmittedDependencyUploaderSuite {
+ def createTempFile(extension: String): String = {
+ val dir = Utils.createTempDir()
+ val file = new File(dir, s"${UUID.randomUUID().toString}.$extension")
+ Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8)
+ file.getAbsolutePath
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala
new file mode 100644
index 0000000000000..c7d80a16a1532
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+
+private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite {
+
+ private val APP_ID = "spark-app-id"
+ private val RESOURCE_NAME_PREFIX = "spark"
+ private val DRIVER_LABELS = Map("labelkey" -> "labelvalue")
+ private val DOCKER_IMAGE_PULL_POLICY = "IfNotPresent"
+ private val APP_NAME = "spark-test"
+ private val MAIN_CLASS = "org.apache.spark.examples.SparkPi"
+ private val APP_ARGS = Array("arg1", "arg2")
+ private val CUSTOM_ANNOTATION_KEY = "customAnnotation"
+ private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue"
+ private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "customAnnotationDeprecated"
+ private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "customAnnotationDeprecatedValue"
+
+ test("Set all possible configurations from the user.") {
+ val sparkConf = new SparkConf()
+ .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod")
+ .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, "/opt/spark/spark-exmaples.jar")
+ .set("spark.driver.cores", "2")
+ .set(KUBERNETES_DRIVER_LIMIT_CORES, "4")
+ .set(org.apache.spark.internal.config.DRIVER_MEMORY, 256L)
+ .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L)
+ .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest")
+ .set(s"spark.kubernetes.driver.annotation.$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE)
+ .set("spark.kubernetes.driver.annotations",
+ s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE")
+ val submissionStep = new BaseDriverConfigurationStep(
+ APP_ID,
+ RESOURCE_NAME_PREFIX,
+ DRIVER_LABELS,
+ DOCKER_IMAGE_PULL_POLICY,
+ APP_NAME,
+ MAIN_CLASS,
+ APP_ARGS,
+ sparkConf)
+ val basePod = new PodBuilder().withNewMetadata().endMetadata().withNewSpec().endSpec().build()
+ val baseDriverSpec = KubernetesDriverSpec(
+ driverPod = basePod,
+ driverContainer = new ContainerBuilder().build(),
+ driverSparkConf = new SparkConf(false),
+ otherKubernetesResources = Seq.empty[HasMetadata])
+
+ val preparedDriverSpec = submissionStep.configureDriver(baseDriverSpec)
+ assert(preparedDriverSpec.driverContainer.getName === DRIVER_CONTAINER_NAME)
+ assert(preparedDriverSpec.driverContainer.getImage === "spark-driver:latest")
+ assert(preparedDriverSpec.driverContainer.getImagePullPolicy === DOCKER_IMAGE_PULL_POLICY)
+ val envs = preparedDriverSpec.driverContainer
+ .getEnv
+ .asScala
+ .map(env => (env.getName, env.getValue))
+ .toMap
+ assert(envs.size === 4)
+ assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === "/opt/spark/spark-exmaples.jar")
+ assert(envs(ENV_DRIVER_MEMORY) === "456m")
+ assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS)
+ assert(envs(ENV_DRIVER_ARGS) === "arg1 arg2")
+ val resourceRequirements = preparedDriverSpec.driverContainer.getResources
+ val requests = resourceRequirements.getRequests.asScala
+ assert(requests("cpu").getAmount === "2")
+ assert(requests("memory").getAmount === "256M")
+ val limits = resourceRequirements.getLimits.asScala
+ assert(limits("memory").getAmount === "456M")
+ assert(limits("cpu").getAmount === "4")
+ val driverPodMetadata = preparedDriverSpec.driverPod.getMetadata
+ assert(driverPodMetadata.getName === "spark-driver-pod")
+ assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS)
+ val expectedAnnotations = Map(
+ CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE,
+ DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE,
+ SPARK_APP_NAME_ANNOTATION -> APP_NAME)
+ assert(driverPodMetadata.getAnnotations.asScala === expectedAnnotations)
+ assert(preparedDriverSpec.driverPod.getSpec.getRestartPolicy === "Never")
+ val resolvedSparkConf = preparedDriverSpec.driverSparkConf.getAll.toMap
+ val expectedSparkConf = Map(
+ KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod",
+ "spark.app.id" -> APP_ID,
+ KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX)
+ assert(resolvedSparkConf === expectedSparkConf)
+
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala
new file mode 100644
index 0000000000000..3f7ec61074b0c
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import java.io.File
+
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.constants._
+
+private[spark] class DependencyResolutionStepSuite extends SparkFunSuite {
+
+ private val SPARK_JARS = Seq(
+ "hdfs://localhost:9000/apps/jars/jar1.jar",
+ "file:///home/user/apps/jars/jar2.jar",
+ "local:///var/apps/jars/jar3.jar")
+
+ private val SPARK_FILES = Seq(
+ "file:///home/user/apps/files/file1.txt",
+ "hdfs://localhost:9000/apps/files/file2.txt",
+ "local:///var/apps/files/file3.txt")
+
+ private val JARS_DOWNLOAD_PATH = "/mnt/spark-data/jars"
+ private val FILES_DOWNLOAD_PATH = "/mnt/spark-data/files"
+
+ test("Added dependencies should be resolved in Spark configuration and environment") {
+ val dependencyResolutionStep = new DependencyResolutionStep(
+ SPARK_JARS,
+ SPARK_FILES,
+ JARS_DOWNLOAD_PATH,
+ FILES_DOWNLOAD_PATH)
+ val driverPod = new PodBuilder().build()
+ val baseDriverSpec = KubernetesDriverSpec(
+ driverPod = driverPod,
+ driverContainer = new ContainerBuilder().build(),
+ driverSparkConf = new SparkConf(false),
+ otherKubernetesResources = Seq.empty[HasMetadata])
+ val preparedDriverSpec = dependencyResolutionStep.configureDriver(baseDriverSpec)
+ assert(preparedDriverSpec.driverPod === driverPod)
+ assert(preparedDriverSpec.otherKubernetesResources.isEmpty)
+ val resolvedSparkJars = preparedDriverSpec.driverSparkConf.get("spark.jars").split(",").toSet
+ val expectedResolvedSparkJars = Set(
+ "hdfs://localhost:9000/apps/jars/jar1.jar",
+ s"$JARS_DOWNLOAD_PATH/jar2.jar",
+ "/var/apps/jars/jar3.jar")
+ assert(resolvedSparkJars === expectedResolvedSparkJars)
+ val resolvedSparkFiles = preparedDriverSpec.driverSparkConf.get("spark.files").split(",").toSet
+ val expectedResolvedSparkFiles = Set(
+ s"$FILES_DOWNLOAD_PATH/file1.txt",
+ s"hdfs://localhost:9000/apps/files/file2.txt",
+ s"/var/apps/files/file3.txt")
+ assert(resolvedSparkFiles === expectedResolvedSparkFiles)
+ val driverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
+ assert(driverEnv.size === 1)
+ assert(driverEnv.head.getName === ENV_MOUNTED_CLASSPATH)
+ val resolvedDriverClasspath = driverEnv.head.getValue.split(File.pathSeparator).toSet
+ val expectedResolvedDriverClasspath = Set(
+ s"$JARS_DOWNLOAD_PATH/jar1.jar",
+ s"$JARS_DOWNLOAD_PATH/jar2.jar",
+ "/var/apps/jars/jar3.jar")
+ assert(resolvedDriverClasspath === expectedResolvedDriverClasspath)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala
new file mode 100644
index 0000000000000..3d5664713a2b8
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala
@@ -0,0 +1,152 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import java.io.File
+
+import com.google.common.base.Charsets
+import com.google.common.io.{BaseEncoding, Files}
+import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret}
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.util.Utils
+
+private[spark] class DriverKubernetesCredentialsStepSuite
+ extends SparkFunSuite with BeforeAndAfter {
+
+ private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark"
+ private var credentialsTempDirectory: File = _
+ private val BASE_DRIVER_SPEC = new KubernetesDriverSpec(
+ driverPod = new PodBuilder().build(),
+ driverContainer = new ContainerBuilder().build(),
+ driverSparkConf = new SparkConf(false),
+ otherKubernetesResources = Seq.empty[HasMetadata])
+
+ before {
+ credentialsTempDirectory = Utils.createTempDir()
+ }
+
+ after {
+ credentialsTempDirectory.delete()
+ }
+
+ test("Don't set any credentials") {
+ val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep(
+ new SparkConf(false), KUBERNETES_RESOURCE_NAME_PREFIX)
+ val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC)
+ assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod)
+ assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer)
+ assert(preparedDriverSpec.otherKubernetesResources.isEmpty)
+ assert(preparedDriverSpec.driverSparkConf.getAll.isEmpty)
+ }
+
+ test("Only set credentials that are manually mounted.") {
+ val submissionSparkConf = new SparkConf(false)
+ .set(
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX",
+ "/mnt/secrets/my-token.txt")
+ .set(
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX",
+ "/mnt/secrets/my-key.pem")
+ .set(
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX",
+ "/mnt/secrets/my-cert.pem")
+ .set(
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX",
+ "/mnt/secrets/my-ca.pem")
+
+ val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep(
+ submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX)
+ val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(BASE_DRIVER_SPEC)
+ assert(preparedDriverSpec.driverPod === BASE_DRIVER_SPEC.driverPod)
+ assert(preparedDriverSpec.driverContainer === BASE_DRIVER_SPEC.driverContainer)
+ assert(preparedDriverSpec.otherKubernetesResources.isEmpty)
+ assert(preparedDriverSpec.driverSparkConf.getAll.toMap === submissionSparkConf.getAll.toMap)
+ }
+
+ test("Mount credentials from the submission client as a secret.") {
+ val caCertFile = writeCredentials("ca.pem", "ca-cert")
+ val clientKeyFile = writeCredentials("key.pem", "key")
+ val clientCertFile = writeCredentials("cert.pem", "cert")
+ val submissionSparkConf = new SparkConf(false)
+ .set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX",
+ "token")
+ .set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX",
+ clientKeyFile.getAbsolutePath)
+ .set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX",
+ clientCertFile.getAbsolutePath)
+ .set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX",
+ caCertFile.getAbsolutePath)
+ val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep(
+ submissionSparkConf, KUBERNETES_RESOURCE_NAME_PREFIX)
+ val preparedDriverSpec = kubernetesCredentialsStep.configureDriver(
+ BASE_DRIVER_SPEC.copy(driverSparkConf = submissionSparkConf))
+ val expectedSparkConf = Map(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$OAUTH_TOKEN_CONF_SUFFIX" -> "",
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$OAUTH_TOKEN_FILE_CONF_SUFFIX" ->
+ DRIVER_CREDENTIALS_OAUTH_TOKEN_PATH,
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" ->
+ DRIVER_CREDENTIALS_CLIENT_KEY_PATH,
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" ->
+ DRIVER_CREDENTIALS_CLIENT_CERT_PATH,
+ s"$APISERVER_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" ->
+ DRIVER_CREDENTIALS_CA_CERT_PATH,
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX" ->
+ clientKeyFile.getAbsolutePath,
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX" ->
+ clientCertFile.getAbsolutePath,
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX" ->
+ caCertFile.getAbsolutePath)
+ assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf)
+ assert(preparedDriverSpec.otherKubernetesResources.size === 1)
+ val credentialsSecret = preparedDriverSpec.otherKubernetesResources.head.asInstanceOf[Secret]
+ assert(credentialsSecret.getMetadata.getName ===
+ s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials")
+ val decodedSecretData = credentialsSecret.getData.asScala.map { data =>
+ (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8))
+ }
+ val expectedSecretData = Map(
+ DRIVER_CREDENTIALS_CA_CERT_SECRET_NAME -> "ca-cert",
+ DRIVER_CREDENTIALS_OAUTH_TOKEN_SECRET_NAME -> "token",
+ DRIVER_CREDENTIALS_CLIENT_KEY_SECRET_NAME -> "key",
+ DRIVER_CREDENTIALS_CLIENT_CERT_SECRET_NAME -> "cert")
+ assert(decodedSecretData === expectedSecretData)
+ val driverPodVolumes = preparedDriverSpec.driverPod.getSpec.getVolumes.asScala
+ assert(driverPodVolumes.size === 1)
+ assert(driverPodVolumes.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME)
+ assert(driverPodVolumes.head.getSecret != null)
+ assert(driverPodVolumes.head.getSecret.getSecretName === credentialsSecret.getMetadata.getName)
+ val driverContainerVolumeMount = preparedDriverSpec.driverContainer.getVolumeMounts.asScala
+ assert(driverContainerVolumeMount.size === 1)
+ assert(driverContainerVolumeMount.head.getName === DRIVER_CREDENTIALS_SECRET_VOLUME_NAME)
+ assert(driverContainerVolumeMount.head.getMountPath === DRIVER_CREDENTIALS_SECRETS_BASE_DIR)
+ }
+
+ private def writeCredentials(credentialsFileName: String, credentialsContents: String): File = {
+ val credentialsFile = new File(credentialsTempDirectory, credentialsFileName)
+ Files.write(credentialsContents, credentialsFile, Charsets.UTF_8)
+ credentialsFile
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala
new file mode 100644
index 0000000000000..ce0dcee6acc46
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import io.fabric8.kubernetes.api.model._
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+
+class PythonStepSuite extends SparkFunSuite with BeforeAndAfter {
+ private val FILE_DOWNLOAD_PATH = "/var/data/spark-files"
+ private val PYSPARK_FILES = Seq(
+ "hdfs://localhost:9000/app/files/file1.py",
+ "file:///app/files/file2.py",
+ "local:///app/files/file3.py",
+ "http://app/files/file4.py")
+ private val RESOLVED_PYSPARK_FILES = Seq(
+ FILE_DOWNLOAD_PATH + "/file1.py",
+ FILE_DOWNLOAD_PATH + "/file2.py",
+ "/app/files/file3.py",
+ FILE_DOWNLOAD_PATH + "/file4.py").mkString(",")
+ private val PYSPARK_PRIMARY_FILE = "file:///app/files/file5.py"
+ private val RESOLVED_PYSPARK_PRIMARY_FILE = FILE_DOWNLOAD_PATH + "/file5.py"
+
+ test("testing PySpark with --py-files both local and remote files") {
+ val pyStep = new PythonStep(
+ PYSPARK_PRIMARY_FILE,
+ PYSPARK_FILES,
+ FILE_DOWNLOAD_PATH)
+ val returnedDriverContainer = pyStep.configureDriver(
+ KubernetesDriverSpec(
+ new Pod(),
+ new Container(),
+ Seq.empty[HasMetadata],
+ new SparkConf))
+ assert(returnedDriverContainer.driverContainer.getEnv
+ .asScala.map(env => (env.getName, env.getValue)).toMap ===
+ Map(
+ "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE,
+ "PYSPARK_FILES" -> RESOLVED_PYSPARK_FILES))
+ }
+
+ test("testing PySpark with empty --py-files ") {
+ val pyStep = new PythonStep(
+ PYSPARK_PRIMARY_FILE,
+ Seq.empty[String],
+ FILE_DOWNLOAD_PATH)
+ val returnedDriverContainer = pyStep.configureDriver(
+ KubernetesDriverSpec(
+ new Pod(),
+ new Container(),
+ Seq.empty[HasMetadata],
+ new SparkConf))
+ assert(returnedDriverContainer.driverContainer.getEnv
+ .asScala.map(env => (env.getName, env.getValue)).toMap ===
+ Map(
+ "PYSPARK_PRIMARY" -> RESOLVED_PYSPARK_PRIMARY_FILE,
+ "PYSPARK_FILES" -> "null"))
+ }
+
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala
new file mode 100644
index 0000000000000..b11b487111496
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps
+
+import java.io.StringReader
+import java.util.Properties
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import com.google.common.collect.Maps
+import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec}
+import org.apache.spark.util.Utils
+
+private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite {
+
+ private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule)
+ private val CONFIG_MAP_NAME = "spark-init-config-map"
+ private val CONFIG_MAP_KEY = "spark-init-config-map-key"
+
+ test("The init container bootstrap step should use all of the init container steps") {
+ val baseDriverSpec = KubernetesDriverSpec(
+ driverPod = new PodBuilder().build(),
+ driverContainer = new ContainerBuilder().build(),
+ driverSparkConf = new SparkConf(false),
+ otherKubernetesResources = Seq.empty[HasMetadata])
+ val initContainerSteps = Seq(
+ FirstTestInitContainerConfigurationStep$,
+ SecondTestInitContainerConfigurationStep$)
+ val bootstrapStep = new InitContainerBootstrapStep(
+ initContainerSteps,
+ CONFIG_MAP_NAME,
+ CONFIG_MAP_KEY)
+ val preparedDriverSpec = bootstrapStep.configureDriver(baseDriverSpec)
+ assert(preparedDriverSpec.driverPod.getMetadata.getLabels.asScala ===
+ FirstTestInitContainerConfigurationStep$.additionalLabels)
+ val additionalDriverEnv = preparedDriverSpec.driverContainer.getEnv.asScala
+ assert(additionalDriverEnv.size === 1)
+ assert(additionalDriverEnv.head.getName ===
+ FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvKey)
+ assert(additionalDriverEnv.head.getValue ===
+ FirstTestInitContainerConfigurationStep$.additionalMainContainerEnvValue)
+ val driverAnnotations = preparedDriverSpec.driverPod.getMetadata.getAnnotations.asScala
+ assert(driverAnnotations.size === 1)
+ val initContainers = OBJECT_MAPPER.readValue(
+ driverAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]])
+ assert(initContainers.length === 1)
+ val initContainerEnv = initContainers.head.getEnv.asScala
+ assert(initContainerEnv.size === 1)
+ assert(initContainerEnv.head.getName ===
+ SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvKey)
+ assert(initContainerEnv.head.getValue ===
+ SecondTestInitContainerConfigurationStep$.additionalInitContainerEnvValue)
+ val expectedSparkConf = Map(
+ EXECUTOR_INIT_CONTAINER_CONFIG_MAP.key -> CONFIG_MAP_NAME,
+ EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY.key -> CONFIG_MAP_KEY,
+ SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfKey ->
+ SecondTestInitContainerConfigurationStep$.additionalDriverSparkConfValue)
+ assert(preparedDriverSpec.driverSparkConf.getAll.toMap === expectedSparkConf)
+ assert(preparedDriverSpec.otherKubernetesResources.size === 2)
+ assert(preparedDriverSpec.otherKubernetesResources.contains(
+ FirstTestInitContainerConfigurationStep$.additionalKubernetesResource))
+ assert(preparedDriverSpec.otherKubernetesResources.exists {
+ case configMap: ConfigMap =>
+ val hasMatchingName = configMap.getMetadata.getName == CONFIG_MAP_NAME
+ val configMapData = configMap.getData.asScala
+ val hasCorrectNumberOfEntries = configMapData.size == 1
+ val initContainerPropertiesRaw = configMapData(CONFIG_MAP_KEY)
+ val initContainerProperties = new Properties()
+ Utils.tryWithResource(new StringReader(initContainerPropertiesRaw)) {
+ initContainerProperties.load(_)
+ }
+ val initContainerPropertiesMap = Maps.fromProperties(initContainerProperties).asScala
+ val expectedInitContainerProperties = Map(
+ SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyKey ->
+ SecondTestInitContainerConfigurationStep$.additionalInitContainerPropertyValue)
+ val hasMatchingProperties = initContainerPropertiesMap == expectedInitContainerProperties
+ hasMatchingName && hasCorrectNumberOfEntries && hasMatchingProperties
+ case _ => false
+ })
+ }
+}
+
+private object FirstTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
+
+ val additionalLabels = Map("additionalLabelkey" -> "additionalLabelValue")
+ val additionalMainContainerEnvKey = "TEST_ENV_MAIN_KEY"
+ val additionalMainContainerEnvValue = "TEST_ENV_MAIN_VALUE"
+ val additionalKubernetesResource = new SecretBuilder()
+ .withNewMetadata()
+ .withName("test-secret")
+ .endMetadata()
+ .addToData("secret-key", "secret-value")
+ .build()
+
+ override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
+ val driverPod = new PodBuilder(initContainerSpec.podToInitialize)
+ .editOrNewMetadata()
+ .addToLabels(additionalLabels.asJava)
+ .endMetadata()
+ .build()
+ val mainContainer = new ContainerBuilder(initContainerSpec.driverContainer)
+ .addNewEnv()
+ .withName(additionalMainContainerEnvKey)
+ .withValue(additionalMainContainerEnvValue)
+ .endEnv()
+ .build()
+ initContainerSpec.copy(
+ podToInitialize = driverPod,
+ driverContainer = mainContainer,
+ initContainerDependentResources = initContainerSpec.initContainerDependentResources ++
+ Seq(additionalKubernetesResource))
+ }
+}
+
+private object SecondTestInitContainerConfigurationStep$ extends InitContainerConfigurationStep {
+ val additionalInitContainerEnvKey = "TEST_ENV_INIT_KEY"
+ val additionalInitContainerEnvValue = "TEST_ENV_INIT_VALUE"
+ val additionalInitContainerPropertyKey = "spark.initcontainer.testkey"
+ val additionalInitContainerPropertyValue = "testvalue"
+ val additionalDriverSparkConfKey = "spark.driver.testkey"
+ val additionalDriverSparkConfValue = "spark.driver.testvalue"
+
+ override def configureInitContainer(initContainerSpec: InitContainerSpec): InitContainerSpec = {
+ val initContainer = new ContainerBuilder(initContainerSpec.initContainer)
+ .addNewEnv()
+ .withName(additionalInitContainerEnvKey)
+ .withValue(additionalInitContainerEnvValue)
+ .endEnv()
+ .build()
+ val initContainerProperties = initContainerSpec.initContainerProperties ++
+ Map(additionalInitContainerPropertyKey -> additionalInitContainerPropertyValue)
+ val driverSparkConf = initContainerSpec.additionalDriverSparkConf ++
+ Map(additionalDriverSparkConfKey -> additionalDriverSparkConfValue)
+ initContainerSpec.copy(
+ initContainer = initContainer,
+ initContainerProperties = initContainerProperties,
+ additionalDriverSparkConf = driverSparkConf)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala
new file mode 100644
index 0000000000000..fe1af4bc5be2a
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import io.fabric8.kubernetes.api.model._
+import org.mockito.{Mock, MockitoAnnotations}
+import org.mockito.Matchers.any
+import org.mockito.Mockito.when
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap}
+import org.apache.spark.deploy.kubernetes.config._
+
+class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{
+ private val SPARK_JARS = Seq(
+ "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
+ private val SPARK_FILES = Seq(
+ "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
+ private val JARS_DOWNLOAD_PATH = "/var/data/jars"
+ private val FILES_DOWNLOAD_PATH = "/var/data/files"
+ private val CONFIG_MAP_NAME = "config-map"
+ private val CONFIG_MAP_KEY = "config-map-key"
+ private val POD_LABEL = Map("bootstrap" -> "true")
+ private val INIT_CONTAINER_NAME = "init-container"
+ private val DRIVER_CONTAINER_NAME = "driver-container"
+
+ @Mock
+ private var podAndInitContainerBootstrap : SparkPodInitContainerBootstrap = _
+
+ before {
+ MockitoAnnotations.initMocks(this)
+ when(podAndInitContainerBootstrap.bootstrapInitContainerAndVolumes(
+ any[PodWithDetachedInitContainer])).thenAnswer(new Answer[PodWithDetachedInitContainer] {
+ override def answer(invocation: InvocationOnMock) : PodWithDetachedInitContainer = {
+ val pod = invocation.getArgumentAt(0, classOf[PodWithDetachedInitContainer])
+ pod.copy(
+ pod =
+ new PodBuilder(pod.pod)
+ .withNewMetadata()
+ .addToLabels("bootstrap", "true")
+ .endMetadata()
+ .withNewSpec().endSpec()
+ .build(),
+ initContainer =
+ new ContainerBuilder()
+ .withName(INIT_CONTAINER_NAME).build(),
+ mainContainer =
+ new ContainerBuilder()
+ .withName(DRIVER_CONTAINER_NAME).build()
+ )}})
+ }
+
+ test("Test of additionalDriverSparkConf with mix of remote files and jars") {
+ val baseInitStep = new BaseInitContainerConfigurationStep(
+ SPARK_JARS,
+ SPARK_FILES,
+ JARS_DOWNLOAD_PATH,
+ FILES_DOWNLOAD_PATH,
+ CONFIG_MAP_NAME,
+ CONFIG_MAP_KEY,
+ podAndInitContainerBootstrap)
+ val expectedDriverSparkConf = Map(
+ INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH,
+ INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH,
+ INIT_CONTAINER_REMOTE_JARS.key -> "hdfs://localhost:9000/app/jars/jar1.jar",
+ INIT_CONTAINER_REMOTE_FILES.key -> "hdfs://localhost:9000/app/files/file1.txt")
+ val initContainerSpec = InitContainerSpec(
+ Map.empty[String, String],
+ Map.empty[String, String],
+ new Container(),
+ new Container(),
+ new Pod,
+ Seq.empty[HasMetadata])
+ val returnContainerSpec = baseInitStep.configureInitContainer(initContainerSpec)
+ assert(expectedDriverSparkConf === returnContainerSpec.initContainerProperties)
+ assert(returnContainerSpec.initContainer.getName == INIT_CONTAINER_NAME)
+ assert(returnContainerSpec.driverContainer.getName == DRIVER_CONTAINER_NAME)
+ assert(returnContainerSpec.podToInitialize.getMetadata.getLabels.asScala === POD_LABEL)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala
new file mode 100644
index 0000000000000..1cc8007803457
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+
+class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite {
+ private val NAMESPACE = "namespace"
+ private val APP_RESOURCE_PREFIX = "spark-prefix"
+ private val SPARK_JARS = Seq(
+ "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar")
+ private val SPARK_FILES = Seq(
+ "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt")
+ private val JARS_DOWNLOAD_PATH = "/var/data/jars"
+ private val FILES_DOWNLOAD_PATH = "/var/data/files"
+ private val DOCKER_IMAGE_PULL_POLICY: String = "IfNotPresent"
+ private val APP_ID = "spark-id"
+ private val CUSTOM_LABEL_KEY = "customLabel"
+ private val CUSTOM_LABEL_VALUE = "customLabelValue"
+ private val DEPRECATED_CUSTOM_LABEL_KEY = "deprecatedCustomLabel"
+ private val DEPRECATED_CUSTOM_LABEL_VALUE = "deprecatedCustomLabelValue"
+ private val DRIVER_LABELS = Map(
+ CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE,
+ DEPRECATED_CUSTOM_LABEL_KEY -> DEPRECATED_CUSTOM_LABEL_VALUE,
+ SPARK_APP_ID_LABEL -> APP_ID,
+ SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE)
+ private val INIT_CONTAINER_CONFIG_MAP_NAME = "spark-init-config-map"
+ private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key"
+ private val STAGING_SERVER_URI = "http://localhost:8000"
+
+ test ("including step to contact resource staging server") {
+ val sparkConf = new SparkConf(true)
+ .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE")
+ .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE)
+ .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI)
+
+ val orchestrator = new InitContainerConfigurationStepsOrchestrator(
+ NAMESPACE,
+ APP_RESOURCE_PREFIX,
+ SPARK_JARS,
+ SPARK_FILES,
+ JARS_DOWNLOAD_PATH,
+ FILES_DOWNLOAD_PATH,
+ DOCKER_IMAGE_PULL_POLICY,
+ DRIVER_LABELS,
+ INIT_CONTAINER_CONFIG_MAP_NAME,
+ INIT_CONTAINER_CONFIG_MAP_KEY,
+ sparkConf)
+ val initSteps : Seq[InitContainerConfigurationStep] =
+ orchestrator.getAllConfigurationSteps()
+ assert(initSteps.length == 2)
+ assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep])
+ assert(initSteps(1).isInstanceOf[SubmittedResourcesInitContainerConfigurationStep])
+ }
+
+ test ("not including steps because no contact to resource staging server") {
+ val sparkConf = new SparkConf(true)
+ .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE")
+ .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE)
+
+ val orchestrator = new InitContainerConfigurationStepsOrchestrator(
+ NAMESPACE,
+ APP_RESOURCE_PREFIX,
+ SPARK_JARS,
+ SPARK_FILES,
+ JARS_DOWNLOAD_PATH,
+ FILES_DOWNLOAD_PATH,
+ DOCKER_IMAGE_PULL_POLICY,
+ DRIVER_LABELS,
+ INIT_CONTAINER_CONFIG_MAP_NAME,
+ INIT_CONTAINER_CONFIG_MAP_KEY,
+ sparkConf)
+ val initSteps : Seq[InitContainerConfigurationStep] =
+ orchestrator.getAllConfigurationSteps()
+ assert(initSteps.length === 1)
+ assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep])
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala
new file mode 100644
index 0000000000000..2edaba93fe07f
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer
+
+import java.io.File
+import java.util.UUID
+
+import com.google.common.base.Charsets
+import com.google.common.io.{BaseEncoding, Files}
+import io.fabric8.kubernetes.api.model._
+import org.mockito.{Mock, MockitoAnnotations}
+import org.mockito.Matchers.any
+import org.mockito.Mockito.when
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.constants._
+import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret}
+import org.apache.spark.util.Utils
+
+class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter {
+ private val RESOURCE_SECRET_NAME = "secret"
+ private val JARS_RESOURCE_ID = "jarsID"
+ private val JARS_SECRET = "jarsSecret"
+ private val FILES_RESOURCE_ID = "filesID"
+ private val FILES_SECRET = "filesSecret"
+ private val STAGING_SERVER_URI = "http://localhost:8000"
+ private val SECRET_MOUNT_PATH = "/tmp"
+ private val RSS_SECRET = Map(
+ INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY ->
+ BaseEncoding.base64().encode(JARS_SECRET.getBytes(Charsets.UTF_8)),
+ INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY ->
+ BaseEncoding.base64().encode(FILES_SECRET.getBytes(Charsets.UTF_8))
+ ).asJava
+ private var RSS_WITH_SSL_SECRET: java.util.Map[String, String] = _
+ private var TRUSTSTORE_FILENAME: String = ""
+ private var TRUSTSTORE_FILE: File = _
+ private var TRUSTSTORE_URI: Option[String] = None
+ private val TRUSTSTORE_PASS = "trustStorePassword"
+ private val TRUSTSTORE_TYPE = "jks"
+ private var CERT_FILENAME: String = ""
+ private var CERT_FILE: File = _
+ private var CERT_URI: Option[String] = None
+
+ @Mock
+ private var submittedDependencyUploader: SubmittedDependencyUploader = _
+ @Mock
+ private var submittedResourcesSecretPlugin: InitContainerResourceStagingServerSecretPlugin = _
+
+ before {
+ MockitoAnnotations.initMocks(this)
+ TRUSTSTORE_FILENAME = createTempFile(".jks")
+ TRUSTSTORE_FILE = new File(TRUSTSTORE_FILENAME)
+ TRUSTSTORE_URI = Some(TRUSTSTORE_FILENAME)
+ CERT_FILENAME = createTempFile("pem")
+ CERT_FILE = new File(CERT_FILENAME)
+ CERT_URI = Some(CERT_FILENAME)
+ RSS_WITH_SSL_SECRET =
+ (RSS_SECRET.asScala ++ Map(
+ INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY ->
+ BaseEncoding.base64().encode(Files.toByteArray(TRUSTSTORE_FILE)),
+ INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY ->
+ BaseEncoding.base64().encode(Files.toByteArray(CERT_FILE))
+ )).asJava
+ when(submittedDependencyUploader.uploadJars()).thenReturn(
+ SubmittedResourceIdAndSecret(JARS_RESOURCE_ID, JARS_SECRET)
+ )
+ when(submittedDependencyUploader.uploadFiles()).thenReturn(
+ SubmittedResourceIdAndSecret(FILES_RESOURCE_ID, FILES_SECRET)
+ )
+ when(submittedResourcesSecretPlugin.addResourceStagingServerSecretVolumeToPod(
+ any[Pod])).thenAnswer(new Answer[Pod] {
+ override def answer(invocation: InvocationOnMock) : Pod = {
+ val pod = invocation.getArgumentAt(0, classOf[Pod])
+ new PodBuilder(pod)
+ .withNewMetadata()
+ .addToLabels("mountedSecret", "true")
+ .endMetadata()
+ .withNewSpec().endSpec()
+ .build()
+ }})
+ when(submittedResourcesSecretPlugin.mountResourceStagingServerSecretIntoInitContainer(
+ any[Container])).thenAnswer(new Answer[Container] {
+ override def answer(invocation: InvocationOnMock) : Container = {
+ val con = invocation.getArgumentAt(0, classOf[Container])
+ new ContainerBuilder(con).withName("mountedSecret").build()
+ }})
+ }
+ after {
+ TRUSTSTORE_FILE.delete()
+ CERT_FILE.delete()
+ }
+ test ("testing vanilla prepareInitContainer on resources and properties") {
+ val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep(
+ RESOURCE_SECRET_NAME,
+ STAGING_SERVER_URI,
+ SECRET_MOUNT_PATH,
+ false,
+ None,
+ None,
+ None,
+ None,
+ submittedDependencyUploader,
+ submittedResourcesSecretPlugin
+ )
+ val returnedInitContainer =
+ submittedResourceStep.configureInitContainer(InitContainerSpec(
+ Map.empty[String, String],
+ Map.empty[String, String],
+ new Container(),
+ new Container(),
+ new Pod(),
+ Seq.empty[HasMetadata]))
+ assert(returnedInitContainer.initContainer.getName === "mountedSecret")
+ assert(returnedInitContainer.podToInitialize.getMetadata.getLabels.asScala
+ === Map("mountedSecret" -> "true"))
+ assert(returnedInitContainer.initContainerDependentResources.length == 1)
+ val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret]
+ assert(secret.getData === RSS_SECRET)
+ assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME)
+ val expectedinitContainerProperties = Map(
+ RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI,
+ INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY",
+ INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY",
+ RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString)
+ assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties)
+ assert(returnedInitContainer.additionalDriverSparkConf ===
+ Map(
+ EXECUTOR_INIT_CONTAINER_SECRET.key -> RESOURCE_SECRET_NAME,
+ EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR.key -> SECRET_MOUNT_PATH))
+ }
+
+ test ("testing prepareInitContainer w/ CERT and TrustStore Files w/o SSL") {
+ val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep(
+ RESOURCE_SECRET_NAME,
+ STAGING_SERVER_URI,
+ SECRET_MOUNT_PATH,
+ false,
+ TRUSTSTORE_URI,
+ CERT_URI,
+ Some(TRUSTSTORE_PASS),
+ Some(TRUSTSTORE_TYPE),
+ submittedDependencyUploader,
+ submittedResourcesSecretPlugin
+ )
+ val returnedInitContainer =
+ submittedResourceStep.configureInitContainer(InitContainerSpec(
+ Map.empty[String, String],
+ Map.empty[String, String],
+ new Container(),
+ new Container(),
+ new Pod(),
+ Seq.empty[HasMetadata]))
+ val expectedinitContainerProperties = Map(
+ RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI,
+ INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY",
+ INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY",
+ RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY",
+ RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_STAGING_SERVER_CLIENT_CERT_SECRET_KEY"
+ )
+ assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties)
+ assert(returnedInitContainer.initContainerDependentResources.length == 1)
+ val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret]
+ assert(secret.getData === RSS_WITH_SSL_SECRET)
+ assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME)
+
+ }
+
+ test ("testing prepareInitContainer w/ local CERT and TrustStore Files w/o SSL") {
+ val LOCAL_TRUST_FILE = "local:///tmp/trust.jsk"
+ val LOCAL_CERT_FILE = "local:///tmp/cert.pem"
+ val submittedResourceStep = new SubmittedResourcesInitContainerConfigurationStep(
+ RESOURCE_SECRET_NAME,
+ STAGING_SERVER_URI,
+ SECRET_MOUNT_PATH,
+ false,
+ Some(LOCAL_TRUST_FILE),
+ Some(LOCAL_CERT_FILE),
+ Some(TRUSTSTORE_PASS),
+ Some(TRUSTSTORE_TYPE),
+ submittedDependencyUploader,
+ submittedResourcesSecretPlugin
+ )
+ val returnedInitContainer =
+ submittedResourceStep.configureInitContainer(InitContainerSpec(
+ Map.empty[String, String],
+ Map.empty[String, String],
+ new Container(),
+ new Container(),
+ new Pod(),
+ Seq.empty[HasMetadata]))
+ val expectedinitContainerProperties = Map(
+ RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI,
+ INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY",
+ INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID,
+ INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key ->
+ s"$SECRET_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY",
+ RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> false.toString,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASS,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE,
+ RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key ->
+ "/tmp/trust.jsk",
+ RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key ->
+ "/tmp/cert.pem"
+ )
+ assert(returnedInitContainer.initContainerProperties === expectedinitContainerProperties)
+ assert(returnedInitContainer.initContainerDependentResources.length == 1)
+ val secret = returnedInitContainer.initContainerDependentResources.head.asInstanceOf[Secret]
+ assert(secret.getData === RSS_SECRET)
+ assert(secret.getMetadata.getName == RESOURCE_SECRET_NAME)
+ }
+ private def createTempFile(extension: String): String = {
+ val dir = Utils.createTempDir()
+ val file = new File(dir, s"${UUID.randomUUID().toString}.$extension")
+ Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8)
+ file.getAbsolutePath
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala
new file mode 100644
index 0000000000000..f2fdf026390cd
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{ByteArrayOutputStream, File}
+import java.util.UUID
+import javax.ws.rs.core
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import okhttp3.{MediaType, ResponseBody}
+import org.mockito.Matchers.any
+import org.mockito.Mockito
+import org.mockito.Mockito.{doAnswer, when}
+import org.mockito.invocation.InvocationOnMock
+import org.mockito.stubbing.Answer
+import org.scalatest.BeforeAndAfter
+import org.scalatest.mock.MockitoSugar._
+import retrofit2.{Call, Callback, Response}
+
+import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions}
+import org.apache.spark.deploy.kubernetes.CompressionUtils
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.util.Utils
+
+class KubernetesSparkDependencyDownloadInitContainerSuite
+ extends SparkFunSuite with BeforeAndAfter {
+ import KubernetesSparkDependencyDownloadInitContainerSuite.createTempFile
+ private val STAGING_SERVER_URI = "http://localhost:8000"
+ private val TRUSTSTORE_FILE = new File(createTempFile(".jks"))
+ private val TRUSTSTORE_PASSWORD = "trustStorePassword"
+ private val TRUSTSTORE_TYPE = "jks"
+ private val STAGING_SERVER_SSL_OPTIONS = SSLOptions(
+ enabled = true,
+ trustStore = Some(TRUSTSTORE_FILE),
+ trustStorePassword = Some(TRUSTSTORE_PASSWORD),
+ trustStoreType = Some(TRUSTSTORE_TYPE))
+ private val JARS = Seq(createTempFile("jar"), createTempFile("jar"))
+ private val FILES = Seq(createTempFile("txt"), createTempFile("csv"))
+ private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt")
+ private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt")
+ private val JARS_RESOURCE_ID = "jarsId"
+ private val FILES_RESOURCE_ID = "filesId"
+
+ private var downloadJarsDir: File = _
+ private var downloadFilesDir: File = _
+ private var downloadJarsSecretValue: String = _
+ private var downloadFilesSecretValue: String = _
+ private var jarsCompressedBytes: Array[Byte] = _
+ private var filesCompressedBytes: Array[Byte] = _
+ private var retrofitClientFactory: RetrofitClientFactory = _
+ private var retrofitClient: ResourceStagingServiceRetrofit = _
+ private var fileFetcher: FileFetcher = _
+
+ override def beforeAll(): Unit = {
+ jarsCompressedBytes = compressPathsToBytes(JARS)
+ filesCompressedBytes = compressPathsToBytes(FILES)
+ downloadJarsSecretValue = Files.toString(
+ new File(DOWNLOAD_JARS_SECRET_LOCATION), Charsets.UTF_8)
+ downloadFilesSecretValue = Files.toString(
+ new File(DOWNLOAD_FILES_SECRET_LOCATION), Charsets.UTF_8)
+ }
+
+ before {
+ downloadJarsDir = Utils.createTempDir()
+ downloadFilesDir = Utils.createTempDir()
+ retrofitClientFactory = mock[RetrofitClientFactory]
+ retrofitClient = mock[ResourceStagingServiceRetrofit]
+ fileFetcher = mock[FileFetcher]
+ when(retrofitClientFactory.createRetrofitClient(
+ STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS))
+ .thenReturn(retrofitClient)
+ }
+
+ after {
+ downloadJarsDir.delete()
+ downloadFilesDir.delete()
+ }
+
+ test("Downloads from resource staging server should unpack response body to directories") {
+ val downloadJarsCall = mock[Call[ResponseBody]]
+ val downloadFilesCall = mock[Call[ResponseBody]]
+ val sparkConf = getSparkConfForResourceStagingServerDownloads
+ val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer(
+ sparkConf,
+ retrofitClientFactory,
+ fileFetcher,
+ resourceStagingServerSslOptions = STAGING_SERVER_SSL_OPTIONS)
+ when(retrofitClient.downloadResources(JARS_RESOURCE_ID, downloadJarsSecretValue))
+ .thenReturn(downloadJarsCall)
+ when(retrofitClient.downloadResources(FILES_RESOURCE_ID, downloadFilesSecretValue))
+ .thenReturn(downloadFilesCall)
+ val jarsResponseBody = ResponseBody.create(
+ MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), jarsCompressedBytes)
+ val filesResponseBody = ResponseBody.create(
+ MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), filesCompressedBytes)
+ doAnswer(new InvokeCallbackAnswer(downloadJarsCall, jarsResponseBody))
+ .when(downloadJarsCall)
+ .enqueue(any())
+ doAnswer(new InvokeCallbackAnswer(downloadFilesCall, filesResponseBody))
+ .when(downloadFilesCall)
+ .enqueue(any())
+ initContainerUnderTest.run()
+ checkWrittenFilesAreTheSameAsOriginal(JARS, downloadJarsDir)
+ checkWrittenFilesAreTheSameAsOriginal(FILES, downloadFilesDir)
+ Mockito.verifyZeroInteractions(fileFetcher)
+ }
+
+ test("Downloads from remote server should invoke the file fetcher") {
+ val sparkConf = getSparkConfForRemoteFileDownloads
+ val initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer(
+ sparkConf,
+ retrofitClientFactory,
+ fileFetcher,
+ resourceStagingServerSslOptions = STAGING_SERVER_SSL_OPTIONS)
+ initContainerUnderTest.run()
+ Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/jar1.jar", downloadJarsDir)
+ Mockito.verify(fileFetcher).fetchFile("hdfs://localhost:9000/jar2.jar", downloadJarsDir)
+ Mockito.verify(fileFetcher).fetchFile("http://localhost:9000/file.txt", downloadFilesDir)
+
+ }
+
+ private def getSparkConfForResourceStagingServerDownloads: SparkConf = {
+ new SparkConf(true)
+ .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI)
+ .set(INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER, JARS_RESOURCE_ID)
+ .set(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION, DOWNLOAD_JARS_SECRET_LOCATION)
+ .set(INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER, FILES_RESOURCE_ID)
+ .set(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION, DOWNLOAD_FILES_SECRET_LOCATION)
+ .set(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath)
+ .set(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath)
+ .set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
+ .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE, TRUSTSTORE_FILE.getAbsolutePath)
+ .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD, TRUSTSTORE_PASSWORD)
+ .set(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE, TRUSTSTORE_TYPE)
+ }
+
+ private def getSparkConfForRemoteFileDownloads: SparkConf = {
+ new SparkConf(true)
+ .set(INIT_CONTAINER_REMOTE_JARS,
+ "http://localhost:9000/jar1.jar,hdfs://localhost:9000/jar2.jar")
+ .set(INIT_CONTAINER_REMOTE_FILES,
+ "http://localhost:9000/file.txt")
+ .set(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath)
+ .set(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION, downloadFilesDir.getAbsolutePath)
+ }
+
+ private def checkWrittenFilesAreTheSameAsOriginal(
+ originalFiles: Iterable[String], downloadDir: File): Unit = {
+ originalFiles.map(new File(_)).foreach { file =>
+ val writtenFile = new File(downloadDir, file.getName)
+ assert(writtenFile.exists)
+ val originalJarContents = Seq(Files.toByteArray(file): _*)
+ val writtenJarContents = Seq(Files.toByteArray(writtenFile): _*)
+ assert(writtenJarContents === originalJarContents)
+ }
+ }
+
+ private def compressPathsToBytes(paths: Iterable[String]): Array[Byte] = {
+ Utils.tryWithResource(new ByteArrayOutputStream()) { compressedBytes =>
+ CompressionUtils.writeTarGzipToStream (compressedBytes, paths)
+ compressedBytes.toByteArray
+ }
+ }
+}
+
+private object KubernetesSparkDependencyDownloadInitContainerSuite {
+ def createTempFile(extension: String): String = {
+ val dir = Utils.createTempDir()
+ val file = new File(dir, s"${UUID.randomUUID().toString}.$extension")
+ Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8)
+ file.getAbsolutePath
+ }
+}
+
+private class InvokeCallbackAnswer(call: Call[ResponseBody], responseBody: ResponseBody)
+ extends Answer[Unit] {
+ override def answer(invocationOnMock: InvocationOnMock): Unit = {
+ val callback = invocationOnMock.getArgumentAt(0, classOf[Callback[ResponseBody]])
+ val response = Response.success(responseBody)
+ callback.onResponse(call, response)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala
new file mode 100644
index 0000000000000..3bb318d713a54
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{File, FileInputStream, StringWriter}
+import java.security.KeyStore
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import org.bouncycastle.openssl.jcajce.JcaPEMWriter
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions}
+import org.apache.spark.deploy.kubernetes.SSLUtils
+import org.apache.spark.util.Utils
+
+class ResourceStagingServerSslOptionsProviderSuite extends SparkFunSuite with BeforeAndAfter {
+
+ private var sslTempDir: File = _
+ private var keyStoreFile: File = _
+
+ private var sparkConf: SparkConf = _
+ private var sslOptionsProvider: ResourceStagingServerSslOptionsProvider = _
+
+ before {
+ sslTempDir = Utils.createTempDir(namePrefix = "resource-staging-server-ssl-test")
+ keyStoreFile = new File(sslTempDir, "keyStore.jks")
+ sparkConf = new SparkConf(true)
+ sslOptionsProvider = new ResourceStagingServerSslOptionsProviderImpl(sparkConf)
+ }
+
+ test("Default SparkConf does not have TLS enabled.") {
+ assert(sslOptionsProvider.getSslOptions === SSLOptions())
+ assert(!sslOptionsProvider.getSslOptions.enabled)
+ keyStoreFile.delete()
+ sslTempDir.delete()
+ }
+
+ test("Setting keyStore, key password, and key field directly.") {
+ sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", keyStoreFile.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStorePassword")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "keyPassword")
+ val sslOptions = sslOptionsProvider.getSslOptions
+ assert(sslOptions.enabled, "SSL should be enabled.")
+ assert(sslOptions.keyStore.map(_.getAbsolutePath) === Some(keyStoreFile.getAbsolutePath),
+ "Incorrect keyStore path or it was not set.")
+ assert(sslOptions.keyStorePassword === Some("keyStorePassword"),
+ "Incorrect keyStore password or it was not set.")
+ assert(sslOptions.keyPassword === Some("keyPassword"),
+ "Incorrect key password or it was not set.")
+ }
+
+ test("Setting key and certificate pem files should write an appropriate keyStore.") {
+ val keyAndCertPem = SSLUtils.generateKeyCertPemPair("127.0.0.1")
+ sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPem",
+ keyAndCertPem.keyPem.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.serverCertPem",
+ keyAndCertPem.certPem.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStorePassword")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "keyPassword")
+ val sslOptions = sslOptionsProvider.getSslOptions
+ assert(sslOptions.enabled, "SSL should be enabled.")
+ assert(sslOptions.keyStore.isDefined, "KeyStore should be defined.")
+ sslOptions.keyStore.foreach { keyStoreFile =>
+ val keyStore = KeyStore.getInstance(KeyStore.getDefaultType)
+ Utils.tryWithResource(new FileInputStream(keyStoreFile)) {
+ keyStore.load(_, "keyStorePassword".toCharArray)
+ }
+ val key = keyStore.getKey("key", "keyPassword".toCharArray)
+ compareJcaPemObjectToFileString(key, keyAndCertPem.keyPem)
+ val certificate = keyStore.getCertificateChain("key")(0)
+ compareJcaPemObjectToFileString(certificate, keyAndCertPem.certPem)
+ }
+ }
+
+ test("Setting pem files without setting passwords should use random passwords.") {
+ val keyAndCertPem = SSLUtils.generateKeyCertPemPair("127.0.0.1")
+ sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPem",
+ keyAndCertPem.keyPem.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.serverCertPem",
+ keyAndCertPem.certPem.getAbsolutePath)
+ val sslOptions = sslOptionsProvider.getSslOptions
+ assert(sslOptions.enabled, "SSL should be enabled.")
+ assert(sslOptions.keyStore.isDefined, "KeyStore should be defined.")
+ assert(sslOptions.keyStorePassword.isDefined)
+ assert(sslOptions.keyPassword.isDefined)
+ for {
+ keyStoreFile <- sslOptions.keyStore
+ keyStorePassword <- sslOptions.keyStorePassword
+ keyPassword <- sslOptions.keyPassword
+ } {
+ val keyStore = KeyStore.getInstance(KeyStore.getDefaultType)
+ Utils.tryWithResource(new FileInputStream(keyStoreFile)) {
+ keyStore.load(_, keyStorePassword.toCharArray)
+ }
+ val key = keyStore.getKey("key", keyPassword.toCharArray)
+ compareJcaPemObjectToFileString(key, keyAndCertPem.keyPem)
+ val certificate = keyStore.getCertificateChain("key")(0)
+ compareJcaPemObjectToFileString(certificate, keyAndCertPem.certPem)
+ }
+ }
+
+ test("Using password files should read from the appropriate locations.") {
+ val keyStorePasswordFile = new File(sslTempDir, "keyStorePassword.txt")
+ Files.write("keyStorePassword", keyStorePasswordFile, Charsets.UTF_8)
+ val keyPasswordFile = new File(sslTempDir, "keyPassword.txt")
+ Files.write("keyPassword", keyPasswordFile, Charsets.UTF_8)
+ sparkConf.set("spark.ssl.kubernetes.resourceStagingServer.enabled", "true")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", keyStoreFile.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile",
+ keyStorePasswordFile.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile",
+ keyPasswordFile.getAbsolutePath)
+ val sslOptions = sslOptionsProvider.getSslOptions
+ assert(sslOptions.keyStorePassword === Some("keyStorePassword"),
+ "Incorrect keyStore password or it was not set.")
+ assert(sslOptions.keyPassword === Some("keyPassword"),
+ "Incorrect key password or it was not set.")
+ }
+
+ private def compareJcaPemObjectToFileString(pemObject: Any, pemFile: File): Unit = {
+ Utils.tryWithResource(new StringWriter()) { stringWriter =>
+ Utils.tryWithResource(new JcaPEMWriter(stringWriter)) { pemWriter =>
+ pemWriter.writeObject(pemObject)
+ }
+ val pemFileAsString = Files.toString(pemFile, Charsets.UTF_8)
+ assert(stringWriter.toString === pemFileAsString)
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala
new file mode 100644
index 0000000000000..1bcd85a611e00
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.net.ServerSocket
+import javax.ws.rs.core.MediaType
+
+import com.fasterxml.jackson.databind.ObjectMapper
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import com.google.common.io.ByteStreams
+import okhttp3.{RequestBody, ResponseBody}
+import org.eclipse.jetty.server.Server
+import org.scalatest.BeforeAndAfter
+import org.scalatest.mock.MockitoSugar.mock
+import retrofit2.Call
+
+import org.apache.spark.{SparkFunSuite, SSLOptions}
+import org.apache.spark.deploy.kubernetes.SSLUtils
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+/**
+ * Tests for {@link ResourceStagingServer} and its APIs. Note that this is not an end-to-end
+ * integration test, and as such does not upload and download files in tar.gz as would be done
+ * in production. Thus we use the retrofit clients directly despite the fact that in practice
+ * we would likely want to create an opinionated abstraction on top of the retrofit client; we
+ * can test this abstraction layer separately, however. This test is mainly for checking that
+ * we've configured the Jetty server correctly and that the endpoints reached over HTTP can
+ * receive streamed uploads and can stream downloads.
+ */
+class ResourceStagingServerSuite extends SparkFunSuite with BeforeAndAfter with Logging {
+
+ private val MAX_SERVER_START_ATTEMPTS = 5
+ private var serviceImpl: ResourceStagingService = _
+ private var stagedResourcesCleaner: StagedResourcesCleaner = _
+ private var server: Option[ResourceStagingServer] = None
+ private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule)
+
+ private val sslOptionsProvider = new SettableReferenceSslOptionsProvider()
+
+ before {
+ stagedResourcesCleaner = mock[StagedResourcesCleaner]
+ serviceImpl = new ResourceStagingServiceImpl(
+ new StagedResourcesStoreImpl(Utils.createTempDir()), stagedResourcesCleaner)
+ }
+
+ after {
+ server.foreach { s =>
+ try {
+ s.stop()
+ } catch {
+ case e: Throwable =>
+ log.warn("Failed to stop the resource staging server.", e)
+ }
+ }
+ server = None
+ }
+
+ test("Accept file and jar uploads and downloads") {
+ val serverPort = startServer()
+ runUploadAndDownload(SSLOptions(), serverPort)
+ }
+
+ test("Enable SSL on the server") {
+ val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair(
+ ipAddress = "127.0.0.1",
+ keyStorePassword = "keyStore",
+ keyPassword = "key",
+ trustStorePassword = "trustStore")
+ val sslOptions = SSLOptions(
+ enabled = true,
+ keyStore = Some(keyStoreAndTrustStore.keyStore),
+ keyStorePassword = Some("keyStore"),
+ keyPassword = Some("key"),
+ trustStore = Some(keyStoreAndTrustStore.trustStore),
+ trustStorePassword = Some("trustStore"))
+ sslOptionsProvider.setOptions(sslOptions)
+ val serverPort = startServer()
+ runUploadAndDownload(sslOptions, serverPort)
+ }
+
+ private def runUploadAndDownload(sslOptions: SSLOptions, serverPort: Int): Unit = {
+ val scheme = if (sslOptions.enabled) "https" else "http"
+ val retrofitService = RetrofitClientFactoryImpl.createRetrofitClient(
+ s"$scheme://127.0.0.1:$serverPort/",
+ classOf[ResourceStagingServiceRetrofit],
+ sslOptions)
+ val resourcesBytes = Array[Byte](1, 2, 3, 4)
+ val labels = Map("label1" -> "label1Value", "label2" -> "label2value")
+ val namespace = "namespace"
+ val resourcesOwner = StagedResourcesOwner(
+ ownerLabels = labels,
+ ownerNamespace = namespace,
+ ownerType = StagedResourcesOwnerType.Pod)
+ val resourcesOwnerJson = OBJECT_MAPPER.writeValueAsString(resourcesOwner)
+ val resourcesOwnerRequestBody = RequestBody.create(
+ okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), resourcesOwnerJson)
+ val resourcesRequestBody = RequestBody.create(
+ okhttp3.MediaType.parse(MediaType.MULTIPART_FORM_DATA), resourcesBytes)
+ val uploadResponse = retrofitService.uploadResources(
+ resourcesRequestBody, resourcesOwnerRequestBody)
+ val resourceIdentifier = getTypedResponseResult(uploadResponse)
+ checkResponseBodyBytesMatches(
+ retrofitService.downloadResources(
+ resourceIdentifier.resourceId, resourceIdentifier.resourceSecret), resourcesBytes)
+ }
+
+ private def getTypedResponseResult[T](call: Call[T]): T = {
+ val response = call.execute()
+ assert(response.code() >= 200 && response.code() < 300, Option(response.errorBody())
+ .map(_.string())
+ .getOrElse("Error executing HTTP request, but error body was not provided."))
+ val callResult = response.body()
+ assert(callResult != null)
+ callResult
+ }
+
+ private def checkResponseBodyBytesMatches(call: Call[ResponseBody], bytes: Array[Byte]): Unit = {
+ val responseBody = getTypedResponseResult(call)
+ val downloadedBytes = ByteStreams.toByteArray(responseBody.byteStream())
+ assert(downloadedBytes.toSeq === bytes)
+ }
+
+ private def startServer(): Int = {
+ var currentAttempt = 0
+ var successfulStart = false
+ var latestServerPort = new ServerSocket(0).getLocalPort
+ while (currentAttempt < MAX_SERVER_START_ATTEMPTS && !successfulStart) {
+ val newServer = new ResourceStagingServer(latestServerPort, serviceImpl, sslOptionsProvider)
+ try {
+ newServer.start()
+ successfulStart = true
+ server = Some(newServer)
+ } catch {
+ case e: Throwable =>
+ try {
+ newServer.stop()
+ } catch {
+ case e1: Throwable =>
+ log.warn("Failed to stop a resource staging server that failed to start.", e1)
+ }
+
+ if (Utils.isBindCollision(e)) {
+ currentAttempt += 1
+ latestServerPort = latestServerPort + 1
+ if (currentAttempt == MAX_SERVER_START_ATTEMPTS) {
+ throw new RuntimeException(s"Failed to bind to a random port" +
+ s" $MAX_SERVER_START_ATTEMPTS times. Last attempted port: $latestServerPort", e)
+ } else {
+ logWarning(s"Attempt $currentAttempt/$MAX_SERVER_START_ATTEMPTS failed to start" +
+ s" server on port $latestServerPort.", e)
+ }
+ } else {
+ throw e
+ }
+ }
+ }
+ logInfo(s"Started resource staging server on port $latestServerPort.")
+ latestServerPort
+ }
+}
+
+private class SettableReferenceSslOptionsProvider extends ResourceStagingServerSslOptionsProvider {
+ private var options = SSLOptions()
+
+ def setOptions(newOptions: SSLOptions): Unit = {
+ this.options = newOptions
+ }
+
+ override def getSslOptions: SSLOptions = options
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala
new file mode 100644
index 0000000000000..8b398a9891f34
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
+
+import io.fabric8.kubernetes.api.model.{DoneableNamespace, DoneablePod, Namespace, NamespaceList, Pod, PodList, PodListBuilder}
+import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher}
+import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource, Resource}
+import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations}
+import org.mockito.Matchers.{eq => mockitoEq}
+import org.mockito.Mockito.{never, verify, when}
+import org.scalatest.BeforeAndAfter
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.util.Clock
+
+private[spark] class StagedResourcesCleanerSuite extends SparkFunSuite with BeforeAndAfter {
+
+ private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
+ private type PODSWITHLABELS = FilterWatchListDeletable[
+ Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]]
+ private type PODSINNAMESPACE = NonNamespaceOperation[
+ Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]
+ private type NAMESPACES = NonNamespaceOperation[
+ Namespace, NamespaceList, DoneableNamespace, Resource[Namespace, DoneableNamespace]]
+ private type NAMESPACEWITHNAME = Resource[Namespace, DoneableNamespace]
+
+ private val INITIAL_ACCESS_EXPIRATION_MS = 5000L
+ private val CURRENT_TIME = 10000L
+ private val RESOURCE_ID = "resource-id"
+ private val POD_NAMESPACE = "namespace"
+ private val POD_LABELS = Map("label1" -> "label1value", "label2" -> "label2value")
+ private val RESOURCES_OWNER = StagedResourcesOwner(
+ ownerNamespace = POD_NAMESPACE,
+ ownerLabels = POD_LABELS,
+ ownerType = StagedResourcesOwnerType.Pod)
+
+ @Mock
+ private var stagedResourcesStore: StagedResourcesStore = _
+ @Mock
+ private var kubernetesClient: KubernetesClient = _
+ @Mock
+ private var clock: Clock = _
+ @Mock
+ private var cleanerExecutorService: ScheduledExecutorService = _
+ @Mock
+ private var podOperations: PODS = _
+ @Mock
+ private var podsInNamespaceOperations: PODSINNAMESPACE = _
+ @Mock
+ private var podsWithLabelsOperations: PODSWITHLABELS = _
+ @Mock
+ private var namespaceOperations: NAMESPACES = _
+ @Mock
+ private var namedNamespaceOperations: NAMESPACEWITHNAME = _
+ private var cleanerUnderTest: StagedResourcesCleaner = _
+
+ before {
+ MockitoAnnotations.initMocks(this)
+ cleanerUnderTest = new StagedResourcesCleanerImpl(
+ stagedResourcesStore,
+ kubernetesClient,
+ cleanerExecutorService,
+ clock,
+ INITIAL_ACCESS_EXPIRATION_MS)
+ when(kubernetesClient.pods()).thenReturn(podOperations)
+ when(podOperations.withLabels(POD_LABELS.asJava)).thenReturn(podsWithLabelsOperations)
+ when(kubernetesClient.namespaces()).thenReturn(namespaceOperations)
+ }
+
+ test("Clean the resource if it is never accessed for the expiration interval.") {
+ val cleanupRunnable = startCleanupAndGetCleanupRunnable()
+ cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER)
+ when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS)
+ cleanupRunnable.run()
+ verify(stagedResourcesStore).removeResources(RESOURCE_ID)
+ verify(kubernetesClient, never()).pods()
+ }
+
+ test("Don't clean the resource if it is accessed in the expiration interval" +
+ " and there are owners available.") {
+ val cleanupRunnable = startCleanupAndGetCleanupRunnable()
+ cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER)
+ cleanerUnderTest.markResourceAsUsed(RESOURCE_ID)
+ when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS)
+ when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations)
+ when(namedNamespaceOperations.get()).thenReturn(new Namespace())
+ when(podOperations.inNamespace(POD_NAMESPACE)).thenReturn(podsInNamespaceOperations)
+ when(podsInNamespaceOperations.withLabels(POD_LABELS.asJava))
+ .thenReturn(podsWithLabelsOperations)
+ when(podsWithLabelsOperations.list()).thenReturn(
+ new PodListBuilder().addNewItemLike(new Pod()).endItem().build())
+ cleanupRunnable.run()
+ verify(stagedResourcesStore, never()).removeResources(RESOURCE_ID)
+ }
+
+ test("Clean the resource if no owners are available.") {
+ val cleanupRunnable = startCleanupAndGetCleanupRunnable()
+ cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER)
+ cleanerUnderTest.markResourceAsUsed(RESOURCE_ID)
+ when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS)
+ when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations)
+ when(namedNamespaceOperations.get()).thenReturn(new Namespace())
+ when(podOperations.inNamespace(POD_NAMESPACE)).thenReturn(podsInNamespaceOperations)
+ when(podsInNamespaceOperations.withLabels(POD_LABELS.asJava))
+ .thenReturn(podsWithLabelsOperations)
+ when(podsWithLabelsOperations.list()).thenReturn(new PodListBuilder().build())
+ cleanupRunnable.run()
+ verify(stagedResourcesStore).removeResources(RESOURCE_ID)
+ }
+
+ test("Clean up the resource if the namespace does not exist.") {
+ val cleanupRunnable = startCleanupAndGetCleanupRunnable()
+ cleanerUnderTest.registerResourceForCleaning(RESOURCE_ID, RESOURCES_OWNER)
+ cleanerUnderTest.markResourceAsUsed(RESOURCE_ID)
+ when(clock.getTimeMillis()).thenReturn(CURRENT_TIME + INITIAL_ACCESS_EXPIRATION_MS)
+ when(namespaceOperations.withName(POD_NAMESPACE)).thenReturn(namedNamespaceOperations)
+ when(namedNamespaceOperations.get()).thenReturn(null)
+ cleanupRunnable.run()
+ verify(stagedResourcesStore).removeResources(RESOURCE_ID)
+ }
+
+ private def startCleanupAndGetCleanupRunnable(): Runnable = {
+ val captor = ArgumentCaptor.forClass(classOf[Runnable])
+ cleanerUnderTest.start()
+ verify(cleanerExecutorService).scheduleAtFixedRate(
+ captor.capture(),
+ mockitoEq(30000L),
+ mockitoEq(30000L),
+ mockitoEq(TimeUnit.MILLISECONDS))
+ captor.getValue
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala
new file mode 100644
index 0000000000000..6b5737ebf2e23
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.rest.kubernetes
+
+import java.io.{ByteArrayInputStream, File}
+import java.nio.file.Paths
+
+import com.google.common.io.Files
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.util.Utils
+
+private[spark] class StagedResourcesStoreSuite extends SparkFunSuite with BeforeAndAfter {
+
+ private val resourceBytes = Array[Byte](1, 2, 3, 4)
+ private val namespace = "namespace"
+ private var dependencyRootDir: File = _
+ private var stagedResourcesStore: StagedResourcesStore = _
+
+ before {
+ dependencyRootDir = Utils.createTempDir()
+ stagedResourcesStore = new StagedResourcesStoreImpl(dependencyRootDir)
+ }
+
+ after {
+ dependencyRootDir.delete()
+ }
+
+ test("Uploads should write data to the underlying disk") {
+ val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) {
+ resourceStream =>
+ stagedResourcesStore.addResources(namespace, resourceStream)
+ }
+ val resourceNamespaceDir = Paths.get(dependencyRootDir.getAbsolutePath, "namespace").toFile
+ assert(resourceNamespaceDir.isDirectory, s"Resource namespace dir was not created at" +
+ s" ${resourceNamespaceDir.getAbsolutePath} or is not a directory.")
+ val resourceDirs = resourceNamespaceDir.listFiles()
+ assert(resourceDirs.length === 1, s"Resource root directory did not have exactly one" +
+ s" subdirectory. Got: ${resourceDirs.map(_.getAbsolutePath).mkString(",")}")
+ assert(resourceDirs(0).getName === resourceIdAndSecret.resourceId)
+ val resourceTgz = new File(resourceDirs(0), "resources.data")
+ assert(resourceTgz.isFile,
+ s"Resources written to ${resourceTgz.getAbsolutePath} does not exist or is not a file.")
+ val resourceTgzBytes = Files.toByteArray(resourceTgz)
+ assert(resourceTgzBytes.toSeq === resourceBytes.toSeq, "Incorrect resource bytes were written.")
+ }
+
+ test("Uploading and then getting should return a stream with the written bytes.") {
+ val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) {
+ resourceStream =>
+ stagedResourcesStore.addResources(namespace, resourceStream)
+ }
+ val resources = stagedResourcesStore.getResources(resourceIdAndSecret.resourceId)
+ assert(resources.map(_.resourcesFile)
+ .map(Files.toByteArray)
+ .exists(resourceBytes.sameElements(_)))
+ assert(resources.exists(_.resourceId == resourceIdAndSecret.resourceId))
+ assert(resources.exists(_.resourceSecret == resourceIdAndSecret.resourceSecret))
+ }
+
+ test("Uploading and then deleting should result in the resource directory being deleted.") {
+ val resourceIdAndSecret = Utils.tryWithResource(new ByteArrayInputStream(resourceBytes)) {
+ resourceStream =>
+ stagedResourcesStore.addResources(namespace, resourceStream)
+ }
+ stagedResourcesStore.removeResources(resourceIdAndSecret.resourceId)
+ val resourceNamespaceDir = Paths.get(dependencyRootDir.getAbsolutePath, "namespace").toFile
+ assert(resourceNamespaceDir.listFiles().isEmpty)
+ assert(stagedResourcesStore.getResources(resourceIdAndSecret.resourceId).isEmpty)
+ }
+}
diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala
new file mode 100644
index 0000000000000..864ff40d88c5c
--- /dev/null
+++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.scheduler.cluster.kubernetes
+
+import scala.collection.mutable.ArrayBuffer
+
+import io.fabric8.kubernetes.api.model.{Pod, PodSpec, PodStatus}
+import org.mockito.Mockito._
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.scheduler.{FakeTask, FakeTaskScheduler, HostTaskLocation, TaskLocation}
+
+class KubernetesTaskSetManagerSuite extends SparkFunSuite with BeforeAndAfter {
+
+ val sc = new SparkContext("local", "test")
+ val sched = new FakeTaskScheduler(sc,
+ ("execA", "10.0.0.1"), ("execB", "10.0.0.2"), ("execC", "10.0.0.3"))
+ val backend = mock(classOf[KubernetesClusterSchedulerBackend])
+ sched.backend = backend
+
+ before {
+ sc.conf.remove(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED)
+ }
+
+ test("Find pending tasks for executors using executor pod IP addresses") {
+ val taskSet = FakeTask.createTaskSet(3,
+ Seq(TaskLocation("10.0.0.1", "execA")), // Task 0 runs on executor pod 10.0.0.1.
+ Seq(TaskLocation("10.0.0.1", "execA")), // Task 1 runs on executor pod 10.0.0.1.
+ Seq(TaskLocation("10.0.0.2", "execB")) // Task 2 runs on executor pod 10.0.0.2.
+ )
+
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0))
+ assert(manager.getPendingTasksForHost("10.0.0.2") == ArrayBuffer(2))
+ }
+
+ test("Find pending tasks for executors using cluster node names that executor pods run on") {
+ val taskSet = FakeTask.createTaskSet(2,
+ Seq(HostTaskLocation("kube-node1")), // Task 0's partition belongs to datanode on kube-node1
+ Seq(HostTaskLocation("kube-node1")) // Task 1's partition belongs to datanode on kube-node2
+ )
+ val spec1 = mock(classOf[PodSpec])
+ when(spec1.getNodeName).thenReturn("kube-node1")
+ val pod1 = mock(classOf[Pod])
+ when(pod1.getSpec).thenReturn(spec1)
+ when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1))
+
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0))
+ }
+
+ test("Find pending tasks for executors using cluster node IPs that executor pods run on") {
+ val taskSet = FakeTask.createTaskSet(2,
+ Seq(HostTaskLocation("196.0.0.5")), // Task 0's partition belongs to datanode on 196.0.0.5.
+ Seq(HostTaskLocation("196.0.0.5")) // Task 1's partition belongs to datanode on 196.0.0.5.
+ )
+ val spec1 = mock(classOf[PodSpec])
+ when(spec1.getNodeName).thenReturn("kube-node1")
+ val pod1 = mock(classOf[Pod])
+ when(pod1.getSpec).thenReturn(spec1)
+ val status1 = mock(classOf[PodStatus])
+ when(status1.getHostIP).thenReturn("196.0.0.5")
+ when(pod1.getStatus).thenReturn(status1)
+ when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1))
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0))
+ }
+
+ test("Test DNS lookup is disabled by default for cluster node full hostnames") {
+ assert(!sc.conf.get(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED))
+ }
+
+ test("Find pending tasks for executors, but avoid looking up cluster node FQDNs from DNS") {
+ sc.conf.set(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED, false)
+ val taskSet = FakeTask.createTaskSet(2,
+ Seq(HostTaskLocation("kube-node1.domain1")), // Task 0's partition belongs to datanode here.
+ Seq(HostTaskLocation("kube-node1.domain1")) // task 1's partition belongs to datanode here.
+ )
+ val spec1 = mock(classOf[PodSpec])
+ when(spec1.getNodeName).thenReturn("kube-node1")
+ val pod1 = mock(classOf[Pod])
+ when(pod1.getSpec).thenReturn(spec1)
+ val status1 = mock(classOf[PodStatus])
+ when(status1.getHostIP).thenReturn("196.0.0.5")
+ when(pod1.getStatus).thenReturn(status1)
+ val inetAddressUtil = mock(classOf[InetAddressUtil])
+ when(inetAddressUtil.getFullHostName("196.0.0.5")).thenReturn("kube-node1.domain1")
+ when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1))
+
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer())
+ }
+
+ test("Find pending tasks for executors using cluster node FQDNs that executor pods run on") {
+ sc.conf.set(KUBERNETES_DRIVER_CLUSTER_NODENAME_DNS_LOOKUP_ENABLED, true)
+ val taskSet = FakeTask.createTaskSet(2,
+ Seq(HostTaskLocation("kube-node1.domain1")), // Task 0's partition belongs to datanode here.
+ Seq(HostTaskLocation("kube-node1.domain1")) // task 1's partition belongs to datanode here.
+ )
+ val spec1 = mock(classOf[PodSpec])
+ when(spec1.getNodeName).thenReturn("kube-node1")
+ val pod1 = mock(classOf[Pod])
+ when(pod1.getSpec).thenReturn(spec1)
+ val status1 = mock(classOf[PodStatus])
+ when(status1.getHostIP).thenReturn("196.0.0.5")
+ when(pod1.getStatus).thenReturn(status1)
+ val inetAddressUtil = mock(classOf[InetAddressUtil])
+ when(inetAddressUtil.getFullHostName("196.0.0.5")).thenReturn("kube-node1.domain1")
+ when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1))
+
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer(1, 0))
+ }
+
+ test("Return empty pending tasks for executors when all look up fail") {
+ val taskSet = FakeTask.createTaskSet(1,
+ Seq(HostTaskLocation("kube-node1.domain1")) // task 0's partition belongs to datanode here.
+ )
+ val spec1 = mock(classOf[PodSpec])
+ when(spec1.getNodeName).thenReturn("kube-node2")
+ val pod1 = mock(classOf[Pod])
+ when(pod1.getSpec).thenReturn(spec1)
+ val status1 = mock(classOf[PodStatus])
+ when(status1.getHostIP).thenReturn("196.0.0.6")
+ when(pod1.getStatus).thenReturn(status1)
+ val inetAddressUtil = mock(classOf[InetAddressUtil])
+ when(inetAddressUtil.getFullHostName("196.0.0.6")).thenReturn("kube-node2.domain1")
+ when(backend.getExecutorPodByIP("10.0.0.1")).thenReturn(Some(pod1))
+
+ val manager = new KubernetesTaskSetManager(sched, taskSet, maxTaskFailures = 2, inetAddressUtil)
+ assert(manager.getPendingTasksForHost("10.0.0.1") == ArrayBuffer())
+ }
+}
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml
new file mode 100644
index 0000000000000..31184794972db
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml
@@ -0,0 +1,125 @@
+
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.11
+ 2.1.0-k8s-0.3.1-SNAPSHOT
+ ../../../pom.xml
+
+
+ spark-docker-minimal-bundle_2.11
+ Spark Project Docker Minimal Bundle
+ http://spark.apache.org/
+ pom
+
+
+ kubernetes-docker-minimal-bundle
+ none
+ pre-integration-test
+
+
+
+
+ org.apache.spark
+ spark-assembly_${scala.binary.version}
+ ${project.version}
+ pom
+
+
+
+
+ com.google.guava
+ guava
+ ${hadoop.deps.scope}
+
+
+
+
+ org.apache.spark
+ spark-kubernetes_${scala.binary.version}
+ ${project.version}
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-assembly-plugin
+
+
+ docker-dist
+ pre-integration-test
+
+ single
+
+
+
+ src/main/assembly/docker-assembly.xml
+
+ posix
+
+
+
+
+
+
+
+
+
+
+ hive
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ hive-thriftserver
+
+
+ org.apache.spark
+ spark-hive-thriftserver_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ spark-ganglia-lgpl
+
+
+ org.apache.spark
+ spark-ganglia-lgpl_${scala.binary.version}
+ ${project.version}
+
+
+
+
+
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml
new file mode 100644
index 0000000000000..2b48d366256fe
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml
@@ -0,0 +1,75 @@
+
+
+ docker-dist
+
+ tar.gz
+ dir
+
+ false
+
+
+
+ ${project.parent.basedir}/sbin/
+
+ sbin
+
+ **/*
+
+
+
+
+ ${project.parent.basedir}/bin/
+
+ bin
+
+ **/*
+
+
+
+
+ ${project.parent.basedir}/conf/
+
+ conf
+
+ **/*
+
+
+
+
+ src/main/docker/
+
+ dockerfiles
+
+ **/*
+
+
+
+
+
+ jars
+ true
+ false
+ runtime
+ false
+
+ org.apache.spark:spark-assembly_${scala.binary.version}:pom
+ org.spark-project.spark:unused
+
+
+
+
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile
new file mode 100644
index 0000000000000..6dcc7511c0dd9
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-py/Dockerfile
@@ -0,0 +1,48 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-driver-py:latest -f dockerfiles/driver-py/Dockerfile .
+
+ADD examples /opt/spark/examples
+ADD python /opt/spark/python
+
+RUN apk add --no-cache python && \
+ python -m ensurepip && \
+ rm -r /usr/lib/python*/ensurepip && \
+ pip install --upgrade pip setuptools && \
+ rm -r /root/.cache
+# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES
+# RUN apk add --update alpine-sdk python-dev
+# RUN pip install numpy
+
+ENV PYTHON_VERSION 2.7.13
+ENV PYSPARK_PYTHON python
+ENV PYSPARK_DRIVER_PYTHON python
+ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH}
+
+CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
+ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
+ exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH \
+ -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY \
+ $SPARK_DRIVER_CLASS $PYSPARK_PRIMARY $PYSPARK_FILES $SPARK_DRIVER_ARGS
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile
new file mode 100644
index 0000000000000..6bbff8ef64a0f
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver/Dockerfile
@@ -0,0 +1,31 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile .
+
+COPY examples /opt/spark/examples
+
+CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
+ if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
+ exec /sbin/tini -- ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile
new file mode 100644
index 0000000000000..7a65a4f879376
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor-py/Dockerfile
@@ -0,0 +1,46 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-executor-py:latest -f dockerfiles/executor-py/Dockerfile .
+
+ADD examples /opt/spark/examples
+ADD python /opt/spark/python
+
+RUN apk add --no-cache python && \
+ python -m ensurepip && \
+ rm -r /usr/lib/python*/ensurepip && \
+ pip install --upgrade pip setuptools && \
+ rm -r /root/.cache
+# UNCOMMENT THE FOLLOWING TO START PIP INSTALLING PYTHON PACKAGES
+# RUN apk add --update alpine-sdk python-dev
+# RUN pip install numpy
+
+ENV PYTHON_VERSION 2.7.13
+ENV PYSPARK_PYTHON python
+ENV PYSPARK_DRIVER_PYTHON python
+ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.4-src.zip:${PYTHONPATH}
+
+# TODO support spark.executor.extraClassPath
+CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
+ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
+ exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP
\ No newline at end of file
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile
new file mode 100644
index 0000000000000..b3b0acc3b64b8
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile
@@ -0,0 +1,32 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile .
+
+COPY examples /opt/spark/examples
+
+# TODO support spark.executor.extraClassPath
+CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \
+ if ! [ -z ${SPARK_MOUNTED_CLASSPATH}+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_EXECUTOR_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXECUTOR_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \
+ if ! [ -z ${SPARK_MOUNTED_FILES_DIR} ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." .; fi && \
+ exec /sbin/tini -- ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp $SPARK_CLASSPATH org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile
new file mode 100644
index 0000000000000..4bafe25e2608f
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile
@@ -0,0 +1,24 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile .
+
+ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.KubernetesSparkDependencyDownloadInitContainer" ]
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile
new file mode 100644
index 0000000000000..9ca96be0f1a88
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile
@@ -0,0 +1,25 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile .
+
+ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.ResourceStagingServer" ]
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile
new file mode 100644
index 0000000000000..ccb2f1a03d88c
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile
@@ -0,0 +1,26 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM spark-base
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-shuffle:latest -f dockerfiles/shuffle-service/Dockerfile .
+
+COPY examples /opt/spark/examples
+
+ENTRYPOINT [ "/sbin/tini", "--", "bin/spark-class", "org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService", "1" ]
diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile
new file mode 100644
index 0000000000000..b0925e3bb0416
--- /dev/null
+++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/spark-base/Dockerfile
@@ -0,0 +1,35 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+FROM openjdk:8-alpine
+
+# If this docker file is being used in the context of building your images from a Spark distribution, the docker build
+# command should be invoked from the top level directory of the Spark distribution. E.g.:
+# docker build -t spark-base:latest -f dockerfiles/spark-base/Dockerfile .
+
+RUN apk upgrade --no-cache && \
+ apk add --no-cache bash tini && \
+ mkdir -p /opt/spark && \
+ touch /opt/spark/RELEASE
+
+COPY jars /opt/spark/jars
+COPY bin /opt/spark/bin
+COPY sbin /opt/spark/sbin
+COPY conf /opt/spark/conf
+
+ENV SPARK_HOME /opt/spark
+
+WORKDIR /opt/spark
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml
new file mode 100644
index 0000000000000..94c982d5e1108
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/pom.xml
@@ -0,0 +1,36 @@
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.11
+ 2.1.0-k8s-0.3.1-SNAPSHOT
+ ../../../pom.xml
+
+
+ spark-kubernetes-integration-tests-spark-jobs-helpers_2.11
+ jar
+ Spark Project Kubernetes Integration Tests Spark Jobs Helpers
+
+ kubernetes-integration-tests-spark-jobs-helpers
+
+
+
+
+
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java
new file mode 100644
index 0000000000000..99d982397bb6e
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest;
+
+/**
+ * Primarily extracted so that a separate jar can be added as a dependency for the
+ * test Spark job.
+ */
+public class PiHelper {
+ public static int helpPi() {
+ double x = Math.random() * 2 - 1;
+ double y = Math.random() * 2 - 1;
+ if (x*x + y*y < 1) {
+ return 1;
+ } else {
+ return 0;
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml
new file mode 100644
index 0000000000000..9183bbc4f72e8
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs/pom.xml
@@ -0,0 +1,54 @@
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.11
+ 2.1.0-k8s-0.3.1-SNAPSHOT
+ ../../../pom.xml
+
+
+ spark-kubernetes-integration-tests-spark-jobs_2.11
+ jar
+ Spark Project Kubernetes Integration Tests Spark Jobs
+
+ kubernetes-integration-tests-spark-jobs
+
+
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version}
+ ${project.version}
+ provided
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${project.version}
+ provided
+
+
+ org.apache.spark
+ spark-sql_${scala.binary.version}
+ ${project.version}
+ provided
+
+
+
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala
new file mode 100644
index 0000000000000..8994c998bffee
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.jobs
+
+import java.nio.file.Paths
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+
+import org.apache.spark.SparkException
+import org.apache.spark.sql.SparkSession
+
+private[spark] object FileExistenceTest {
+
+ def main(args: Array[String]): Unit = {
+ if (args.length < 2) {
+ throw new IllegalArgumentException(
+ s"Invalid args: ${args.mkString}, " +
+ "Usage: FileExistenceTest ")
+ }
+ // Can't use SparkContext.textFile since the file is local to the driver
+ val file = Paths.get(args(0)).toFile
+ if (!file.exists()) {
+ throw new SparkException(s"Failed to find file at ${file.getAbsolutePath}")
+ } else {
+ // scalastyle:off println
+ val contents = Files.toString(file, Charsets.UTF_8)
+ if (args(1) != contents) {
+ throw new SparkException(s"Contents do not match. Expected: ${args(1)}," +
+ s" actual: $contents")
+ } else {
+ println(s"File found at ${file.getAbsolutePath} with correct contents.")
+ }
+ // scalastyle:on println
+ }
+ while (true) {
+ Thread.sleep(600000)
+ }
+ }
+
+}
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala
new file mode 100644
index 0000000000000..fe47d42485b24
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.jobs
+
+import java.util.Random
+
+import org.apache.spark.sql.SparkSession
+
+object GroupByTest {
+ def main(args: Array[String]) {
+ val spark = SparkSession
+ .builder
+ .appName("GroupBy Test")
+ .getOrCreate()
+
+ val numMappers = if (args.length > 0) args(0).toInt else 5
+ val numKVPairs = if (args.length > 1) args(1).toInt else 200000
+ val valSize = if (args.length > 2) args(2).toInt else 2
+ val numReducers = if (args.length > 3) args(3).toInt else numMappers
+
+ val pairs1 = spark.sparkContext.parallelize(0 until numMappers, numMappers).flatMap { p =>
+ val ranGen = new Random
+ val arr1 = new Array[(Int, Array[Byte])](numKVPairs)
+ for (i <- 0 until numKVPairs) {
+ val byteArr = new Array[Byte](valSize)
+ ranGen.nextBytes(byteArr)
+ arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr)
+ }
+ arr1
+ }.cache()
+ // Enforce that everything has been calculated and in cache
+ pairs1.count()
+
+ // scalastyle:off println
+ println("The Result is", pairs1.groupByKey(numReducers).count())
+ // scalastyle:on println
+ spark.stop()
+ }
+}
+
diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala
new file mode 100644
index 0000000000000..d3372749f999e
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.jobs
+
+import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper
+import org.apache.spark.sql.SparkSession
+
+// Equivalent to SparkPi except does not stop the Spark Context
+// at the end and spins forever, so other things can inspect the
+// Spark UI immediately after the fact.
+private[spark] object SparkPiWithInfiniteWait {
+
+ def main(args: Array[String]): Unit = {
+ val spark = SparkSession
+ .builder
+ .appName("Spark Pi")
+ .getOrCreate()
+ val slices = if (args.length > 0) args(0).toInt else 10
+ val n = math.min(100000L * slices, Int.MaxValue).toInt // avoid overflow
+ val count = spark.sparkContext.parallelize(1 until n, slices).map { _ =>
+ PiHelper.helpPi()
+ }.reduce(_ + _)
+ // scalastyle:off println
+ println("Pi is roughly " + 4.0 * count / (n - 1))
+ // scalastyle:on println
+
+ // Spin forever to keep the Spark UI active, so other things can inspect the job.
+ while (true) {
+ Thread.sleep(600000)
+ }
+ }
+
+}
diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml
new file mode 100644
index 0000000000000..32db53c670337
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/pom.xml
@@ -0,0 +1,389 @@
+
+
+
+ 4.0.0
+
+ org.apache.spark
+ spark-parent_2.11
+ 2.1.0-k8s-0.3.1-SNAPSHOT
+ ../../../pom.xml
+
+
+ spark-kubernetes-integration-tests_2.11
+
+ kubernetes-integration-tests
+
+ jar
+ Spark Project Kubernetes Integration Tests
+
+
+
+ org.apache.spark
+ spark-kubernetes_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${project.version}
+
+
+ org.apache.spark
+ spark-kubernetes_${scala.binary.version}
+ ${project.version}
+ test
+ test-jar
+
+
+ org.apache.spark
+ spark-core_${scala.binary.version}
+ ${project.version}
+ test-jar
+ test
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version}
+ ${project.version}
+ test
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version}
+ ${project.version}
+ test
+
+
+ org.apache.spark
+ spark-docker-minimal-bundle_${scala.binary.version}
+ ${project.version}
+ tar.gz
+ docker-dist
+ test
+
+
+ *
+ *
+
+
+
+
+ com.google.guava
+ guava
+ test
+
+ 18.0
+
+
+ com.spotify
+ docker-client
+ test
+
+
+
+ com.fasterxml.jackson.jaxrs
+ jackson-jaxrs-json-provider
+
+
+ com.fasterxml.jackson.core
+ jackson-databind
+
+
+ org.glassfish.jersey.core
+ jersey-client
+
+
+ org.glassfish.jersey.core
+ jersey-common
+
+
+ javax.ws.rs
+ jsr311-api
+
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+
+
+ copy-test-spark-jobs
+ pre-integration-test
+
+ copy
+
+
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version}
+ ${project.version}
+ jar
+ ${project.build.directory}/integration-tests-spark-jobs
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version}
+ ${project.version}
+ jar
+ ${project.build.directory}/integration-tests-spark-jobs-helpers
+
+
+
+
+
+ copy-test-spark-jobs-to-docker-dist
+ pre-integration-test
+
+ copy
+
+
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version}
+ ${project.version}
+ jar
+ ${project.build.directory}/docker/examples/integration-tests-jars
+
+
+ org.apache.spark
+ spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version}
+ ${project.version}
+ jar
+ ${project.build.directory}/docker/examples/integration-tests-jars
+
+
+
+
+
+ unpack-docker-bundle
+ pre-integration-test
+
+ unpack
+
+
+
+
+ org.apache.spark
+ spark-docker-minimal-bundle_${scala.binary.version}
+ ${project.version}
+ docker-dist
+ tar.gz
+ true
+ ${project.build.directory}/docker/
+
+
+
+
+
+
+
+ maven-resources-plugin
+ 3.0.2
+
+
+ copy-integration-test-http-server-dockerfile
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/dockerfiles
+
+
+ src/main/docker
+ true
+
+
+
+
+
+ copy-integration-python
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/python
+
+
+ ${project.parent.basedir}/python
+
+ ${project.parent.basedir}/python/.egg
+ ${project.parent.basedir}/python/dist
+
+
+
+
+
+
+ copy-integration-data
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/data
+
+
+ ${project.parent.basedir}/data
+ true
+
+
+
+
+
+ copy-integration-licenses
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/licenses
+
+
+ ${project.parent.basedir}/licenses
+ true
+
+
+
+
+
+ copy-integration-examples-jar
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/examples/jars
+
+
+ ${project.parent.basedir}/examples/target/scala-2.11/jars
+ true
+
+
+
+
+
+ copy-integration-examples-src
+ pre-integration-test
+
+ copy-resources
+
+
+ ${project.build.directory}/docker/examples/src/main
+
+
+ ${project.parent.basedir}/examples/src/main
+ true
+
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-antrun-plugin
+ 1.6
+
+
+ create-release-file
+ pre-integration-test
+
+ run
+
+
+
+
+
+
+
+
+
+
+ com.googlecode.maven-download-plugin
+ download-maven-plugin
+ 1.3.0
+
+
+ download-minikube-linux
+ pre-integration-test
+
+ wget
+
+
+ https://storage.googleapis.com/minikube/releases/v0.16.0/minikube-linux-amd64
+ ${project.build.directory}/minikube-bin/linux-amd64
+ minikube
+
+
+
+ download-minikube-darwin
+ pre-integration-test
+
+ wget
+
+
+ https://storage.googleapis.com/minikube/releases/v0.16.0/minikube-darwin-amd64
+ ${project.build.directory}/minikube-bin/darwin-amd64
+ minikube
+
+
+
+
+
+
+ org.scalatest
+ scalatest-maven-plugin
+
+
+ test
+
+ test
+
+
+
+ (?<!Suite)
+
+
+
+ integration-test
+ integration-test
+
+ test
+
+
+
+
+
+
+
+
+
diff --git a/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile b/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile
new file mode 100644
index 0000000000000..e26d207cf4397
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/main/docker/integration-test-asset-server/Dockerfile
@@ -0,0 +1,21 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Simple asset server that can provide the integration test jars over HTTP.
+FROM trinitronx/python-simplehttpserver:travis-12
+
+ADD examples/integration-tests-jars /var/www
diff --git a/resource-managers/kubernetes/integration-tests/src/test/python/pi.py b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py
new file mode 100755
index 0000000000000..e3f0c4aeef1b7
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/python/pi.py
@@ -0,0 +1,46 @@
+from __future__ import print_function
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+import sys
+from random import random
+from operator import add
+
+from pyspark.sql import SparkSession
+
+
+if __name__ == "__main__":
+ """
+ Usage: pi [partitions]
+ """
+ spark = SparkSession\
+ .builder\
+ .appName("PythonPi")\
+ .getOrCreate()
+
+ partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2
+ n = 100000 * partitions
+
+ def f(_):
+ x = random() * 2 - 1
+ y = random() * 2 - 1
+ return 1 if x ** 2 + y ** 2 < 1 else 0
+
+ count = spark.sparkContext.parallelize(range(1, n + 1), partitions).map(f).reduce(add)
+ print("Pi is roughly %f" % (4.0 * count / n))
+
+ spark.stop()
diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties
new file mode 100644
index 0000000000000..866126bc3c1c2
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties
@@ -0,0 +1,31 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Set everything to be logged to the file target/integration-tests.log
+log4j.rootCategory=INFO, file
+log4j.appender.file=org.apache.log4j.FileAppender
+log4j.appender.file.append=true
+log4j.appender.file.file=target/integration-tests.log
+log4j.appender.file.layout=org.apache.log4j.PatternLayout
+log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+# Ignore messages below warning level from a few verbose libraries.
+log4j.logger.com.sun.jersey=WARN
+log4j.logger.org.apache.hadoop=WARN
+log4j.logger.org.eclipse.jetty=WARN
+log4j.logger.org.mortbay=WARN
+log4j.logger.org.spark_project.jetty=WARN
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala
new file mode 100644
index 0000000000000..c6cd6a74c88d1
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala
@@ -0,0 +1,374 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.io.File
+import java.nio.file.Paths
+import java.util.UUID
+
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import io.fabric8.kubernetes.client.internal.readiness.Readiness
+import org.scalatest.BeforeAndAfter
+import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
+import org.scalatest.time.{Minutes, Seconds, Span}
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions}
+import org.apache.spark.deploy.kubernetes.SSLUtils
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube
+import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND
+import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource}
+import org.apache.spark.launcher.SparkLauncher
+import org.apache.spark.util.Utils
+
+private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter {
+ import KubernetesSuite._
+ private val testBackend = IntegrationTestBackendFactory.getTestBackend()
+ private val APP_LOCATOR_LABEL = UUID.randomUUID().toString.replaceAll("-", "")
+ private var kubernetesTestComponents: KubernetesTestComponents = _
+ private var sparkConf: SparkConf = _
+ private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _
+ private var staticAssetServerLauncher: StaticAssetServerLauncher = _
+
+ override def beforeAll(): Unit = {
+ testBackend.initialize()
+ kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient)
+ resourceStagingServerLauncher = new ResourceStagingServerLauncher(
+ kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace))
+ staticAssetServerLauncher = new StaticAssetServerLauncher(
+ kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace))
+ }
+
+ override def afterAll(): Unit = {
+ testBackend.cleanUp()
+ }
+
+ before {
+ sparkConf = kubernetesTestComponents.newSparkConf()
+ .set(INIT_CONTAINER_DOCKER_IMAGE, s"spark-init:latest")
+ .set(DRIVER_DOCKER_IMAGE, s"spark-driver:latest")
+ .set(KUBERNETES_DRIVER_LABELS, s"spark-app-locator=$APP_LOCATOR_LABEL")
+ kubernetesTestComponents.createNamespace()
+ }
+
+ after {
+ kubernetesTestComponents.deleteNamespace()
+ }
+
+ test("Run PySpark Job on file from SUBMITTER with --py-files") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ launchStagingServer(SSLOptions(), None)
+ sparkConf
+ .set(DRIVER_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
+ .set(EXECUTOR_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
+
+ runPySparkPiAndVerifyCompletion(
+ PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION,
+ Seq(PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION)
+ )
+ }
+
+ test("Run PySpark Job on file from CONTAINER with spark.jar defined") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+ sparkConf
+ .set(DRIVER_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.driverImage", "spark-driver-py:latest"))
+ .set(EXECUTOR_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.executorImage", "spark-executor-py:latest"))
+
+ runPySparkPiAndVerifyCompletion(PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION, Seq.empty[String])
+ }
+
+ test("Simple submission test with the resource staging server.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ launchStagingServer(SSLOptions(), None)
+ runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+ }
+
+ test("Enable SSL on the resource staging server") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ val keyStoreAndTrustStore = SSLUtils.generateKeyStoreTrustStorePair(
+ ipAddress = Minikube.getMinikubeIp,
+ keyStorePassword = "keyStore",
+ keyPassword = "key",
+ trustStorePassword = "trustStore")
+ sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStore",
+ keyStoreAndTrustStore.keyStore.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.trustStore",
+ keyStoreAndTrustStore.trustStore.getAbsolutePath)
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore")
+ .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key")
+ .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore")
+ launchStagingServer(SSLOptions(
+ enabled = true,
+ keyStore = Some(keyStoreAndTrustStore.keyStore),
+ trustStore = Some(keyStoreAndTrustStore.trustStore),
+ keyStorePassword = Some("keyStore"),
+ keyPassword = Some("key"),
+ trustStorePassword = Some("trustStore")),
+ None)
+ runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+ }
+
+ test("Use container-local resources without the resource staging server") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+ runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
+ }
+
+ test("Dynamic executor scaling basic test") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ launchStagingServer(SSLOptions(), None)
+ createShuffleServiceDaemonSet()
+
+ sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH))
+ sparkConf.set("spark.dynamicAllocation.enabled", "true")
+ sparkConf.set("spark.shuffle.service.enabled", "true")
+ sparkConf.set("spark.kubernetes.shuffle.labels", "app=spark-shuffle-service")
+ sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace)
+ sparkConf.set("spark.app.name", "group-by-test")
+ runSparkApplicationAndVerifyCompletion(
+ JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
+ GROUP_BY_MAIN_CLASS,
+ Seq("The Result is"),
+ Array.empty[String],
+ Seq.empty[String])
+ }
+
+ test("Use remote resources without the resource staging server.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
+ sparkConf.setJars(Seq(
+ s"$assetServerUri/${EXAMPLES_JAR_FILE.getName}",
+ s"$assetServerUri/${HELPER_JAR_FILE.getName}"
+ ))
+ runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+ }
+
+ test("Mix remote resources with submitted ones.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ launchStagingServer(SSLOptions(), None)
+ val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer()
+ sparkConf.setJars(Seq(
+ SUBMITTER_LOCAL_MAIN_APP_RESOURCE, s"$assetServerUri/${HELPER_JAR_FILE.getName}"
+ ))
+ runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+ }
+
+ test("Use key and certificate PEM files for TLS.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ val keyAndCertificate = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp)
+ launchStagingServer(
+ SSLOptions(enabled = true),
+ Some(keyAndCertificate))
+ sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true)
+ .set(
+ RESOURCE_STAGING_SERVER_CLIENT_CERT_PEM.key, keyAndCertificate.certPem.getAbsolutePath)
+ runSparkPiAndVerifyCompletion(SUBMITTER_LOCAL_MAIN_APP_RESOURCE)
+ }
+
+ test("Use client key and client cert file when requesting executors") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ sparkConf.setJars(Seq(
+ CONTAINER_LOCAL_MAIN_APP_RESOURCE,
+ CONTAINER_LOCAL_HELPER_JAR_PATH))
+ sparkConf.set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_KEY_FILE_CONF_SUFFIX",
+ kubernetesTestComponents.clientConfig.getClientKeyFile)
+ sparkConf.set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CLIENT_CERT_FILE_CONF_SUFFIX",
+ kubernetesTestComponents.clientConfig.getClientCertFile)
+ sparkConf.set(
+ s"$APISERVER_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX",
+ kubernetesTestComponents.clientConfig.getCaCertFile)
+ runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE)
+ }
+
+ test("Added files should be placed in the driver's working directory.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+ val testExistenceFileTempDir = Utils.createTempDir(namePrefix = "test-existence-file-temp-dir")
+ val testExistenceFile = new File(testExistenceFileTempDir, "input.txt")
+ Files.write(TEST_EXISTENCE_FILE_CONTENTS, testExistenceFile, Charsets.UTF_8)
+ launchStagingServer(SSLOptions(), None)
+ sparkConf.set("spark.files", testExistenceFile.getAbsolutePath)
+ runSparkApplicationAndVerifyCompletion(
+ JavaMainAppResource(SUBMITTER_LOCAL_MAIN_APP_RESOURCE),
+ FILE_EXISTENCE_MAIN_CLASS,
+ Seq(s"File found at /opt/spark/${testExistenceFile.getName} with correct contents."),
+ Array(testExistenceFile.getName, TEST_EXISTENCE_FILE_CONTENTS),
+ Seq.empty[String])
+ }
+
+ test("Use a very long application name.") {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)).setAppName("long" * 40)
+ runSparkPiAndVerifyCompletion(CONTAINER_LOCAL_MAIN_APP_RESOURCE)
+ }
+
+ private def launchStagingServer(
+ resourceStagingServerSslOptions: SSLOptions, keyAndCertPem: Option[KeyAndCertPem]): Unit = {
+ assume(testBackend.name == MINIKUBE_TEST_BACKEND)
+
+ val resourceStagingServerPort = resourceStagingServerLauncher.launchStagingServer(
+ resourceStagingServerSslOptions, keyAndCertPem)
+ val resourceStagingServerUriScheme = if (resourceStagingServerSslOptions.enabled) {
+ "https"
+ } else {
+ "http"
+ }
+ sparkConf.set(RESOURCE_STAGING_SERVER_URI,
+ s"$resourceStagingServerUriScheme://" +
+ s"${Minikube.getMinikubeIp}:$resourceStagingServerPort")
+ }
+
+ private def runSparkPiAndVerifyCompletion(appResource: String): Unit = {
+ runSparkApplicationAndVerifyCompletion(
+ JavaMainAppResource(appResource),
+ SPARK_PI_MAIN_CLASS,
+ Seq("Pi is roughly 3"),
+ Array.empty[String],
+ Seq.empty[String])
+ }
+
+ private def runPySparkPiAndVerifyCompletion(
+ appResource: String, otherPyFiles: Seq[String]): Unit = {
+ runSparkApplicationAndVerifyCompletion(
+ PythonMainAppResource(appResource),
+ PYSPARK_PI_MAIN_CLASS,
+ Seq("Submitting 5 missing tasks from ResultStage", "Pi is roughly 3"),
+ Array("5"),
+ otherPyFiles)
+ }
+
+ private def runSparkApplicationAndVerifyCompletion(
+ appResource: MainAppResource,
+ mainClass: String,
+ expectedLogOnCompletion: Seq[String],
+ appArgs: Array[String],
+ otherPyFiles: Seq[String]): Unit = {
+ val clientArguments = ClientArguments(
+ mainAppResource = appResource,
+ mainClass = mainClass,
+ driverArgs = appArgs,
+ otherPyFiles = otherPyFiles)
+ Client.run(sparkConf, clientArguments)
+ val driverPod = kubernetesTestComponents.kubernetesClient
+ .pods()
+ .withLabel("spark-app-locator", APP_LOCATOR_LABEL)
+ .list()
+ .getItems
+ .get(0)
+ Eventually.eventually(TIMEOUT, INTERVAL) {
+ expectedLogOnCompletion.foreach { e =>
+ assert(kubernetesTestComponents.kubernetesClient
+ .pods()
+ .withName(driverPod.getMetadata.getName)
+ .getLog
+ .contains(e), "The application did not complete.")
+ }
+ }
+ }
+
+ private def createShuffleServiceDaemonSet(): Unit = {
+ val ds = kubernetesTestComponents.kubernetesClient.extensions().daemonSets()
+ .createNew()
+ .withNewMetadata()
+ .withName("shuffle")
+ .endMetadata()
+ .withNewSpec()
+ .withNewTemplate()
+ .withNewMetadata()
+ .withLabels(Map("app" -> "spark-shuffle-service").asJava)
+ .endMetadata()
+ .withNewSpec()
+ .addNewVolume()
+ .withName("shuffle-dir")
+ .withNewHostPath()
+ .withPath("/tmp")
+ .endHostPath()
+ .endVolume()
+ .addNewContainer()
+ .withName("shuffle")
+ .withImage("spark-shuffle:latest")
+ .withImagePullPolicy("IfNotPresent")
+ .addNewVolumeMount()
+ .withName("shuffle-dir")
+ .withMountPath("/tmp")
+ .endVolumeMount()
+ .endContainer()
+ .endSpec()
+ .endTemplate()
+ .endSpec()
+ .done()
+
+ // wait for daemonset to become available.
+ Eventually.eventually(TIMEOUT, INTERVAL) {
+ val pods = kubernetesTestComponents.kubernetesClient.pods()
+ .withLabel("app", "spark-shuffle-service").list().getItems
+
+ if (pods.size() == 0 || !Readiness.isReady(pods.get(0))) {
+ throw ShuffleNotReadyException
+ }
+ }
+ }
+}
+
+private[spark] object KubernetesSuite {
+ val EXAMPLES_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs")
+ .toFile
+ .listFiles()(0)
+
+ val HELPER_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs-helpers")
+ .toFile
+ .listFiles()(0)
+ val SUBMITTER_LOCAL_MAIN_APP_RESOURCE = s"file://${EXAMPLES_JAR_FILE.getAbsolutePath}"
+ val CONTAINER_LOCAL_MAIN_APP_RESOURCE = s"local:///opt/spark/examples/" +
+ s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}"
+ val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" +
+ s"integration-tests-jars/${HELPER_JAR_FILE.getName}"
+ val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
+ val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
+ val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
+ ".integrationtest.jobs.SparkPiWithInfiniteWait"
+ val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner"
+ val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION =
+ "local:///opt/spark/examples/src/main/python/pi.py"
+ val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION =
+ "local:///opt/spark/examples/src/main/python/sort.py"
+ val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py"
+ val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
+ ".integrationtest.jobs.FileExistenceTest"
+ val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" +
+ ".integrationtest.jobs.GroupByTest"
+ val TEST_EXISTENCE_FILE_CONTENTS = "contents"
+
+ case object ShuffleNotReadyException extends Exception
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala
new file mode 100644
index 0000000000000..0ca1f482269db
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.util.UUID
+
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
+import org.scalatest.concurrent.Eventually
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SparkConf
+import org.apache.spark.deploy.kubernetes.config._
+
+private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) {
+
+ val namespace = UUID.randomUUID().toString.replaceAll("-", "")
+ val kubernetesClient = defaultClient.inNamespace(namespace)
+ val clientConfig = kubernetesClient.getConfiguration
+
+ def createNamespace(): Unit = {
+ defaultClient.namespaces.createNew()
+ .withNewMetadata()
+ .withName(namespace)
+ .endMetadata()
+ .done()
+ }
+
+ def deleteNamespace(): Unit = {
+ defaultClient.namespaces.withName(namespace).delete()
+ Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) {
+ val namespaceList = defaultClient
+ .namespaces()
+ .list()
+ .getItems()
+ .asScala
+ require(!namespaceList.exists(_.getMetadata.getName == namespace))
+ }
+ }
+
+ def newSparkConf(): SparkConf = {
+ new SparkConf(true)
+ .setMaster(s"k8s://${kubernetesClient.getMasterUrl}")
+ .set(KUBERNETES_NAMESPACE, namespace)
+ .set(DRIVER_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.driverImage", "spark-driver:latest"))
+ .set(EXECUTOR_DOCKER_IMAGE,
+ System.getProperty("spark.docker.test.executorImage", "spark-executor:latest"))
+ .setJars(Seq(KubernetesSuite.HELPER_JAR_FILE.getAbsolutePath))
+ .set("spark.executor.memory", "500m")
+ .set("spark.executor.cores", "1")
+ .set("spark.executors.instances", "1")
+ .set("spark.app.name", "spark-test-app")
+ .set("spark.ui.enabled", "true")
+ .set("spark.testing", "false")
+ .set(WAIT_FOR_APP_COMPLETION, false)
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala
new file mode 100644
index 0000000000000..4008007b72fc4
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.io.{BufferedReader, InputStreamReader}
+import java.util.concurrent.TimeUnit
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+object ProcessUtils extends Logging {
+ /**
+ * executeProcess is used to run a command and return the output if it
+ * completes within timeout seconds.
+ */
+ def executeProcess(fullCommand: Array[String], timeout: Long): Seq[String] = {
+ val pb = new ProcessBuilder().command(fullCommand: _*)
+ pb.redirectErrorStream(true)
+ val proc = pb.start()
+ val outputLines = new ArrayBuffer[String]
+
+ Utils.tryWithResource(new InputStreamReader(proc.getInputStream)) { procOutput =>
+ Utils.tryWithResource(new BufferedReader(procOutput)) { (bufferedOutput: BufferedReader) =>
+ var line: String = null
+ do {
+ line = bufferedOutput.readLine()
+ if (line != null) {
+ logInfo(line)
+ outputLines += line
+ }
+ } while (line != null)
+ }
+ }
+ assert(proc.waitFor(timeout, TimeUnit.SECONDS),
+ s"Timed out while executing ${fullCommand.mkString(" ")}")
+ assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}")
+ outputLines.toSeq
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala
new file mode 100644
index 0000000000000..e5e1b1f085f9f
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.io.{File, StringWriter}
+import java.util.Properties
+
+import com.google.common.io.{BaseEncoding, Files}
+import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, Endpoints, HTTPGetActionBuilder, KeyToPathBuilder, Pod, PodBuilder, SecretBuilder, ServiceBuilder}
+import io.fabric8.kubernetes.client.KubernetesClient
+import scala.collection.JavaConverters._
+
+import org.apache.spark.SSLOptions
+import org.apache.spark.deploy.kubernetes.config._
+import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, KeyAndCertPem}
+import org.apache.spark.util.Utils
+
+/**
+ * Launches a pod that runs the resource staging server, exposing it over a NodePort.
+ */
+private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesClient) {
+
+ private val SECRETS_ROOT_DIR = "/mnt/secrets/spark-staging"
+ private val KEYSTORE_SECRET_KEY = "keyStore"
+ private val KEYSTORE_FILE = s"$SECRETS_ROOT_DIR/$KEYSTORE_SECRET_KEY"
+ private val KEY_PEM_SECRET_KEY = "keyPem"
+ private val CERT_PEM_SECRET_KEY = "certPem"
+ private val KEY_PEM_FILE = s"$SECRETS_ROOT_DIR/$KEY_PEM_SECRET_KEY"
+ private val CERT_PEM_FILE = s"$SECRETS_ROOT_DIR/$CERT_PEM_SECRET_KEY"
+ private val SSL_SECRET_NAME = "resource-staging-server-ssl-secrets"
+ private val PROPERTIES_FILE_NAME = "staging-server.properties"
+ private val PROPERTIES_DIR = "/var/data/spark-staging-server"
+ private val PROPERTIES_FILE_PATH = s"$PROPERTIES_DIR/$PROPERTIES_FILE_NAME"
+
+ // Returns the NodePort the staging server is listening on
+ def launchStagingServer(
+ sslOptions: SSLOptions,
+ keyAndCertPem: Option[KeyAndCertPem] = None): Int = {
+ val stagingServerProperties = new Properties()
+ val stagingServerSecret = sslOptions.keyStore.map { keyStore =>
+ val keyStoreBytes = Files.toByteArray(keyStore)
+ val keyStoreBase64 = BaseEncoding.base64().encode(keyStoreBytes)
+ Map(KEYSTORE_SECRET_KEY -> keyStoreBase64)
+ }.orElse {
+ keyAndCertPem.map { keyAndCert =>
+ val keyPemBytes = Files.toByteArray(keyAndCert.keyPem)
+ val keyPemBase64 = BaseEncoding.base64().encode(keyPemBytes)
+ val certPemBytes = Files.toByteArray(keyAndCert.certPem)
+ val certPemBase64 = BaseEncoding.base64().encode(certPemBytes)
+ Map(KEY_PEM_SECRET_KEY -> keyPemBase64, CERT_PEM_SECRET_KEY -> certPemBase64)
+ }
+ }.map { secretData =>
+ new SecretBuilder()
+ .withNewMetadata().withName(SSL_SECRET_NAME).endMetadata()
+ .withData(secretData.asJava)
+ .build()
+ }
+ stagingServerProperties.setProperty(
+ RESOURCE_STAGING_SERVER_SSL_ENABLED.key, sslOptions.enabled.toString)
+ sslOptions.keyStorePassword.foreach { password =>
+ stagingServerProperties.setProperty(
+ "spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", password)
+ }
+ sslOptions.keyPassword.foreach { password =>
+ stagingServerProperties.setProperty(
+ "spark.ssl.kubernetes.resourceStagingServer.keyPassword", password)
+ }
+ sslOptions.keyStore.foreach { _ =>
+ stagingServerProperties.setProperty(
+ "spark.ssl.kubernetes.resourceStagingServer.keyStore", KEYSTORE_FILE)
+ }
+ keyAndCertPem.foreach { _ =>
+ stagingServerProperties.setProperty(
+ RESOURCE_STAGING_SERVER_KEY_PEM.key, KEY_PEM_FILE)
+ }
+ keyAndCertPem.foreach { _ =>
+ stagingServerProperties.setProperty(
+ RESOURCE_STAGING_SERVER_CERT_PEM.key, CERT_PEM_FILE)
+ }
+ val propertiesWriter = new StringWriter()
+ stagingServerProperties.store(propertiesWriter, "Resource staging server properties.")
+ val stagingServerConfigMap = new ConfigMapBuilder()
+ .withNewMetadata()
+ .withName(s"staging-server-properties")
+ .endMetadata()
+ .addToData("staging-server", propertiesWriter.toString)
+ .build()
+ val probePingHttpGet = new HTTPGetActionBuilder()
+ .withScheme(if (sslOptions.enabled) "HTTPS" else "HTTP")
+ .withPath("/api/v0/ping")
+ .withNewPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get)
+ .build()
+ val basePod = new PodBuilder()
+ .withNewMetadata()
+ .withName("resource-staging-server")
+ .addToLabels("resource-staging-server", "staging-server")
+ .endMetadata()
+ .withNewSpec()
+ .addNewVolume()
+ .withName("staging-server-properties")
+ .withNewConfigMap()
+ .withName(stagingServerConfigMap.getMetadata.getName)
+ .withItems(
+ new KeyToPathBuilder()
+ .withKey("staging-server")
+ .withPath(PROPERTIES_FILE_NAME)
+ .build())
+ .endConfigMap()
+ .endVolume()
+ .addNewContainer()
+ .withName("staging-server-container")
+ .withImage("spark-resource-staging-server:latest")
+ .withImagePullPolicy("IfNotPresent")
+ .withNewReadinessProbe()
+ .withHttpGet(probePingHttpGet)
+ .endReadinessProbe()
+ .addNewVolumeMount()
+ .withName("staging-server-properties")
+ .withMountPath(PROPERTIES_DIR)
+ .endVolumeMount()
+ .addToArgs(PROPERTIES_FILE_PATH)
+ .endContainer()
+ .endSpec()
+ val withMountedKeyStorePod = stagingServerSecret.map { secret =>
+ basePod.editSpec()
+ .addNewVolume()
+ .withName("keystore-volume")
+ .withNewSecret()
+ .withSecretName(secret.getMetadata.getName)
+ .endSecret()
+ .endVolume()
+ .editMatchingContainer(new ContainerNameEqualityPredicate("staging-server-container"))
+ .addNewVolumeMount()
+ .withName("keystore-volume")
+ .withMountPath(SECRETS_ROOT_DIR)
+ .endVolumeMount()
+ .endContainer()
+ .endSpec()
+ }.getOrElse(basePod).build()
+ val stagingServerService = new ServiceBuilder()
+ .withNewMetadata()
+ .withName("resource-staging-server")
+ .endMetadata()
+ .withNewSpec()
+ .withType("NodePort")
+ .addToSelector("resource-staging-server", "staging-server")
+ .addNewPort()
+ .withName("staging-server-port")
+ .withPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get)
+ .withNewTargetPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get)
+ .endPort()
+ .endSpec()
+ .build()
+ val stagingServerPodReadyWatcher = new SparkReadinessWatcher[Pod]
+ val serviceReadyWatcher = new SparkReadinessWatcher[Endpoints]
+ val allResources = Seq(
+ stagingServerService,
+ stagingServerConfigMap,
+ withMountedKeyStorePod) ++
+ stagingServerSecret.toSeq
+ Utils.tryWithResource(kubernetesClient.pods()
+ .withName(withMountedKeyStorePod.getMetadata.getName)
+ .watch(stagingServerPodReadyWatcher)) { _ =>
+ Utils.tryWithResource(kubernetesClient.endpoints()
+ .withName(stagingServerService.getMetadata.getName)
+ .watch(serviceReadyWatcher)) { _ =>
+ kubernetesClient.resourceList(allResources: _*).createOrReplace()
+ stagingServerPodReadyWatcher.waitUntilReady()
+ serviceReadyWatcher.waitUntilReady()
+ }
+ }
+ kubernetesClient.services().withName(stagingServerService.getMetadata.getName).get()
+ .getSpec
+ .getPorts
+ .get(0)
+ .getNodePort
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala
new file mode 100644
index 0000000000000..20517eb2fc2a6
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import java.util.concurrent.TimeUnit
+
+import com.google.common.util.concurrent.SettableFuture
+import io.fabric8.kubernetes.api.model.HasMetadata
+import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher}
+import io.fabric8.kubernetes.client.Watcher.Action
+import io.fabric8.kubernetes.client.internal.readiness.Readiness
+
+private[spark] class SparkReadinessWatcher[T <: HasMetadata] extends Watcher[T] {
+
+ private val signal = SettableFuture.create[Boolean]
+
+ override def eventReceived(action: Action, resource: T): Unit = {
+ if ((action == Action.MODIFIED || action == Action.ADDED) &&
+ Readiness.isReady(resource)) {
+ signal.set(true)
+ }
+ }
+
+ override def onClose(cause: KubernetesClientException): Unit = {}
+
+ def waitUntilReady(): Boolean = signal.get(30, TimeUnit.SECONDS)
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala
new file mode 100644
index 0000000000000..6b483769f5254
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+import io.fabric8.kubernetes.api.model.{HTTPGetActionBuilder, Pod}
+import io.fabric8.kubernetes.client.KubernetesClient
+
+import org.apache.spark.util.Utils
+
+/**
+ * Launches a simple HTTP server which provides jars that can be downloaded by Spark applications
+ * in integration tests.
+ */
+private[spark] class StaticAssetServerLauncher(kubernetesClient: KubernetesClient) {
+
+ // Returns the HTTP Base URI of the server.
+ def launchStaticAssetServer(): String = {
+ val readinessWatcher = new SparkReadinessWatcher[Pod]
+ val probePingHttpGet = new HTTPGetActionBuilder()
+ .withNewPort(8080)
+ .withScheme("HTTP")
+ .withPath("/")
+ .build()
+ Utils.tryWithResource(kubernetesClient
+ .pods()
+ .withName("integration-test-static-assets")
+ .watch(readinessWatcher)) { _ =>
+ val pod = kubernetesClient.pods().createNew()
+ .withNewMetadata()
+ .withName("integration-test-static-assets")
+ .endMetadata()
+ .withNewSpec()
+ .addNewContainer()
+ .withName("static-asset-server-container")
+ .withImage("spark-integration-test-asset-server:latest")
+ .withImagePullPolicy("IfNotPresent")
+ .withNewReadinessProbe()
+ .withHttpGet(probePingHttpGet)
+ .endReadinessProbe()
+ .endContainer()
+ .endSpec()
+ .done()
+ readinessWatcher.waitUntilReady()
+ val podIP = kubernetesClient.pods().withName(pod.getMetadata.getName).get()
+ .getStatus
+ .getPodIP
+ s"http://$podIP:8080"
+ }
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala
new file mode 100644
index 0000000000000..1ef096be4af02
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE
+
+import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+
+import org.apache.spark.deploy.kubernetes.config.resolveK8sMaster
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend
+import org.apache.spark.deploy.kubernetes.integrationtest.constants.GCE_TEST_BACKEND
+
+private[spark] class GCETestBackend(val master: String) extends IntegrationTestBackend {
+ private var defaultClient: DefaultKubernetesClient = _
+
+ override def initialize(): Unit = {
+ var k8ConfBuilder = new ConfigBuilder()
+ .withApiVersion("v1")
+ .withMasterUrl(resolveK8sMaster(master))
+ defaultClient = new DefaultKubernetesClient(k8ConfBuilder.build)
+ }
+
+ override def getKubernetesClient(): DefaultKubernetesClient = {
+ defaultClient
+ }
+
+ override def name(): String = GCE_TEST_BACKEND
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala
new file mode 100644
index 0000000000000..c5bc923dd51a6
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.kubernetes.integrationtest.backend
+
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
+
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE.GCETestBackend
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend}
+import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder
+
+private[spark] trait IntegrationTestBackend {
+ def name(): String
+ def initialize(): Unit
+ def getKubernetesClient(): DefaultKubernetesClient
+ def cleanUp(): Unit = {}
+}
+
+private[spark] object IntegrationTestBackendFactory {
+ def getTestBackend(): IntegrationTestBackend = {
+ Option(System.getProperty("spark.kubernetes.test.master"))
+ .map(new GCETestBackend(_))
+ .getOrElse(new MinikubeTestBackend())
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala
new file mode 100644
index 0000000000000..7c4b344e8f72b
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube
+
+import java.nio.file.Paths
+
+import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient}
+
+import org.apache.spark.deploy.kubernetes.integrationtest.ProcessUtils
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.Utils
+
+// TODO support windows
+private[spark] object Minikube extends Logging {
+ private val MINIKUBE_EXECUTABLE_DEST = if (Utils.isMac) {
+ Paths.get("target", "minikube-bin", "darwin-amd64", "minikube").toFile
+ } else if (Utils.isWindows) {
+ throw new IllegalStateException("Executing Minikube based integration tests not yet " +
+ " available on Windows.")
+ } else {
+ Paths.get("target", "minikube-bin", "linux-amd64", "minikube").toFile
+ }
+
+ private val EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE = "Minikube is not downloaded, expected at " +
+ s"${MINIKUBE_EXECUTABLE_DEST.getAbsolutePath}"
+
+ private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60
+
+ def startMinikube(): Unit = synchronized {
+ assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE)
+ if (getMinikubeStatus != MinikubeStatus.RUNNING) {
+ executeMinikube("start", "--memory", "6000", "--cpus", "8")
+ } else {
+ logInfo("Minikube is already started.")
+ }
+ }
+
+ def getMinikubeIp: String = synchronized {
+ assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE)
+ val outputs = executeMinikube("ip")
+ .filter(_.matches("^\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}\\.\\d{1,3}$"))
+ assert(outputs.size == 1, "Unexpected amount of output from minikube ip")
+ outputs.head
+ }
+
+ def getMinikubeStatus: MinikubeStatus.Value = synchronized {
+ assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE)
+ val statusString = executeMinikube("status")
+ .filter(_.contains("minikubeVM: "))
+ .head
+ .replaceFirst("minikubeVM: ", "")
+ MinikubeStatus.unapply(statusString)
+ .getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))
+ }
+
+ def getDockerEnv: Map[String, String] = synchronized {
+ assert(MINIKUBE_EXECUTABLE_DEST.exists(), EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE)
+ executeMinikube("docker-env", "--shell", "bash")
+ .filter(_.startsWith("export"))
+ .map(_.replaceFirst("export ", "").split('='))
+ .map(arr => (arr(0), arr(1).replaceAllLiterally("\"", "")))
+ .toMap
+ }
+
+ def deleteMinikube(): Unit = synchronized {
+ assert(MINIKUBE_EXECUTABLE_DEST.exists, EXPECTED_DOWNLOADED_MINIKUBE_MESSAGE)
+ if (getMinikubeStatus != MinikubeStatus.DOES_NOT_EXIST) {
+ executeMinikube("delete")
+ } else {
+ logInfo("Minikube was already not running.")
+ }
+ }
+
+ def getKubernetesClient: DefaultKubernetesClient = synchronized {
+ val kubernetesMaster = s"https://${getMinikubeIp}:8443"
+ val userHome = System.getProperty("user.home")
+ val kubernetesConf = new ConfigBuilder()
+ .withApiVersion("v1")
+ .withMasterUrl(kubernetesMaster)
+ .withCaCertFile(Paths.get(userHome, ".minikube", "ca.crt").toFile.getAbsolutePath)
+ .withClientCertFile(Paths.get(userHome, ".minikube", "apiserver.crt").toFile.getAbsolutePath)
+ .withClientKeyFile(Paths.get(userHome, ".minikube", "apiserver.key").toFile.getAbsolutePath)
+ .build()
+ new DefaultKubernetesClient(kubernetesConf)
+ }
+
+ def executeMinikubeSsh(command: String): Unit = {
+ executeMinikube("ssh", command)
+ }
+
+ private def executeMinikube(action: String, args: String*): Seq[String] = {
+ if (!MINIKUBE_EXECUTABLE_DEST.canExecute) {
+ if (!MINIKUBE_EXECUTABLE_DEST.setExecutable(true)) {
+ throw new IllegalStateException("Failed to make the Minikube binary executable.")
+ }
+ }
+ ProcessUtils.executeProcess(Array(MINIKUBE_EXECUTABLE_DEST.getAbsolutePath, action) ++ args,
+ MINIKUBE_STARTUP_TIMEOUT_SECONDS)
+ }
+}
+
+private[spark] object MinikubeStatus extends Enumeration {
+
+ val RUNNING = status("Running")
+ val STOPPED = status("Stopped")
+ val DOES_NOT_EXIST = status("Does Not Exist")
+ val SAVED = status("Saved")
+
+ def status(value: String): Value = new Val(nextId, value)
+ def unapply(s: String): Option[Value] = values.find(s == _.toString)
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala
new file mode 100644
index 0000000000000..461264877edc2
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube
+
+import io.fabric8.kubernetes.client.DefaultKubernetesClient
+
+import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend
+import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND
+import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder
+
+private[spark] class MinikubeTestBackend extends IntegrationTestBackend {
+ private var defaultClient: DefaultKubernetesClient = _
+
+ override def initialize(): Unit = {
+ Minikube.startMinikube()
+ new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages()
+ defaultClient = Minikube.getKubernetesClient
+ }
+
+ override def getKubernetesClient(): DefaultKubernetesClient = {
+ defaultClient
+ }
+
+ override def cleanUp(): Unit = {
+ if (!System.getProperty("spark.docker.test.persistMinikube", "false").toBoolean) {
+ Minikube.deleteMinikube()
+ }
+ }
+
+ override def name(): String = MINIKUBE_TEST_BACKEND
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala
new file mode 100644
index 0000000000000..bfded1003fc25
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest
+
+package object constants {
+ val MINIKUBE_TEST_BACKEND = "minikube"
+ val GCE_TEST_BACKEND = "gce"
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala
new file mode 100644
index 0000000000000..e240fcf953f8c
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.docker
+
+import java.io.File
+import java.net.URI
+import java.nio.file.Paths
+
+import scala.collection.JavaConverters._
+
+import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler}
+import org.apache.http.client.utils.URIBuilder
+import org.scalatest.concurrent.{Eventually, PatienceConfiguration}
+import org.scalatest.time.{Minutes, Seconds, Span}
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.RedirectThread
+
+
+
+private[spark] class SparkDockerImageBuilder
+ (private val dockerEnv: Map[String, String]) extends Logging{
+
+ private val DOCKER_BUILD_PATH = Paths.get("target", "docker")
+ // Dockerfile paths must be relative to the build path.
+ private val BASE_DOCKER_FILE = "dockerfiles/spark-base/Dockerfile"
+ private val DRIVER_DOCKER_FILE = "dockerfiles/driver/Dockerfile"
+ private val DRIVERPY_DOCKER_FILE = "dockerfiles/driver-py/Dockerfile"
+ private val EXECUTOR_DOCKER_FILE = "dockerfiles/executor/Dockerfile"
+ private val EXECUTORPY_DOCKER_FILE = "dockerfiles/executor-py/Dockerfile"
+ private val SHUFFLE_SERVICE_DOCKER_FILE = "dockerfiles/shuffle-service/Dockerfile"
+ private val INIT_CONTAINER_DOCKER_FILE = "dockerfiles/init-container/Dockerfile"
+ private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-server/Dockerfile"
+ private val STATIC_ASSET_SERVER_DOCKER_FILE =
+ "dockerfiles/integration-test-asset-server/Dockerfile"
+ private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes))
+ private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds))
+ private val dockerHost = dockerEnv.getOrElse("DOCKER_HOST",
+ throw new IllegalStateException("DOCKER_HOST env not found."))
+
+ private val originalDockerUri = URI.create(dockerHost)
+ private val httpsDockerUri = new URIBuilder()
+ .setHost(originalDockerUri.getHost)
+ .setPort(originalDockerUri.getPort)
+ .setScheme("https")
+ .build()
+
+ private val dockerCerts = dockerEnv.getOrElse("DOCKER_CERT_PATH",
+ throw new IllegalStateException("DOCKER_CERT_PATH env not found."))
+
+ private val dockerClient = new DefaultDockerClient.Builder()
+ .uri(httpsDockerUri)
+ .dockerCertificates(DockerCertificates
+ .builder()
+ .dockerCertPath(Paths.get(dockerCerts))
+ .build().get())
+ .build()
+
+ def buildSparkDockerImages(): Unit = {
+ Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() }
+ // Building Python distribution environment
+ val pythonExec = sys.env.get("PYSPARK_DRIVER_PYTHON")
+ .orElse(sys.env.get("PYSPARK_PYTHON"))
+ .getOrElse("/usr/bin/python")
+ val builder = new ProcessBuilder(
+ Seq(pythonExec, "setup.py", "sdist").asJava)
+ builder.directory(new File(DOCKER_BUILD_PATH.toFile, "python"))
+ builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize
+ val process = builder.start()
+ new RedirectThread(process.getInputStream, System.out, "redirect output").start()
+ val exitCode = process.waitFor()
+ if (exitCode != 0) {
+ logInfo(s"exitCode: $exitCode")
+ }
+ buildImage("spark-base", BASE_DOCKER_FILE)
+ buildImage("spark-driver", DRIVER_DOCKER_FILE)
+ buildImage("spark-driver-py", DRIVERPY_DOCKER_FILE)
+ buildImage("spark-executor", EXECUTOR_DOCKER_FILE)
+ buildImage("spark-executor-py", EXECUTORPY_DOCKER_FILE)
+ buildImage("spark-shuffle", SHUFFLE_SERVICE_DOCKER_FILE)
+ buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE)
+ buildImage("spark-init", INIT_CONTAINER_DOCKER_FILE)
+ buildImage("spark-integration-test-asset-server", STATIC_ASSET_SERVER_DOCKER_FILE)
+ }
+
+ private def buildImage(name: String, dockerFile: String): Unit = {
+ dockerClient.build(
+ DOCKER_BUILD_PATH,
+ name,
+ dockerFile,
+ new LoggingBuildHandler())
+ }
+}
diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala
new file mode 100644
index 0000000000000..7a3b06b1b5e58
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.deploy.kubernetes.integrationtest.restapis
+
+import java.util.{List => JList}
+import javax.ws.rs._
+import javax.ws.rs.core.MediaType
+
+import org.apache.spark.status.api.v1._
+
+@Path("/api/v1")
+@Consumes(Array(MediaType.APPLICATION_JSON))
+@Produces(Array(MediaType.APPLICATION_JSON))
+trait SparkRestApiV1 {
+
+ @GET
+ @Path("/applications")
+ @Consumes(Array(MediaType.APPLICATION_JSON))
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ def getApplications(
+ @QueryParam("status") applicationStatuses: JList[ApplicationStatus]): Seq[ApplicationInfo]
+
+ @GET
+ @Path("applications/{appId}/stages")
+ @Consumes(Array(MediaType.APPLICATION_JSON))
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ def getStages(
+ @PathParam("appId") appId: String,
+ @QueryParam("status") statuses: JList[StageStatus]): Seq[StageData]
+
+ @GET
+ @Path("applications/{appId}/executors")
+ @Consumes(Array(MediaType.APPLICATION_JSON))
+ @Produces(Array(MediaType.APPLICATION_JSON))
+ def getExecutors(@PathParam("appId") appId: String): Seq[ExecutorSummary]
+}
diff --git a/resource-managers/kubernetes/integration-tests/test-data/input.txt b/resource-managers/kubernetes/integration-tests/test-data/input.txt
new file mode 100644
index 0000000000000..dfe437bdebebc
--- /dev/null
+++ b/resource-managers/kubernetes/integration-tests/test-data/input.txt
@@ -0,0 +1 @@
+Contents
diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml
index c58e0f43b2ac7..160f0aedd2293 100644
--- a/sql/catalyst/pom.xml
+++ b/sql/catalyst/pom.xml
@@ -22,7 +22,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../../pom.xml
diff --git a/sql/core/pom.xml b/sql/core/pom.xml
index 37e7dccd2e27d..80adae9015935 100644
--- a/sql/core/pom.xml
+++ b/sql/core/pom.xml
@@ -22,7 +22,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../../pom.xml
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala
index e219cfde12656..41c096ae2b824 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala
@@ -68,12 +68,12 @@ class StateOperatorProgress private[sql](
* incremented.
* @param durationMs The amount of time taken to perform various operations in milliseconds.
* @param eventTime Statistics of event time seen in this batch. It may contain the following keys:
- * {
+ * {{{
* "max" -> "2016-12-05T20:54:20.827Z" // maximum event time seen in this trigger
* "min" -> "2016-12-05T20:54:20.827Z" // minimum event time seen in this trigger
* "avg" -> "2016-12-05T20:54:20.827Z" // average event time seen in this trigger
* "watermark" -> "2016-12-05T20:54:20.827Z" // watermark used in this trigger
- * }
+ * }}}
* All timestamps are in ISO8601 format, i.e. UTC timestamps.
* @param stateOperators Information about operators in the query that store state.
* @param sources detailed statistics on data being read from each of the streaming sources.
diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml
index 468d758a77884..5ae2130732055 100644
--- a/sql/hive-thriftserver/pom.xml
+++ b/sql/hive-thriftserver/pom.xml
@@ -22,7 +22,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../../pom.xml
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
index 7bf4fc0df45e8..d1bbaea15f557 100644
--- a/sql/hive/pom.xml
+++ b/sql/hive/pom.xml
@@ -22,7 +22,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../../pom.xml
diff --git a/streaming/pom.xml b/streaming/pom.xml
index 06569e6ee2231..72e558d5aae77 100644
--- a/streaming/pom.xml
+++ b/streaming/pom.xml
@@ -21,7 +21,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../pom.xml
diff --git a/tools/pom.xml b/tools/pom.xml
index 35d53b30191a5..dc3f922ba0ccb 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -20,7 +20,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../pom.xml
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 38374b5ae5a3b..f4d5f05a810be 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -20,7 +20,7 @@
org.apache.spark
spark-parent_2.11
- 2.1.0
+ 2.1.0-k8s-0.3.1-SNAPSHOT
../pom.xml