diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala new file mode 100644 index 0000000000000..45b881a8a3737 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, Secret} + +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait InitContainerResourceStagingServerSecretPlugin { + + /** + * Configure the init-container to mount the secret files that allow it to retrieve dependencies + * from a resource staging server. + */ + def mountResourceStagingServerSecretIntoInitContainer( + initContainer: ContainerBuilder): ContainerBuilder + + /** + * Configure the pod to attach a Secret volume which hosts secret files allowing the + * init-container to retrieve dependencies from the resource staging server. + */ + def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder +} + +private[spark] class InitContainerResourceStagingServerSecretPluginImpl( + initContainerSecretName: String, + initContainerSecretMountPath: String) + extends InitContainerResourceStagingServerSecretPlugin { + + override def mountResourceStagingServerSecretIntoInitContainer( + initContainer: ContainerBuilder): ContainerBuilder = { + initContainer.addNewVolumeMount() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withMountPath(initContainerSecretMountPath) + .endVolumeMount() + } + + override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder): PodBuilder = { + basePod.editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(initContainerSecretName) + .endSecret() + .endVolume() + .endSpec() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala new file mode 100644 index 0000000000000..227420db4636d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.v2.{ContainerNameEqualityPredicate, InitContainerUtil} + +private[spark] trait SparkPodInitContainerBootstrap { + /** + * Bootstraps an init-container that downloads dependencies to be used by a main container. + * Note that this primarily assumes that the init-container's configuration is being provided + * by a ConfigMap that was installed by some other component; that is, the implementation + * here makes no assumptions about how the init-container is specifically configured. For + * example, this class is unaware if the init-container is fetching remote dependencies or if + * it is fetching dependencies from a resource staging server. + */ + def bootstrapInitContainerAndVolumes( + mainContainerName: String, originalPodSpec: PodBuilder): PodBuilder +} + +private[spark] class SparkPodInitContainerBootstrapImpl( + initContainerImage: String, + jarsDownloadPath: String, + filesDownloadPath: String, + downloadTimeoutMinutes: Long, + initContainerConfigMapName: String, + initContainerConfigMapKey: String, + resourceStagingServerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) + extends SparkPodInitContainerBootstrap { + + override def bootstrapInitContainerAndVolumes( + mainContainerName: String, + originalPodSpec: PodBuilder): PodBuilder = { + val sharedVolumeMounts = Seq[VolumeMount]( + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withMountPath(jarsDownloadPath) + .build(), + new VolumeMountBuilder() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withMountPath(filesDownloadPath) + .build()) + + val initContainer = new ContainerBuilder() + .withName(s"spark-init") + .withImage(initContainerImage) + .withImagePullPolicy("IfNotPresent") + .addNewVolumeMount() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR) + .endVolumeMount() + .addToVolumeMounts(sharedVolumeMounts: _*) + .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) + val resolvedInitContainer = resourceStagingServerSecretPlugin.map { plugin => + plugin.mountResourceStagingServerSecretIntoInitContainer(initContainer) + }.getOrElse(initContainer).build() + val podWithBasicVolumes = InitContainerUtil.appendInitContainer( + originalPodSpec, resolvedInitContainer) + .editSpec() + .addNewVolume() + .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) + .withNewConfigMap() + .withName(initContainerConfigMapName) + .addNewItem() + .withKey(initContainerConfigMapKey) + .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .addNewVolume() + .withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME) + .withEmptyDir(new EmptyDirVolumeSource()) + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate(mainContainerName)) + .addToVolumeMounts(sharedVolumeMounts: _*) + .endContainer() + .endSpec() + resourceStagingServerSecretPlugin.map { plugin => + plugin.addResourceStagingServerSecretVolumeToPod(podWithBasicVolumes) + }.getOrElse(podWithBasicVolumes) + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 09b2d38cb8e38..f0a39fe359227 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -349,42 +349,43 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val RESOURCE_STAGING_SERVER_SSL_NAMESPACE = "kubernetes.resourceStagingServer" private[spark] val RESOURCE_STAGING_SERVER_CERT_PEM = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.serverCertPem") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.serverCertPem") .doc("Certificate PEM file to use when having the Kubernetes dependency server" + " listen on TLS.") .stringConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_PASSWORD_FILE = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyStorePasswordFile") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.keyStorePasswordFile") .doc("File containing the keystore password for the Kubernetes dependency server.") .stringConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_KEYSTORE_KEY_PASSWORD_FILE = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.keyPasswordFile") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.keyPasswordFile") .doc("File containing the key password for the Kubernetes dependency server.") .stringConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_SSL_ENABLED = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.enabled") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.enabled") .doc("Whether or not to use SSL when communicating with the dependency server.") .booleanConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.trustStore") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStore") .doc("File containing the trustStore to communicate with the Kubernetes dependency server.") .stringConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStorePassword") .doc("Password for the trustStore for talking to the dependency server.") .stringConf .createOptional private[spark] val RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE = - ConfigBuilder("spark.ssl.kubernetes.resourceStagingServer.trustStoreType") + ConfigBuilder(s"spark.ssl.$RESOURCE_STAGING_SERVER_SSL_NAMESPACE.trustStoreType") .doc("Type of trustStore for communicating with the dependency server.") .stringConf .createOptional @@ -397,64 +398,120 @@ package object config extends Logging { .createOptional private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER = - ConfigBuilder("spark.kubernetes.driver.initcontainer.downloadJarsResourceIdentifier") + ConfigBuilder("spark.kubernetes.initcontainer.downloadJarsResourceIdentifier") .doc("Identifier for the jars tarball that was uploaded to the staging service.") .internal() .stringConf .createOptional private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION = - ConfigBuilder("spark.kubernetes.driver.initcontainer.downloadJarsSecretLocation") + ConfigBuilder("spark.kubernetes.initcontainer.downloadJarsSecretLocation") .doc("Location of the application secret to use when the init-container contacts the" + " resource staging server to download jars.") .internal() .stringConf - .createWithDefault(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_PATH) + .createWithDefault(s"$INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH/" + + s"$INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY") private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER = - ConfigBuilder("spark.kubernetes.driver.initcontainer.downloadFilesResourceIdentifier") + ConfigBuilder("spark.kubernetes.initcontainer.downloadFilesResourceIdentifier") .doc("Identifier for the files tarball that was uploaded to the staging service.") .internal() .stringConf .createOptional private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION = - ConfigBuilder("spark.kubernetes.driver.initcontainer.downloadFilesSecretLocation") + ConfigBuilder("spark.kubernetes.initcontainer.downloadFilesSecretLocation") .doc("Location of the application secret to use when the init-container contacts the" + " resource staging server to download files.") .internal() .stringConf - .createWithDefault(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_PATH) + .createWithDefault( + s"$INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH/$INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY") + + private[spark] val INIT_CONTAINER_REMOTE_JARS = + ConfigBuilder("spark.kubernetes.initcontainer.remoteJars") + .doc("Comma-separated list of jar URIs to download in the init-container. This is" + + " calculated from spark.jars.") + .internal() + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_REMOTE_FILES = + ConfigBuilder("spark.kubernetes.initcontainer.remoteFiles") + .doc("Comma-separated list of file URIs to download in the init-container. This is" + + " calculated from spark.files.") + .internal() + .stringConf + .createOptional private[spark] val INIT_CONTAINER_DOCKER_IMAGE = - ConfigBuilder("spark.kubernetes.driver.initcontainer.docker.image") - .doc("Image for the driver's init-container that downloads mounted dependencies.") + ConfigBuilder("spark.kubernetes.initcontainer.docker.image") + .doc("Image for the driver and executor's init-container that downloads dependencies.") .stringConf - .createWithDefault(s"spark-driver-init:$sparkVersion") + .createWithDefault(s"spark-init:$sparkVersion") - private[spark] val DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION = - ConfigBuilder("spark.kubernetes.driver.mountdependencies.jarsDownloadDir") - .doc("Location to download local jars to in the driver. When using spark-submit, this" + - " directory must be empty and will be mounted as an empty directory volume on the" + - " driver pod.") + private[spark] val INIT_CONTAINER_JARS_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountdependencies.jarsDownloadDir") + .doc("Location to download jars to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pod.") .stringConf - .createWithDefault("/var/spark-data/spark-local-jars") + .createWithDefault("/var/spark-data/spark-submitted-jars") - private[spark] val DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION = - ConfigBuilder("spark.kubernetes.driver.mountdependencies.filesDownloadDir") - .doc("Location to download local files to in the driver. When using spark-submit, this" + - " directory must be empty and will be mounted as an empty directory volume on the" + - " driver pod.") + private[spark] val INIT_CONTAINER_FILES_DOWNLOAD_LOCATION = + ConfigBuilder("spark.kubernetes.mountdependencies.filesDownloadDir") + .doc("Location to download files to in the driver and executors. When using" + + " spark-submit, this directory must be empty and will be mounted as an empty directory" + + " volume on the driver and executor pods.") .stringConf - .createWithDefault("/var/spark-data/spark-local-files") + .createWithDefault("/var/spark-data/spark-submitted-files") - private[spark] val DRIVER_MOUNT_DEPENDENCIES_INIT_TIMEOUT = + private[spark] val INIT_CONTAINER_MOUNT_TIMEOUT = ConfigBuilder("spark.kubernetes.mountdependencies.mountTimeout") .doc("Timeout before aborting the attempt to download and unpack local dependencies from" + - " the dependency staging server when initializing the driver pod.") + " remote locations and the resource staging server when initializing the driver and" + + " executor pods.") .timeConf(TimeUnit.MINUTES) .createWithDefault(5) + private[spark] val EXECUTOR_INIT_CONTAINER_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.initcontainer.executor.configmapname") + .doc("Name of the config map to use in the init-container that retrieves submitted files" + + " for the executor.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY = + ConfigBuilder("spark.kubernetes.initcontainer.executor.configmapkey") + .doc("Key for the entry in the init container config map for submitted files that" + + " corresponds to the properties for this init-container.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_SECRET = + ConfigBuilder("spark.kubernetes.initcontainer.executor.stagingServerSecret.name") + .doc("Name of the secret to mount into the init-container that retrieves submitted files.") + .internal() + .stringConf + .createOptional + + private[spark] val EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR = + ConfigBuilder("spark.kubernetes.initcontainer.executor.stagingServerSecret.mountDir") + .doc("Directory to mount the resource staging server secrets into for the executor" + + " init-containers. This must be exactly the same as the directory that the submission" + + " client mounted the secret into because the config map's properties specify the" + + " secret location as to be the same between the driver init-container and the executor" + + " init-container. Thus the submission client will always set this and the driver will" + + " never rely on a constant or convention, in order to protect against cases where the" + + " submission client has a different version from the driver itself, and hence might" + + " have different constants loaded in constants.scala.") + .internal() + .stringConf + .createOptional + private[spark] def resolveK8sMaster(rawMasterString: String): String = { if (!rawMasterString.startsWith("k8s://")) { throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 27e47eb61933f..4c4f7b9fc3b23 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -70,8 +70,8 @@ package object constants { private[spark] val ENV_EXECUTOR_ID = "SPARK_EXECUTOR_ID" private[spark] val ENV_EXECUTOR_POD_IP = "SPARK_EXECUTOR_POD_IP" private[spark] val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY" - private[spark] val ENV_UPLOADED_JARS_DIR = "SPARK_UPLOADED_JARS_DIR" private[spark] val ENV_SUBMIT_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" + private[spark] val ENV_EXECUTOR_EXTRA_CLASSPATH = "SPARK_SUBMIT_EXTRA_CLASSPATH" private[spark] val ENV_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" private[spark] val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" @@ -92,26 +92,21 @@ package object constants { // V2 submission init container private[spark] val INIT_CONTAINER_ANNOTATION = "pod.beta.kubernetes.io/init-containers" - private[spark] val INIT_CONTAINER_SECRETS_VOLUME_NAME = "dependency-secret" - private[spark] val INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH = "/mnt/secrets/spark-init" - private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_SECRET_KEY = "downloadJarsSecret" - private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_SECRET_KEY = "downloadFilesSecret" - private[spark] val INIT_CONTAINER_TRUSTSTORE_SECRET_KEY = "trustStore" - private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_SECRET_PATH = - s"$INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH/$INIT_CONTAINER_DOWNLOAD_JARS_SECRET_KEY" - private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_SECRET_PATH = - s"$INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH/$INIT_CONTAINER_DOWNLOAD_FILES_SECRET_KEY" - private[spark] val INIT_CONTAINER_TRUSTSTORE_PATH = - s"$INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH/$INIT_CONTAINER_TRUSTSTORE_SECRET_KEY" - private[spark] val INIT_CONTAINER_DOWNLOAD_CREDENTIALS_PATH = - "/mnt/secrets/kubernetes-credentials" - private[spark] val INIT_CONTAINER_CONFIG_MAP_KEY = "init-driver" - private[spark] val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "init-container-properties" - private[spark] val INIT_CONTAINER_PROPERTIES_FILE_MOUNT_PATH = "/etc/spark-init/" - private[spark] val INIT_CONTAINER_PROPERTIES_FILE_NAME = "init-driver.properties" + private[spark] val INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH = + "/mnt/secrets/spark-init" + private[spark] val INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY = + "downloadSubmittedJarsSecret" + private[spark] val INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY = + "downloadSubmittedFilesSecret" + private[spark] val INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY = "trustStore" + private[spark] val INIT_CONTAINER_CONFIG_MAP_KEY = "download-submitted-files" + private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume" + private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init" + private[spark] val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties" private[spark] val INIT_CONTAINER_PROPERTIES_FILE_PATH = - s"$INIT_CONTAINER_PROPERTIES_FILE_MOUNT_PATH/$INIT_CONTAINER_PROPERTIES_FILE_NAME" - private[spark] val DOWNLOAD_JARS_VOLUME_NAME = "download-jars" - private[spark] val DOWNLOAD_FILES_VOLUME_NAME = "download-files" + s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME" private[spark] val DEFAULT_SHUFFLE_MOUNT_NAME = "shuffle" + private[spark] val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala similarity index 88% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala index b8e644219097e..1b0af3fa9fb01 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes.v1 +package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.util.Utils @@ -41,4 +41,7 @@ private[spark] object KubernetesFileUtils { Option(Utils.resolveURI(uri).getScheme).getOrElse("file") == "file" } + def getOnlyRemoteFiles(uris: Iterable[String]): Iterable[String] = { + filterUriStringsByScheme(uris, scheme => scheme != "file" && scheme != "local") + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/Client.scala index a4dfe90f71a8a..0f1e7886a1ba2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/Client.scala @@ -33,7 +33,8 @@ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.kubernetes.CompressionUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.v1.{AppResource, ContainerAppResource, HttpClientUtil, KubernetesCreateSubmissionRequest, KubernetesCredentials, KubernetesFileUtils, KubernetesSparkRestApi, RemoteAppResource, UploadedAppResource} +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.rest.kubernetes.v1.{AppResource, ContainerAppResource, HttpClientUtil, KubernetesCreateSubmissionRequest, KubernetesCredentials, KubernetesSparkRestApi, RemoteAppResource, UploadedAppResource} import org.apache.spark.internal.Logging import org.apache.spark.util.{ShutdownHookManager, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverSubmitSslConfigurationProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverSubmitSslConfigurationProvider.scala index 10ffddcd7e7fc..174e9c57a65ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverSubmitSslConfigurationProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverSubmitSslConfigurationProvider.scala @@ -29,7 +29,8 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException, SSLOptions} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.v1.{KubernetesFileUtils, PemsToKeyStoreConverter} +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.rest.kubernetes.v1.PemsToKeyStoreConverter import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala index a70c93942ffb5..a403a91840bd6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala @@ -19,11 +19,10 @@ package org.apache.spark.deploy.kubernetes.submit.v2 import java.io.File import java.util.Collections -import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata, OwnerReferenceBuilder, PodBuilder} +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, OwnerReferenceBuilder, PodBuilder} import scala.collection.JavaConverters._ -import scala.collection.mutable -import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException} +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.internal.Logging @@ -35,33 +34,25 @@ import org.apache.spark.util.Utils * * This class is responsible for instantiating Kubernetes resources that allow a Spark driver to * run in a pod on the Kubernetes cluster with the Spark configurations specified by spark-submit. - * Application submitters that desire to provide their application's dependencies from their local - * disk must provide a resource staging server URI to this client so that the client can push the - * local resources to the resource staging server and have the driver pod pull the resources in an - * init-container. Interactions with the resource staging server are offloaded to the - * {@link MountedDependencyManager} class. If instead the application submitter has their - * dependencies pre-staged in remote locations like HDFS or their own HTTP servers already, then - * the mounted dependency manager is bypassed entirely, but the init-container still needs to - * fetch these remote dependencies (TODO https://github.com/apache-spark-on-k8s/spark/issues/238). + * The API of this class makes it such that much of the specific behavior can be stubbed for + * testing; most of the detailed logic must be dependency-injected when constructing an instance + * of this client. Therefore the submission process is designed to be as modular as possible, + * where different steps of submission should be factored out into separate classes. */ private[spark] class Client( + appName: String, + kubernetesAppId: String, mainClass: String, sparkConf: SparkConf, appArgs: Array[String], - mainAppResource: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], kubernetesClientProvider: SubmissionKubernetesClientProvider, - mountedDependencyManagerProvider: MountedDependencyManagerProvider) extends Logging { + initContainerComponentsProvider: DriverInitContainerComponentsProvider) extends Logging { - private val namespace = sparkConf.get(KUBERNETES_NAMESPACE) - private val master = resolveK8sMaster(sparkConf.get("spark.master")) - private val launchTime = System.currentTimeMillis - private val appName = sparkConf.getOption("spark.app.name") - .getOrElse("spark") - private val kubernetesAppId = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") private val kubernetesDriverPodName = sparkConf.get(KUBERNETES_DRIVER_POD_NAME) .getOrElse(kubernetesAppId) private val driverDockerImage = sparkConf.get(DRIVER_DOCKER_IMAGE) - private val maybeStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) private val driverMemoryMb = sparkConf.get(org.apache.spark.internal.config.DRIVER_MEMORY) private val memoryOverheadMb = sparkConf .get(KUBERNETES_DRIVER_MEMORY_OVERHEAD) @@ -70,22 +61,15 @@ private[spark] class Client( private val driverContainerMemoryWithOverhead = driverMemoryMb + memoryOverheadMb private val customLabels = sparkConf.get(KUBERNETES_DRIVER_LABELS) private val customAnnotations = sparkConf.get(KUBERNETES_DRIVER_ANNOTATIONS) - private val sparkJars = sparkConf.getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty[String]) ++ - Option(mainAppResource) - .filterNot(_ == SparkLauncher.NO_RESOURCE) - .toSeq - private val sparkFiles = sparkConf.getOption("spark.files") - .map(_.split(",")) - .getOrElse(Array.empty[String]) private val driverExtraClasspath = sparkConf.get( org.apache.spark.internal.config.DRIVER_CLASS_PATH) private val driverJavaOptions = sparkConf.get( org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) def run(): Unit = { + validateNoDuplicateFileNames(sparkJars) + validateNoDuplicateFileNames(sparkFiles) val parsedCustomLabels = parseKeyValuePairs(customLabels, KUBERNETES_DRIVER_LABELS.key, "labels") require(!parsedCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + @@ -133,50 +117,39 @@ private[spark] class Client( .addToContainers(driverContainer) .endSpec() - val nonDriverPodKubernetesResources = mutable.Buffer[HasMetadata]() - val resolvedJars = mutable.Buffer[String]() - val resolvedFiles = mutable.Buffer[String]() - val driverPodWithMountedDeps = maybeStagingServerUri.map { stagingServerUri => - val mountedDependencyManager = mountedDependencyManagerProvider.getMountedDependencyManager( - kubernetesAppId, - stagingServerUri, - allLabels, - namespace, - sparkJars, - sparkFiles) - val jarsResourceIdentifier = mountedDependencyManager.uploadJars() - val filesResourceIdentifier = mountedDependencyManager.uploadFiles() - val initContainerKubernetesSecret = mountedDependencyManager.buildInitContainerSecret( - jarsResourceIdentifier.resourceSecret, filesResourceIdentifier.resourceSecret) - val initContainerConfigMap = mountedDependencyManager.buildInitContainerConfigMap( - jarsResourceIdentifier.resourceId, filesResourceIdentifier.resourceId) - resolvedJars ++= mountedDependencyManager.resolveSparkJars() - resolvedFiles ++= mountedDependencyManager.resolveSparkFiles() - nonDriverPodKubernetesResources += initContainerKubernetesSecret - nonDriverPodKubernetesResources += initContainerConfigMap - mountedDependencyManager.configurePodToMountLocalDependencies( - driverContainer.getName, initContainerKubernetesSecret, initContainerConfigMap, basePod) - }.getOrElse { - sparkJars.map(Utils.resolveURI).foreach { jar => - require(Option.apply(jar.getScheme).getOrElse("file") != "file", - "When submitting with local jars, a resource staging server must be provided to" + - s" deploy your jars into the driver pod. Cannot send jar with URI $jar.") - } - sparkFiles.map(Utils.resolveURI).foreach { file => - require(Option.apply(file.getScheme).getOrElse("file") != "file", - "When submitting with local files, a resource staging server must be provided to" + - s" deploy your files into the driver pod. Cannot send file with URI $file") - } - resolvedJars ++= sparkJars - resolvedFiles ++= sparkFiles - basePod + val maybeSubmittedDependencyUploader = initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(allLabels) + val maybeSubmittedResourceIdentifiers = maybeSubmittedDependencyUploader.map { uploader => + SubmittedResources(uploader.uploadJars(), uploader.uploadFiles()) } - val resolvedSparkConf = sparkConf.clone() - if (resolvedJars.nonEmpty) { - resolvedSparkConf.set("spark.jars", resolvedJars.mkString(",")) + val maybeSecretBuilder = initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceIdentifiers.map(_.secrets())) + val maybeSubmittedDependenciesSecret = maybeSecretBuilder.map(_.build()) + val initContainerConfigMap = initContainerComponentsProvider + .provideInitContainerConfigMapBuilder(maybeSubmittedResourceIdentifiers.map(_.ids())) + .build() + val podWithInitContainer = initContainerComponentsProvider + .provideInitContainerBootstrap() + .bootstrapInitContainerAndVolumes(driverContainer.getName, basePod) + + val driverOwnedResources = Seq(initContainerConfigMap) ++ + maybeSubmittedDependenciesSecret.toSeq + + val containerLocalizedFilesResolver = initContainerComponentsProvider + .provideContainerLocalizedFilesResolver() + val resolvedSparkJars = containerLocalizedFilesResolver.resolveSubmittedSparkJars() + val resolvedSparkFiles = containerLocalizedFilesResolver.resolveSubmittedSparkFiles() + + val executorInitContainerConfiguration = initContainerComponentsProvider + .provideExecutorInitContainerConfiguration() + val resolvedSparkConf = executorInitContainerConfiguration + .configureSparkConfForExecutorInitContainer(sparkConf) + if (resolvedSparkJars.nonEmpty) { + resolvedSparkConf.set("spark.jars", resolvedSparkJars.mkString(",")) } - if (resolvedFiles.nonEmpty) { - resolvedSparkConf.set("spark.files", resolvedFiles.mkString(",")) + if (resolvedSparkFiles.nonEmpty) { + resolvedSparkConf.set("spark.files", resolvedSparkFiles.mkString(",")) } resolvedSparkConf.setIfMissing(KUBERNETES_DRIVER_POD_NAME, kubernetesDriverPodName) resolvedSparkConf.set("spark.app.id", kubernetesAppId) @@ -188,19 +161,16 @@ private[spark] class Client( resolvedSparkConf.get(KUBERNETES_DRIVER_OAUTH_TOKEN).foreach { _ => resolvedSparkConf.set(KUBERNETES_DRIVER_OAUTH_TOKEN.key, "") } - - val mountedClassPath = resolvedJars.map(Utils.resolveURI).filter { jarUri => - val scheme = Option.apply(jarUri.getScheme).getOrElse("file") - scheme == "local" || scheme == "file" - }.map(_.getPath).mkString(File.pathSeparator) - val resolvedDriverJavaOpts = resolvedSparkConf.getAll.map { case (confKey, confValue) => - s"-D$confKey=$confValue" + val resolvedLocalClasspath = containerLocalizedFilesResolver + .resolveSubmittedAndRemoteSparkJars() + val resolvedDriverJavaOpts = resolvedSparkConf.getAll.map { + case (confKey, confValue) => s"-D$confKey=$confValue" }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") - val resolvedDriverPod = driverPodWithMountedDeps.editSpec() + val resolvedDriverPod = podWithInitContainer.editSpec() .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainer.getName)) .addNewEnv() .withName(ENV_MOUNTED_CLASSPATH) - .withValue(mountedClassPath) + .withValue(resolvedLocalClasspath.mkString(File.pathSeparator)) .endEnv() .addNewEnv() .withName(ENV_DRIVER_JAVA_OPTS) @@ -218,11 +188,11 @@ private[spark] class Client( .withKind(createdDriverPod.getKind) .withController(true) .build() - nonDriverPodKubernetesResources.foreach { resource => + driverOwnedResources.foreach { resource => val originalMetadata = resource.getMetadata originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) } - kubernetesClient.resourceList(nonDriverPodKubernetesResources: _*).createOrReplace() + kubernetesClient.resourceList(driverOwnedResources: _*).createOrReplace() } catch { case e: Throwable => kubernetesClient.pods().delete(createdDriverPod) @@ -231,6 +201,17 @@ private[spark] class Client( } } + private def validateNoDuplicateFileNames(allFiles: Seq[String]): Unit = { + val fileNamesToUris = allFiles.map { file => + (new File(Utils.resolveURI(file).getPath).getName, file) + } + fileNamesToUris.groupBy(_._1).foreach { + case (fileName, urisWithFileName) => + require(urisWithFileName.size == 1, "Cannot add multiple files with the same name, but" + + s" file name $fileName is shared by all of these URIs: $urisWithFileName") + } + } + private def parseKeyValuePairs( maybeKeyValues: Option[String], configKey: String, @@ -249,3 +230,46 @@ private[spark] class Client( }).getOrElse(Map.empty[String, String]) } } + +private[spark] object Client { + def main(args: Array[String]): Unit = { + val sparkConf = new SparkConf(true) + val mainAppResource = args(0) + val mainClass = args(1) + val appArgs = args.drop(2) + run(sparkConf, mainAppResource, mainClass, appArgs) + } + + def run( + sparkConf: SparkConf, + mainAppResource: String, + mainClass: String, + appArgs: Array[String]): Unit = { + val sparkJars = sparkConf.getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty[String]) ++ + Option(mainAppResource) + .filterNot(_ == SparkLauncher.NO_RESOURCE) + .toSeq + val launchTime = System.currentTimeMillis + val sparkFiles = sparkConf.getOption("spark.files") + .map(_.split(",")) + .getOrElse(Array.empty[String]) + val appName = sparkConf.getOption("spark.app.name") + .getOrElse("spark") + val kubernetesAppId = s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") + val initContainerComponentsProvider = new DriverInitContainerComponentsProviderImpl( + sparkConf, kubernetesAppId, sparkJars, sparkFiles) + val kubernetesClientProvider = new SubmissionKubernetesClientProviderImpl(sparkConf) + new Client( + appName, + kubernetesAppId, + mainClass, + sparkConf, + appArgs, + sparkJars, + sparkFiles, + kubernetesClientProvider, + initContainerComponentsProvider).run() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolver.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolver.scala new file mode 100644 index 0000000000000..5505d87fa8072 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolver.scala @@ -0,0 +1,68 @@ +/* + * 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.v2 + +import java.io.File + +import org.apache.spark.util.Utils + +private[spark] trait ContainerLocalizedFilesResolver { + def resolveSubmittedAndRemoteSparkJars(): Seq[String] + def resolveSubmittedSparkJars(): Seq[String] + def resolveSubmittedSparkFiles(): Seq[String] +} + +private[spark] class ContainerLocalizedFilesResolverImpl( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String) extends ContainerLocalizedFilesResolver { + + override def resolveSubmittedAndRemoteSparkJars(): Seq[String] = { + sparkJars.map { jar => + val jarUri = Utils.resolveURI(jar) + Option(jarUri.getScheme).getOrElse("file") match { + case "local" => + jarUri.getPath + case _ => + val jarFileName = new File(jarUri.getPath).getName + s"$jarsDownloadPath/$jarFileName" + } + } + } + + override def resolveSubmittedSparkJars(): Seq[String] = { + resolveSubmittedFiles(sparkJars, jarsDownloadPath) + } + + override def resolveSubmittedSparkFiles(): Seq[String] = { + resolveSubmittedFiles(sparkFiles, filesDownloadPath) + } + + private def resolveSubmittedFiles(files: Seq[String], downloadPath: String): Seq[String] = { + files.map { file => + val fileUri = Utils.resolveURI(file) + Option(fileUri.getScheme).getOrElse("file") match { + case "file" => + val fileName = new File(fileUri.getPath).getName + s"$downloadPath/$fileName" + case _ => + file + } + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/DriverInitContainerComponentsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/DriverInitContainerComponentsProvider.scala new file mode 100644 index 0000000000000..5b649735f2b3d --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/DriverInitContainerComponentsProvider.scala @@ -0,0 +1,155 @@ +/* + * 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.v2 + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkPodInitContainerBootstrap, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.rest.kubernetes.v2.RetrofitClientFactoryImpl + +/** + * Interface that wraps the provision of everything the submission client needs to set up the + * driver's init-container. This is all wrapped in the same place to ensure that related + * components are being constructed with consistent configurations with respect to one another. + */ +private[spark] trait DriverInitContainerComponentsProvider { + + def provideInitContainerConfigMapBuilder( + maybeSubmittedResourceIds: Option[SubmittedResourceIds]) + : SparkInitContainerConfigMapBuilder + def provideContainerLocalizedFilesResolver(): ContainerLocalizedFilesResolver + def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration + def provideInitContainerSubmittedDependencyUploader( + driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] + def provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) + : Option[SubmittedDependencySecretBuilder] + def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap +} + +private[spark] class DriverInitContainerComponentsProviderImpl( + sparkConf: SparkConf, + kubernetesAppId: String, + sparkJars: Seq[String], + sparkFiles: Seq[String]) + extends DriverInitContainerComponentsProvider { + + private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) + private val resourceStagingServerSslOptions = new SecurityManager(sparkConf) + .getSSLOptions(RESOURCE_STAGING_SERVER_SSL_NAMESPACE) + private val jarsDownloadPath = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + private val filesDownloadPath = sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION) + private val maybeSecretName = maybeResourceStagingServerUri.map { _ => + s"$kubernetesAppId-init-secret" + } + private val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + private val configMapName = s"$kubernetesAppId-init-config" + private val configMapKey = s"$kubernetesAppId-init-config-key" + private val initContainerImage = sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE) + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) + + override def provideInitContainerConfigMapBuilder( + maybeSubmittedResourceIds: Option[SubmittedResourceIds]) + : SparkInitContainerConfigMapBuilder = { + val submittedDependencyConfigPlugin = for { + stagingServerUri <- maybeResourceStagingServerUri + jarsResourceId <- maybeSubmittedResourceIds.map(_.jarsResourceId) + filesResourceId <- maybeSubmittedResourceIds.map(_.filesResourceId) + } yield { + new SubmittedDependencyInitContainerConfigPluginImpl( + stagingServerUri, + jarsResourceId, + filesResourceId, + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, + resourceStagingServerSslOptions) + } + new SparkInitContainerConfigMapBuilderImpl( + sparkJars, + sparkFiles, + jarsDownloadPath, + filesDownloadPath, + configMapName, + configMapKey, + submittedDependencyConfigPlugin) + } + + override def provideContainerLocalizedFilesResolver(): ContainerLocalizedFilesResolver = { + new ContainerLocalizedFilesResolverImpl( + sparkJars, sparkFiles, jarsDownloadPath, filesDownloadPath) + } + + override def provideExecutorInitContainerConfiguration(): ExecutorInitContainerConfiguration = { + new ExecutorInitContainerConfigurationImpl( + maybeSecretName, + INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH, + configMapName, + configMapKey) + } + + override def provideInitContainerSubmittedDependencyUploader( + driverPodLabels: Map[String, String]): Option[SubmittedDependencyUploader] = { + maybeResourceStagingServerUri.map { stagingServerUri => + new SubmittedDependencyUploaderImpl( + kubernetesAppId, + driverPodLabels, + namespace, + stagingServerUri, + sparkJars, + sparkFiles, + resourceStagingServerSslOptions, + RetrofitClientFactoryImpl) + } + } + + override def provideSubmittedDependenciesSecretBuilder( + maybeSubmittedResourceSecrets: Option[SubmittedResourceSecrets]) + : Option[SubmittedDependencySecretBuilder] = { + for { + secretName <- maybeSecretName + jarsResourceSecret <- maybeSubmittedResourceSecrets.map(_.jarsResourceSecret) + filesResourceSecret <- maybeSubmittedResourceSecrets.map(_.filesResourceSecret) + } yield { + new SubmittedDependencySecretBuilderImpl( + secretName, + jarsResourceSecret, + filesResourceSecret, + INIT_CONTAINER_SUBMITTED_JARS_SECRET_KEY, + INIT_CONTAINER_SUBMITTED_FILES_SECRET_KEY, + INIT_CONTAINER_STAGING_SERVER_TRUSTSTORE_SECRET_KEY, + resourceStagingServerSslOptions) + } + } + + override def provideInitContainerBootstrap(): SparkPodInitContainerBootstrap = { + val resourceStagingServerSecretPlugin = maybeSecretName.map { secret => + new InitContainerResourceStagingServerSecretPluginImpl( + secret, INIT_CONTAINER_SECRET_VOLUME_MOUNT_PATH) + } + new SparkPodInitContainerBootstrapImpl( + initContainerImage, + jarsDownloadPath, + filesDownloadPath, + downloadTimeoutMinutes, + configMapName, + configMapKey, + resourceStagingServerSecretPlugin) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfiguration.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfiguration.scala new file mode 100644 index 0000000000000..adfdc060f0d0f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfiguration.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.v2 + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ + +private[spark] trait ExecutorInitContainerConfiguration { + /** + * Provide the driver with configuration that allows it to configure executors to + * fetch resources in the same way the driver does. + */ + def configureSparkConfForExecutorInitContainer(originalSparkConf: SparkConf): SparkConf +} + +private[spark] class ExecutorInitContainerConfigurationImpl( + initContainerSecretName: Option[String], + initContainerSecretMountDir: String, + initContainerConfigMapName: String, + initContainerConfigMapKey: String) + extends ExecutorInitContainerConfiguration { + def configureSparkConfForExecutorInitContainer(originalSparkConf: SparkConf): SparkConf = { + val configuredSparkConf = originalSparkConf.clone() + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP, + initContainerConfigMapName) + .set(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY, + initContainerConfigMapKey) + .set(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR, initContainerSecretMountDir) + initContainerSecretName.map { secret => + configuredSparkConf.set(EXECUTOR_INIT_CONTAINER_SECRET, secret) + }.getOrElse(configuredSparkConf) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/InitContainerUtil.scala new file mode 100644 index 0000000000000..0526ca53baaab --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/InitContainerUtil.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import io.fabric8.kubernetes.api.model.{Container, PodBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] object InitContainerUtil { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + + def appendInitContainer( + originalPodSpec: PodBuilder, initContainer: Container): PodBuilder = { + val resolvedInitContainers = originalPodSpec + .editMetadata() + .getAnnotations + .asScala + .get(INIT_CONTAINER_ANNOTATION) + .map { existingInitContainerAnnotation => + val existingInitContainers = OBJECT_MAPPER.readValue( + existingInitContainerAnnotation, classOf[List[Container]]) + existingInitContainers ++ Seq(initContainer) + }.getOrElse(Seq(initContainer)) + val resolvedSerializedInitContainers = OBJECT_MAPPER.writeValueAsString(resolvedInitContainers) + originalPodSpec + .editMetadata() + .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) + .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) + .endMetadata() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala deleted file mode 100644 index 9dbbcd0d56a3b..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala +++ /dev/null @@ -1,324 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.kubernetes.submit.v2 - -import java.io.{File, FileOutputStream, StringWriter} -import java.util.Properties -import javax.ws.rs.core.MediaType - -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import com.google.common.base.Charsets -import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, Container, ContainerBuilder, EmptyDirVolumeSource, PodBuilder, Secret, SecretBuilder, VolumeMount, VolumeMountBuilder} -import okhttp3.RequestBody -import retrofit2.Call -import scala.collection.JavaConverters._ -import scala.collection.mutable - -import org.apache.spark.{SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.v1.{KubernetesCredentials, KubernetesFileUtils} -import org.apache.spark.deploy.rest.kubernetes.v2.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourceIdentifier} -import org.apache.spark.util.Utils - -private[spark] trait MountedDependencyManager { - - /** - * Upload submitter-local jars to the resource staging server. - * @return The resource ID and secret to use to retrieve these jars. - */ - def uploadJars(): StagedResourceIdentifier - - /** - * Upload submitter-local files to the resource staging server. - * @return The resource ID and secret to use to retrieve these files. - */ - def uploadFiles(): StagedResourceIdentifier - - def configurePodToMountLocalDependencies( - driverContainerName: String, - initContainerSecret: Secret, - initContainerConfigMap: ConfigMap, - originalPodSpec: PodBuilder): PodBuilder - - def buildInitContainerSecret(jarsSecret: String, filesSecret: String): Secret - - def buildInitContainerConfigMap( - jarsResourceId: String, filesResourceId: String): ConfigMap - - /** - * Convert the Spark jar paths from their locations on the submitter's disk to - * the locations they will be downloaded to on the driver's disk. - */ - def resolveSparkJars(): Seq[String] - - /** - * Convert the Spark file paths from their locations on the submitter's disk to - * the locations they will be downloaded to on the driver's disk. - */ - def resolveSparkFiles(): Seq[String] -} - -/** - * Default implementation of a MountedDependencyManager that is backed by a - * Resource Staging Service. - */ -private[spark] class MountedDependencyManagerImpl( - kubernetesAppId: String, - podLabels: Map[String, String], - podNamespace: String, - stagingServerUri: String, - initContainerImage: String, - jarsDownloadPath: String, - filesDownloadPath: String, - downloadTimeoutMinutes: Long, - sparkJars: Seq[String], - sparkFiles: Seq[String], - stagingServiceSslOptions: SSLOptions, - retrofitClientFactory: RetrofitClientFactory) extends MountedDependencyManager { - private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) - - private def localUriStringsToFiles(uris: Seq[String]): Iterable[File] = { - KubernetesFileUtils.getOnlySubmitterLocalFiles(uris) - .map(Utils.resolveURI) - .map(uri => new File(uri.getPath)) - } - private def localJars: Iterable[File] = localUriStringsToFiles(sparkJars) - private def localFiles: Iterable[File] = localUriStringsToFiles(sparkFiles) - - override def uploadJars(): StagedResourceIdentifier = doUpload(localJars, "uploaded-jars") - override def uploadFiles(): StagedResourceIdentifier = doUpload(localFiles, "uploaded-files") - - private def doUpload(files: Iterable[File], fileNamePrefix: String): StagedResourceIdentifier = { - val filesDir = Utils.createTempDir(namePrefix = fileNamePrefix) - val filesTgz = new File(filesDir, s"$fileNamePrefix.tgz") - Utils.tryWithResource(new FileOutputStream(filesTgz)) { filesOutputStream => - CompressionUtils.writeTarGzipToStream(filesOutputStream, files.map(_.getAbsolutePath)) - } - // TODO provide credentials properly when the staging server monitors the Kubernetes API. - val kubernetesCredentialsString = OBJECT_MAPPER.writer() - .writeValueAsString(KubernetesCredentials(None, None, None, None)) - val labelsAsString = OBJECT_MAPPER.writer().writeValueAsString(podLabels) - - val filesRequestBody = RequestBody.create( - okhttp3.MediaType.parse(MediaType.MULTIPART_FORM_DATA), filesTgz) - - val kubernetesCredentialsBody = RequestBody.create( - okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), kubernetesCredentialsString) - - val namespaceRequestBody = RequestBody.create( - okhttp3.MediaType.parse(MediaType.TEXT_PLAIN), podNamespace) - - val labelsRequestBody = RequestBody.create( - okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), labelsAsString) - - val service = retrofitClientFactory.createRetrofitClient( - stagingServerUri, - classOf[ResourceStagingServiceRetrofit], - stagingServiceSslOptions) - val uploadResponse = service.uploadResources( - labelsRequestBody, namespaceRequestBody, filesRequestBody, kubernetesCredentialsBody) - getTypedResponseResult(uploadResponse) - } - - override def configurePodToMountLocalDependencies( - driverContainerName: String, - initContainerSecret: Secret, - initContainerConfigMap: ConfigMap, - originalPodSpec: PodBuilder): PodBuilder = { - val sharedVolumeMounts = Seq[VolumeMount]( - new VolumeMountBuilder() - .withName(DOWNLOAD_JARS_VOLUME_NAME) - .withMountPath(jarsDownloadPath) - .build(), - new VolumeMountBuilder() - .withName(DOWNLOAD_FILES_VOLUME_NAME) - .withMountPath(filesDownloadPath) - .build()) - - val initContainers = Seq(new ContainerBuilder() - .withName("spark-driver-init") - .withImage(initContainerImage) - .withImagePullPolicy("IfNotPresent") - .addNewVolumeMount() - .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) - .withMountPath(INIT_CONTAINER_PROPERTIES_FILE_MOUNT_PATH) - .endVolumeMount() - .addNewVolumeMount() - .withName(INIT_CONTAINER_SECRETS_VOLUME_NAME) - .withMountPath(INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH) - .endVolumeMount() - .addToVolumeMounts(sharedVolumeMounts: _*) - .addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH) - .build()) - - // Make sure we don't override any user-provided init containers by just appending ours to - // the existing list. - val resolvedInitContainers = originalPodSpec - .editMetadata() - .getAnnotations - .asScala - .get(INIT_CONTAINER_ANNOTATION) - .map { existingInitContainerAnnotation => - val existingInitContainers = OBJECT_MAPPER.readValue( - existingInitContainerAnnotation, classOf[List[Container]]) - existingInitContainers ++ initContainers - }.getOrElse(initContainers) - val resolvedSerializedInitContainers = OBJECT_MAPPER.writeValueAsString(resolvedInitContainers) - originalPodSpec - .editMetadata() - .removeFromAnnotations(INIT_CONTAINER_ANNOTATION) - .addToAnnotations(INIT_CONTAINER_ANNOTATION, resolvedSerializedInitContainers) - .endMetadata() - .editSpec() - .addNewVolume() - .withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME) - .withNewConfigMap() - .withName(initContainerConfigMap.getMetadata.getName) - .addNewItem() - .withKey(INIT_CONTAINER_CONFIG_MAP_KEY) - .withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME) - .endItem() - .endConfigMap() - .endVolume() - .addNewVolume() - .withName(DOWNLOAD_JARS_VOLUME_NAME) - .withEmptyDir(new EmptyDirVolumeSource()) - .endVolume() - .addNewVolume() - .withName(DOWNLOAD_FILES_VOLUME_NAME) - .withEmptyDir(new EmptyDirVolumeSource()) - .endVolume() - .addNewVolume() - .withName(INIT_CONTAINER_SECRETS_VOLUME_NAME) - .withNewSecret() - .withSecretName(initContainerSecret.getMetadata.getName) - .endSecret() - .endVolume() - .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainerName)) - .addToVolumeMounts(sharedVolumeMounts: _*) - .addNewEnv() - .withName(ENV_UPLOADED_JARS_DIR) - .withValue(jarsDownloadPath) - .endEnv() - .endContainer() - .endSpec() - } - - override def buildInitContainerSecret(jarsSecret: String, filesSecret: String): Secret = { - val trustStoreBase64 = stagingServiceSslOptions.trustStore.map { trustStoreFile => - require(trustStoreFile.isFile, "Dependency server trustStore provided at" + - trustStoreFile.getAbsolutePath + " does not exist or is not a file.") - (INIT_CONTAINER_TRUSTSTORE_SECRET_KEY, - BaseEncoding.base64().encode(Files.toByteArray(trustStoreFile))) - }.toMap - val jarsSecretBase64 = BaseEncoding.base64().encode(jarsSecret.getBytes(Charsets.UTF_8)) - val filesSecretBase64 = BaseEncoding.base64().encode(filesSecret.getBytes(Charsets.UTF_8)) - val secretData = Map( - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_KEY -> jarsSecretBase64, - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_KEY -> filesSecretBase64) ++ - trustStoreBase64 - val kubernetesSecret = new SecretBuilder() - .withNewMetadata() - .withName(s"$kubernetesAppId-spark-init") - .endMetadata() - .addToData(secretData.asJava) - .build() - kubernetesSecret - } - - override def buildInitContainerConfigMap( - jarsResourceId: String, filesResourceId: String): ConfigMap = { - val initContainerProperties = new Properties() - initContainerProperties.setProperty(RESOURCE_STAGING_SERVER_URI.key, stagingServerUri) - initContainerProperties.setProperty(DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION.key, jarsDownloadPath) - initContainerProperties.setProperty(DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION.key, filesDownloadPath) - initContainerProperties.setProperty( - INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key, jarsResourceId) - initContainerProperties.setProperty( - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key, INIT_CONTAINER_DOWNLOAD_JARS_SECRET_PATH) - initContainerProperties.setProperty( - INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key, filesResourceId) - initContainerProperties.setProperty( - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key, INIT_CONTAINER_DOWNLOAD_FILES_SECRET_PATH) - initContainerProperties.setProperty(DRIVER_MOUNT_DEPENDENCIES_INIT_TIMEOUT.key, - s"${downloadTimeoutMinutes}m") - stagingServiceSslOptions.trustStore.foreach { _ => - initContainerProperties.setProperty(RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key, - INIT_CONTAINER_TRUSTSTORE_PATH) - } - initContainerProperties.setProperty(RESOURCE_STAGING_SERVER_SSL_ENABLED.key, - stagingServiceSslOptions.enabled.toString) - stagingServiceSslOptions.trustStorePassword.foreach { password => - initContainerProperties.setProperty(RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) - } - stagingServiceSslOptions.trustStoreType.foreach { storeType => - initContainerProperties.setProperty(RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) - } - val propertiesWriter = new StringWriter() - initContainerProperties.store(propertiesWriter, "Init-container properties.") - new ConfigMapBuilder() - .withNewMetadata() - .withName(s"$kubernetesAppId-init-properties") - .endMetadata() - .addToData(INIT_CONTAINER_CONFIG_MAP_KEY, propertiesWriter.toString) - .build() - } - - override def resolveSparkJars(): Seq[String] = resolveLocalFiles(sparkJars, jarsDownloadPath) - - override def resolveSparkFiles(): Seq[String] = resolveLocalFiles(sparkFiles, filesDownloadPath) - - private def resolveLocalFiles( - allFileUriStrings: Seq[String], localDownloadRoot: String): Seq[String] = { - val usedLocalFileNames = mutable.HashSet.empty[String] - val resolvedFiles = mutable.Buffer.empty[String] - for (fileUriString <- allFileUriStrings) { - val fileUri = Utils.resolveURI(fileUriString) - val resolvedFile = Option(fileUri.getScheme).getOrElse("file") match { - case "file" => - // Deduplication logic matches that of CompressionUtils#writeTarGzipToStream - val file = new File(fileUri.getPath) - val extension = Files.getFileExtension(file.getName) - val nameWithoutExtension = Files.getNameWithoutExtension(file.getName) - var resolvedFileName = file.getName - var deduplicationCounter = 1 - while (usedLocalFileNames.contains(resolvedFileName)) { - resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" - deduplicationCounter += 1 - } - s"file://$localDownloadRoot/$resolvedFileName" - case _ => fileUriString - } - resolvedFiles += resolvedFile - } - resolvedFiles - } - - private def getTypedResponseResult[T](call: Call[T]): T = { - val response = call.execute() - if (response.code() < 200 || response.code() >= 300) { - throw new SparkException("Unexpected response from dependency server when uploading" + - s" dependencies: ${response.code()}. Error body: " + - Option(response.errorBody()).map(_.string()).getOrElse("N/A")) - } - response.body() - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerProvider.scala deleted file mode 100644 index 8f09112132b2c..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerProvider.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.kubernetes.submit.v2 - -import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.rest.kubernetes.v2.RetrofitClientFactoryImpl - -private[spark] trait MountedDependencyManagerProvider { - def getMountedDependencyManager( - kubernetesAppId: String, - stagingServerUri: String, - podLabels: Map[String, String], - podNamespace: String, - sparkJars: Seq[String], - sparkFiles: Seq[String]): MountedDependencyManager -} - -private[spark] class MountedDependencyManagerProviderImpl(sparkConf: SparkConf) - extends MountedDependencyManagerProvider { - override def getMountedDependencyManager( - kubernetesAppId: String, - stagingServerUri: String, - podLabels: Map[String, String], - podNamespace: String, - sparkJars: Seq[String], - sparkFiles: Seq[String]): MountedDependencyManager = { - val resourceStagingServerSslOptions = new SparkSecurityManager(sparkConf) - .getSSLOptions("kubernetes.resourceStagingServer") - new MountedDependencyManagerImpl( - kubernetesAppId, - podLabels, - podNamespace, - stagingServerUri, - sparkConf.get(INIT_CONTAINER_DOCKER_IMAGE), - sparkConf.get(DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION), - sparkConf.get(DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION), - sparkConf.get(DRIVER_MOUNT_DEPENDENCIES_INIT_TIMEOUT), - sparkJars, - sparkFiles, - resourceStagingServerSslOptions, - RetrofitClientFactoryImpl) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/PropertiesConfigMapFromScalaMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/PropertiesConfigMapFromScalaMapBuilder.scala new file mode 100644 index 0000000000000..cb9194552d2b6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/PropertiesConfigMapFromScalaMapBuilder.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.kubernetes.submit.v2 + +import java.io.StringWriter +import java.util.Properties + +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder} + +/** + * Creates a config map from a map object, with a single given key + * and writing the map in a {@link java.util.Properties} format. + */ +private[spark] object PropertiesConfigMapFromScalaMapBuilder { + + def buildConfigMap( + configMapName: String, + configMapKey: String, + config: Map[String, String]): ConfigMap = { + val properties = new Properties() + config.foreach { case (key, value) => properties.setProperty(key, value) } + val propertiesWriter = new StringWriter() + properties.store(propertiesWriter, + s"Java properties built from Kubernetes config map with name: $configMapName" + + " and config map key: $configMapKey") + new ConfigMapBuilder() + .withNewMetadata() + .withName(configMapName) + .endMetadata() + .addToData(configMapKey, propertiesWriter.toString) + .build() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SparkInitContainerConfigMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SparkInitContainerConfigMapBuilder.scala new file mode 100644 index 0000000000000..362fbbdf517dc --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SparkInitContainerConfigMapBuilder.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils + +private[spark] trait SparkInitContainerConfigMapBuilder { + /** + * Construct a config map that an init-container should reference for fetching + * remote dependencies. The config map includes the remote jars and files to download, + * as well as details to fetch files from a resource staging server, if applicable. + */ + def build(): ConfigMap +} + +private[spark] class SparkInitContainerConfigMapBuilderImpl( + sparkJars: Seq[String], + sparkFiles: Seq[String], + jarsDownloadPath: String, + filesDownloadPath: String, + configMapName: String, + configMapKey: String, + submittedDependenciesPlugin: Option[SubmittedDependencyInitContainerConfigPlugin]) + extends SparkInitContainerConfigMapBuilder { + + override def build(): ConfigMap = { + 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 submittedDependenciesConfig = submittedDependenciesPlugin.map { plugin => + plugin.configurationsToFetchSubmittedDependencies() + }.toSeq.flatten.toMap + PropertiesConfigMapFromScalaMapBuilder.buildConfigMap( + configMapName, + configMapKey, + baseInitContainerConfig ++ submittedDependenciesConfig) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPlugin.scala new file mode 100644 index 0000000000000..bc9abc4eaba81 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPlugin.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import org.apache.spark.SSLOptions +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait SubmittedDependencyInitContainerConfigPlugin { + /** + * Obtain configuration to fetch submitted dependencies from a resource staging server. + * This includes the resource identifiers for the jar and file bundles, as well as the + * remote location of the resource staging server, and the location of secret files for + * authenticating to the resource staging server. Note that the secret file paths here need to + * line up with the locations the secrets are mounted by + * SubmittedDependencyInitContainerVolumesPlugin; constants provide the consistency and + * convention for these to line up. + */ + def configurationsToFetchSubmittedDependencies(): Map[String, String] +} + +private[spark] class SubmittedDependencyInitContainerConfigPluginImpl( + resourceStagingServerUri: String, + jarsResourceId: String, + filesResourceId: String, + jarsSecretKey: String, + filesSecretKey: String, + trustStoreSecretKey: String, + secretsVolumeMountPath: String, + resourceStagingServiceSslOptions: SSLOptions) + extends SubmittedDependencyInitContainerConfigPlugin { + + override def configurationsToFetchSubmittedDependencies(): Map[String, String] = { + Map[String, String]( + RESOURCE_STAGING_SERVER_URI.key -> resourceStagingServerUri, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> jarsResourceId, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$secretsVolumeMountPath/$jarsSecretKey", + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> filesResourceId, + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$secretsVolumeMountPath/$filesSecretKey", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> + resourceStagingServiceSslOptions.enabled.toString) ++ + resourceStagingServiceSslOptions.trustStore.map { _ => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key, + s"$secretsVolumeMountPath/$trustStoreSecretKey") + }.toMap ++ + resourceStagingServiceSslOptions.trustStorePassword.map { password => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key, password) + }.toMap ++ + resourceStagingServiceSslOptions.trustStoreType.map { storeType => + (RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key, storeType) + }.toMap + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilder.scala new file mode 100644 index 0000000000000..1853b2ecce6d2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilder.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +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.SSLOptions +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] trait SubmittedDependencySecretBuilder { + /** + * Construct a Kubernetes secret bundle that init-containers can use to retrieve an + * application's dependencies. + */ + def build(): Secret +} + +private[spark] class SubmittedDependencySecretBuilderImpl( + secretName: String, + jarsResourceSecret: String, + filesResourceSecret: String, + jarsSecretKey: String, + filesSecretKey: String, + trustStoreSecretKey: String, + resourceStagingServerSslOptions: SSLOptions) + extends SubmittedDependencySecretBuilder { + + override def build(): Secret = { + val trustStoreBase64 = resourceStagingServerSslOptions.trustStore.map { trustStoreFile => + require(trustStoreFile.isFile, "Dependency server trustStore provided at" + + trustStoreFile.getAbsolutePath + " does not exist or is not a file.") + (trustStoreSecretKey, BaseEncoding.base64().encode(Files.toByteArray(trustStoreFile))) + }.toMap + val jarsSecretBase64 = BaseEncoding.base64().encode(jarsResourceSecret.getBytes(Charsets.UTF_8)) + val filesSecretBase64 = BaseEncoding.base64().encode( + filesResourceSecret.getBytes(Charsets.UTF_8)) + val secretData = Map( + jarsSecretKey -> jarsSecretBase64, + filesSecretKey -> filesSecretBase64) ++ + trustStoreBase64 + val kubernetesSecret = new SecretBuilder() + .withNewMetadata() + .withName(secretName) + .endMetadata() + .addToData(secretData.asJava) + .build() + kubernetesSecret + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderImpl.scala new file mode 100644 index 0000000000000..f22759d463cb7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderImpl.scala @@ -0,0 +1,116 @@ +/* + * 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.v2 + +import java.io.{File, FileOutputStream} +import javax.ws.rs.core.MediaType + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import okhttp3.RequestBody +import retrofit2.Call + +import org.apache.spark.{SparkException, SSLOptions} +import org.apache.spark.deploy.kubernetes.CompressionUtils +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.rest.kubernetes.v1.KubernetesCredentials +import org.apache.spark.deploy.rest.kubernetes.v2.{ResourceStagingServiceRetrofit, RetrofitClientFactory} +import org.apache.spark.util.Utils + +private[spark] trait SubmittedDependencyUploader { + /** + * Upload submitter-local jars to the resource staging server. + * @return The resource ID and secret to use to retrieve these jars. + */ + def uploadJars(): SubmittedResourceIdAndSecret + + /** + * Upload submitter-local files to the resource staging server. + * @return The resource ID and secret to use to retrieve these files. + */ + def uploadFiles(): SubmittedResourceIdAndSecret +} + +/** + * Default implementation of a SubmittedDependencyManager that is backed by a + * Resource Staging Service. + */ +private[spark] class SubmittedDependencyUploaderImpl( + kubernetesAppId: String, + podLabels: Map[String, String], + podNamespace: String, + stagingServerUri: String, + sparkJars: Seq[String], + sparkFiles: Seq[String], + stagingServiceSslOptions: SSLOptions, + retrofitClientFactory: RetrofitClientFactory) extends SubmittedDependencyUploader { + private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) + + private def localUriStringsToFiles(uris: Seq[String]): Iterable[File] = { + KubernetesFileUtils.getOnlySubmitterLocalFiles(uris) + .map(Utils.resolveURI) + .map(uri => new File(uri.getPath)) + } + private def localJars: Iterable[File] = localUriStringsToFiles(sparkJars) + private def localFiles: Iterable[File] = localUriStringsToFiles(sparkFiles) + + override def uploadJars(): SubmittedResourceIdAndSecret = doUpload(localJars, "uploaded-jars") + override def uploadFiles(): SubmittedResourceIdAndSecret = doUpload(localFiles, "uploaded-files") + + private def doUpload(files: Iterable[File], fileNamePrefix: String) + : SubmittedResourceIdAndSecret = { + val filesDir = Utils.createTempDir(namePrefix = fileNamePrefix) + val filesTgz = new File(filesDir, s"$fileNamePrefix.tgz") + Utils.tryWithResource(new FileOutputStream(filesTgz)) { filesOutputStream => + CompressionUtils.writeTarGzipToStream(filesOutputStream, files.map(_.getAbsolutePath)) + } + // TODO provide credentials properly when the staging server monitors the Kubernetes API. + val kubernetesCredentialsString = OBJECT_MAPPER.writer() + .writeValueAsString(KubernetesCredentials(None, None, None, None)) + val labelsAsString = OBJECT_MAPPER.writer().writeValueAsString(podLabels) + + val filesRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.MULTIPART_FORM_DATA), filesTgz) + + val kubernetesCredentialsBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), kubernetesCredentialsString) + + val namespaceRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.TEXT_PLAIN), podNamespace) + + val labelsRequestBody = RequestBody.create( + okhttp3.MediaType.parse(MediaType.APPLICATION_JSON), labelsAsString) + + val service = retrofitClientFactory.createRetrofitClient( + stagingServerUri, + classOf[ResourceStagingServiceRetrofit], + stagingServiceSslOptions) + val uploadResponse = service.uploadResources( + labelsRequestBody, namespaceRequestBody, filesRequestBody, kubernetesCredentialsBody) + getTypedResponseResult(uploadResponse) + } + + private def getTypedResponseResult[T](call: Call[T]): T = { + val response = call.execute() + if (response.code() < 200 || response.code() >= 300) { + throw new SparkException("Unexpected response from dependency server when uploading" + + s" dependencies: ${response.code()}. Error body: " + + Option(response.errorBody()).map(_.string()).getOrElse("N/A")) + } + response.body() + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/StagedResourceIdentifier.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedResources.scala similarity index 51% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/StagedResourceIdentifier.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedResources.scala index 65bc9bc17dae9..f4e5e991180ce 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/StagedResourceIdentifier.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedResources.scala @@ -14,6 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes.v2 +package org.apache.spark.deploy.kubernetes.submit.v2 -case class StagedResourceIdentifier(resourceId: String, resourceSecret: String) +case class SubmittedResourceIdAndSecret(resourceId: String, resourceSecret: String) + +case class SubmittedResources( + jarsResourceIdAndSecret: SubmittedResourceIdAndSecret, + filesResourceIdAndSecret: SubmittedResourceIdAndSecret) { + def ids(): SubmittedResourceIds = SubmittedResourceIds( + jarsResourceIdAndSecret.resourceId, filesResourceIdAndSecret.resourceId) + def secrets(): SubmittedResourceSecrets = SubmittedResourceSecrets( + jarsResourceIdAndSecret.resourceSecret, filesResourceIdAndSecret.resourceSecret) +} + +case class SubmittedResourceIds(jarsResourceId: String, filesResourceId: String) + +case class SubmittedResourceSecrets(jarsResourceSecret: String, filesResourceSecret: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesSparkRestServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesSparkRestServer.scala index ca05fe767146b..7847ba2546594 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesSparkRestServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v1/KubernetesSparkRestServer.scala @@ -35,6 +35,7 @@ import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkCo import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.kubernetes.CompressionUtils import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils import org.apache.spark.deploy.rest._ import org.apache.spark.internal.config.OptionalConfigEntry import org.apache.spark.util.{ShutdownHookManager, ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala index 680d305985cc0..67caa176930ea 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala @@ -25,12 +25,15 @@ 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, SparkException} +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.CompressionUtils import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private trait WaitableCallback[T] extends Callback[T] { private val complete = SettableFuture.create[Boolean] @@ -61,55 +64,149 @@ private class DownloadTarGzCallback(downloadDir: File) extends WaitableCallback[ } } +// Extracted for testing so that unit tests don't have to depend on Utils.fetchFile +private[v2] trait FileFetcher { + def fetchFile(uri: String, targetDir: File): Unit +} + +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) + } +} + +/** + * 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) extends Logging { + sparkConf: SparkConf, + retrofitClientFactory: RetrofitClientFactory, + fileFetcher: FileFetcher, + securityManager: SparkSecurityManager) extends Logging { - private val resourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) - .getOrElse(throw new SparkException("No dependency server URI was provided.")) + private implicit val downloadExecutor = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("download-executor")) + private val maybeResourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) - private val downloadJarsResourceIdentifier = sparkConf + private val maybeDownloadJarsResourceIdentifier = sparkConf .get(INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER) - .getOrElse(throw new SparkException("No resource identifier provided for jars.")) private val downloadJarsSecretLocation = new File( sparkConf.get(INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION)) - private val downloadFilesResourceIdentifier = sparkConf + private val maybeDownloadFilesResourceIdentifier = sparkConf .get(INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER) - .getOrElse(throw new SparkException("No resource identifier provided for files.")) private val downloadFilesSecretLocation = new File( sparkConf.get(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION)) - require(downloadJarsSecretLocation.isFile, "Application jars download secret provided" + - s" at ${downloadJarsSecretLocation.getAbsolutePath} does not exist or is not a file.") - require(downloadFilesSecretLocation.isFile, "Application files download secret provided" + - s" at ${downloadFilesSecretLocation.getAbsolutePath} does not exist or is not a file.") - private val jarsDownloadDir = new File(sparkConf.get(DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION)) - require(jarsDownloadDir.isDirectory, "Application jars download directory provided at" + - s" ${jarsDownloadDir.getAbsolutePath} does not exist or is not a directory.") + 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 filesDownloadDir = new File(sparkConf.get(DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION)) - require(filesDownloadDir.isDirectory, "Application files download directory provided at" + - s" ${filesDownloadDir.getAbsolutePath} does not exist or is not a directory.") - private val downloadTimeoutMinutes = sparkConf.get(DRIVER_MOUNT_DEPENDENCIES_INIT_TIMEOUT) + private val downloadTimeoutMinutes = sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT) def run(): Unit = { - val securityManager = new SparkSecurityManager(sparkConf) - val sslOptions = securityManager.getSSLOptions("kubernetes.resourceStagingServer") - val service = retrofitClientFactory.createRetrofitClient( - resourceStagingServerUri, classOf[ResourceStagingServiceRetrofit], sslOptions) - val jarsSecret = Files.toString(downloadJarsSecretLocation, Charsets.UTF_8) - val filesSecret = Files.toString(downloadFilesSecretLocation, Charsets.UTF_8) - val downloadJarsCallback = new DownloadTarGzCallback(jarsDownloadDir) - val downloadFilesCallback = new DownloadTarGzCallback(filesDownloadDir) - service.downloadResources(downloadJarsResourceIdentifier, jarsSecret) - .enqueue(downloadJarsCallback) - service.downloadResources(downloadFilesResourceIdentifier, filesSecret) - .enqueue(downloadFilesCallback) - logInfo("Waiting to download jars...") - downloadJarsCallback.waitForCompletion(downloadTimeoutMinutes, TimeUnit.MINUTES) - logInfo(s"Jars downloaded to ${jarsDownloadDir.getAbsolutePath}") - logInfo("Waiting to download files...") - downloadFilesCallback.waitForCompletion(downloadTimeoutMinutes, TimeUnit.MINUTES) - logInfo(s"Files downloaded to ${filesDownloadDir.getAbsolutePath}") + 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 sslOptions = securityManager.getSSLOptions("kubernetes.resourceStagingServer") + val service = retrofitClientFactory.createRetrofitClient( + resourceStagingServerUri, classOf[ResourceStagingServiceRetrofit], sslOptions) + 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)) + } } } @@ -121,7 +218,13 @@ object KubernetesSparkDependencyDownloadInitContainer extends Logging { } else { new SparkConf(true) } - new KubernetesSparkDependencyDownloadInitContainer(sparkConf, RetrofitClientFactoryImpl).run() + val securityManager = new SparkSecurityManager(sparkConf) + val fileFetcher = new FileFetcherImpl(sparkConf, securityManager) + new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, + RetrofitClientFactoryImpl, + fileFetcher, + securityManager).run() logInfo("Finished downloading application dependencies.") } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingService.scala index 844809dec995c..b7c6c4fb913da 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingService.scala @@ -22,6 +22,7 @@ import javax.ws.rs.core.{MediaType, StreamingOutput} import org.glassfish.jersey.media.multipart.FormDataParam +import org.apache.spark.deploy.kubernetes.submit.v2.SubmittedResourceIdAndSecret import org.apache.spark.deploy.rest.kubernetes.v1.KubernetesCredentials /** @@ -69,7 +70,7 @@ private[spark] trait ResourceStagingService { @FormDataParam("podNamespace") podNamespace: String, @FormDataParam("resources") resources: InputStream, @FormDataParam("kubernetesCredentials") kubernetesCredentials: KubernetesCredentials) - : StagedResourceIdentifier + : SubmittedResourceIdAndSecret /** * Download an application's resources. The resources are provided as a stream, where the stream's diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala index cf6180fbf53d4..3dfa83c85e6dd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala @@ -26,6 +26,7 @@ import com.google.common.io.{BaseEncoding, ByteStreams, Files} import scala.collection.concurrent.TrieMap import org.apache.spark.SparkException +import org.apache.spark.deploy.kubernetes.submit.v2.SubmittedResourceIdAndSecret import org.apache.spark.deploy.rest.kubernetes.v1.KubernetesCredentials import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -41,7 +42,7 @@ private[spark] class ResourceStagingServiceImpl(dependenciesRootDir: File) podLabels: Map[String, String], podNamespace: String, resources: InputStream, - kubernetesCredentials: KubernetesCredentials): StagedResourceIdentifier = { + kubernetesCredentials: KubernetesCredentials): SubmittedResourceIdAndSecret = { val resourceId = UUID.randomUUID().toString val secretBytes = new Array[Byte](1024) SECURE_RANDOM.nextBytes(secretBytes) @@ -65,7 +66,7 @@ private[spark] class ResourceStagingServiceImpl(dependenciesRootDir: File) podNamespace, resourcesTgz, kubernetesCredentials) - StagedResourceIdentifier(resourceId, resourceSecret) + SubmittedResourceIdAndSecret(resourceId, resourceSecret) } catch { case e: Throwable => if (!resourcesDir.delete()) { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceRetrofit.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceRetrofit.scala index b1a3cc0676757..e0079a372f0d9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceRetrofit.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceRetrofit.scala @@ -20,6 +20,8 @@ import okhttp3.{RequestBody, ResponseBody} import retrofit2.Call import retrofit2.http.{Multipart, Path, Streaming} +import org.apache.spark.deploy.kubernetes.submit.v2.SubmittedResourceIdAndSecret + /** * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on * how to use this service, see the aforementioned JAX-RS based interface. @@ -33,7 +35,7 @@ private[spark] trait ResourceStagingServiceRetrofit { @retrofit2.http.Part("podNamespace") podNamespace: RequestBody, @retrofit2.http.Part("resources") resources: RequestBody, @retrofit2.http.Part("kubernetesCredentials") - kubernetesCredentials: RequestBody): Call[StagedResourceIdentifier] + kubernetesCredentials: RequestBody): Call[SubmittedResourceIdAndSecret] @Streaming @retrofit2.http.GET("/api/v0/resources/{resourceId}") 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 index 70098f1f46ac0..e2630b9918b61 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -17,9 +17,12 @@ package org.apache.spark.scheduler.cluster.kubernetes import org.apache.spark.SparkContext +import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -private[spark] class KubernetesClusterManager extends ExternalClusterManager { +private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { override def canCreate(masterURL: String): Boolean = masterURL.startsWith("k8s") @@ -31,7 +34,49 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager { override def createSchedulerBackend(sc: SparkContext, masterURL: String, scheduler: TaskScheduler) : SchedulerBackend = { - new KubernetesClusterSchedulerBackend(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc) + 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(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION), + sparkConf.get(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION), + sparkConf.get(INIT_CONTAINER_MOUNT_TIMEOUT), + configMap, + configMapKey, + executorInitContainerSecretVolumePlugin) + } + 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.") + } + new KubernetesClusterSchedulerBackend( + sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, bootStrap) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { 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 index 669a073b1fab6..0dd875b307a6d 100644 --- 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 @@ -20,17 +20,16 @@ import java.io.Closeable import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.concurrent.{ExecutionContext, Future} - -import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.api.model.{ContainerPortBuilder, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.apache.commons.io.FilenameUtils +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.ConfigurationUtils +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.rpc.{RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -41,7 +40,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, - val sc: SparkContext) + val sc: SparkContext, + executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { import KubernetesClusterSchedulerBackend._ @@ -52,6 +52,9 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_PODS_BY_IPS_LOCK = new Object private val executorPodsByIPs = new mutable.HashMap[String, Pod] // Indexed by executor IP addrs. + private val executorExtraClasspath = conf.get( + org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) private var shufflePodCache: Option[ShufflePodCache] = None private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) @@ -258,13 +261,20 @@ private[spark] class KubernetesClusterSchedulerBackend( val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCores) .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), (ENV_EXECUTOR_CORES, executorCores), (ENV_EXECUTOR_MEMORY, executorMemoryString), (ENV_APPLICATION_ID, applicationId()), - (ENV_EXECUTOR_ID, executorId)) + (ENV_EXECUTOR_ID, executorId), + (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) .map(env => new EnvVarBuilder() .withName(env._1) .withValue(env._2) @@ -317,7 +327,7 @@ private[spark] class KubernetesClusterSchedulerBackend( .endContainer() .endSpec() - val resolvedPodBuilder = shuffleServiceConfig + val withMaybeShuffleConfigPodBuilder = shuffleServiceConfig .map { config => config.shuffleDirs.foldLeft(basePodBuilder) { (builder, dir) => builder @@ -337,9 +347,14 @@ private[spark] class KubernetesClusterSchedulerBackend( .endSpec() } }.getOrElse(basePodBuilder) + val resolvedExecutorPod = executorInitContainerBootstrap.map { bootstrap => + bootstrap.bootstrapInitContainerAndVolumes( + "executor", + withMaybeShuffleConfigPodBuilder) + }.getOrElse(withMaybeShuffleConfigPodBuilder) try { - (executorId, kubernetesClient.pods().create(resolvedPodBuilder.build())) + (executorId, kubernetesClient.pods.create(resolvedExecutorPod.build())) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) 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..6db7d3ff2da53 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala @@ -0,0 +1,164 @@ +/* + * 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 com.fasterxml.jackson.databind.ObjectMapper +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.constants._ + +class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { + private val OBJECT_MAPPER = new ObjectMapper() + private val INIT_CONTAINER_IMAGE = "spark-init:latest" + 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 ADDED_SUBMITTED_DEPENDENCY_ENV = "ADDED_SUBMITTED_DEPENDENCY" + private val ADDED_SUBMITTED_DEPENDENCY_ANNOTATION = "added-submitted-dependencies" + private val MAIN_CONTAINER_NAME = "spark-main" + private val TRUE = "true" + + private val submittedDependencyPlugin = new InitContainerResourceStagingServerSecretPlugin { + override def addResourceStagingServerSecretVolumeToPod(basePod: PodBuilder) + : PodBuilder = { + basePod.editMetadata() + .addToAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION, TRUE) + .endMetadata() + } + + override def mountResourceStagingServerSecretIntoInitContainer(container: ContainerBuilder) + : ContainerBuilder = { + container + .addNewEnv() + .withName(ADDED_SUBMITTED_DEPENDENCY_ENV) + .withValue(TRUE) + .endEnv() + } + } + + test("Running without submitted dependencies adds init-container with volume mounts.") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala + assert(podAnnotations.contains(INIT_CONTAINER_ANNOTATION)) + val initContainers = OBJECT_MAPPER.readValue( + podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainer = initContainers.head + val initContainerVolumeMounts = initContainer.getVolumeMounts.asScala.map { + mount => (mount.getName, mount.getMountPath) + }.toMap + val expectedInitContainerVolumeMounts = Map( + INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_DIR, + INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) + assert(initContainerVolumeMounts === expectedInitContainerVolumeMounts) + assert(initContainer.getName === "spark-init") + assert(initContainer.getImage === INIT_CONTAINER_IMAGE) + assert(initContainer.getImagePullPolicy === "IfNotPresent") + assert(initContainer.getArgs.asScala === List(INIT_CONTAINER_PROPERTIES_FILE_PATH)) + } + + test("Running without submitted dependencies adds volume mounts to main container.") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val containers = bootstrappedPod.getSpec.getContainers.asScala + val mainContainer = containers.find(_.getName === MAIN_CONTAINER_NAME) + assert(mainContainer.isDefined) + val volumeMounts = mainContainer.map(_.getVolumeMounts.asScala).toSeq.flatten.map { + mount => (mount.getName, mount.getMountPath) + }.toMap + val expectedVolumeMounts = Map( + INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, + INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) + assert(volumeMounts === expectedVolumeMounts) + } + + test("Running without submitted dependencies adds volumes to the pod") { + val bootstrappedPod = bootstrapPodWithoutSubmittedDependencies() + val podVolumes = bootstrappedPod.getSpec.getVolumes.asScala + assert(podVolumes.size === 3) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_PROPERTIES_FILE_VOLUME && + Option(volume.getConfigMap).map { configMap => + configMap.getItems.asScala.map { + keyToPath => (keyToPath.getKey, keyToPath.getPath) + }.toMap + }.contains(Map(INIT_CONTAINER_CONFIG_MAP_KEY -> INIT_CONTAINER_PROPERTIES_FILE_NAME)) + }) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME && volume.getEmptyDir != null + }) + assert(podVolumes.exists { volume => + volume.getName == INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME && volume.getEmptyDir != null + }) + } + + test("Running with submitted dependencies modifies the init container with the plugin.") { + val bootstrappedPod = bootstrapPodWithSubmittedDependencies() + val podAnnotations = bootstrappedPod.getMetadata.getAnnotations.asScala + assert(podAnnotations(ADDED_SUBMITTED_DEPENDENCY_ANNOTATION) === TRUE) + val initContainers = OBJECT_MAPPER.readValue( + podAnnotations(INIT_CONTAINER_ANNOTATION), classOf[Array[Container]]) + assert(initContainers.length === 1) + val initContainer = initContainers.head + assert(initContainer.getEnv.asScala.exists { + env => env.getName === ADDED_SUBMITTED_DEPENDENCY_ENV && env.getValue === TRUE + }) + } + + private def bootstrapPodWithoutSubmittedDependencies(): Pod = { + val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( + INIT_CONTAINER_IMAGE, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOWNLOAD_TIMEOUT_MINUTES, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + None) + bootstrapUnderTest.bootstrapInitContainerAndVolumes( + MAIN_CONTAINER_NAME, basePod()).build() + } + + private def bootstrapPodWithSubmittedDependencies(): Pod = { + val bootstrapUnderTest = new SparkPodInitContainerBootstrapImpl( + INIT_CONTAINER_IMAGE, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOWNLOAD_TIMEOUT_MINUTES, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + Some(submittedDependencyPlugin)) + bootstrapUnderTest.bootstrapInitContainerAndVolumes( + MAIN_CONTAINER_NAME, basePod()).build() + } + + private def basePod(): PodBuilder = { + new PodBuilder() + .withNewMetadata() + .withName("spark-pod") + .endMetadata() + .withNewSpec() + .addNewContainer() + .withName(MAIN_CONTAINER_NAME) + .endContainer() + .endSpec() + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala new file mode 100644 index 0000000000000..473d369c8eca3 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SubmittedDependencyInitContainerVolumesPluginSuite.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.constants._ + +class SubmittedDependencyInitContainerVolumesPluginSuite extends SparkFunSuite { + + private val SECRET_NAME = "secret" + private val SECRET_MOUNT_PATH = "/mnt/secrets" + private val plugin = new InitContainerResourceStagingServerSecretPluginImpl( + SECRET_NAME, SECRET_MOUNT_PATH) + + test("The init container should have the secret volume mount.") { + val baseInitContainer = new ContainerBuilder().withName("container") + val configuredInitContainer = plugin.mountResourceStagingServerSecretIntoInitContainer( + baseInitContainer).build() + val volumeMounts = configuredInitContainer.getVolumeMounts.asScala + assert(volumeMounts.size === 1) + assert(volumeMounts.exists { volumeMount => + volumeMount.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && + volumeMount.getMountPath === SECRET_MOUNT_PATH + }) + } + + test("The pod should have the secret volume.") { + val basePod = new PodBuilder() + .withNewMetadata().withName("pod").endMetadata() + .withNewSpec() + .addNewContainer() + .withName("container") + .endContainer() + .endSpec() + val configuredPod = plugin.addResourceStagingServerSecretVolumeToPod(basePod).build() + val volumes = configuredPod.getSpec.getVolumes.asScala + assert(volumes.size === 1) + assert(volumes.exists { volume => + volume.getName === INIT_CONTAINER_SECRET_VOLUME_NAME && + Option(volume.getSecret).map(_.getSecretName).contains(SECRET_NAME) + }) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala index e6536fbaa6941..4dc1e2e44980a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala @@ -18,311 +18,331 @@ package org.apache.spark.deploy.kubernetes.submit.v2 import java.io.File -import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, Container, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} import org.hamcrest.{BaseMatcher, Description} -import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq, startsWith} -import org.mockito.Mockito.when +import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, anyVararg, argThat, eq => mockitoEq} +import org.mockito.Mockito.{times, 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 scala.reflect.ClassTag +import scala.collection.mutable import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.SparkPodInitContainerBootstrap import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.v2.StagedResourceIdentifier -import org.apache.spark.util.Utils class ClientV2Suite extends SparkFunSuite with BeforeAndAfter { - private val MAIN_CLASS = "org.apache.spark.test.Main" - private val APP_ARGS = Array[String]("arg1", "arg2") - private val MAIN_APP_RESOURCE = "local:///app/jars/spark-main.jar" - private val APP_NAME = "spark-test-app" - private val STAGING_SERVER_URI = "http://localhost:9000" + private val JARS_RESOURCE = SubmittedResourceIdAndSecret("jarsId", "jarsSecret") + private val FILES_RESOURCE = SubmittedResourceIdAndSecret("filesId", "filesSecret") + private val SUBMITTED_RESOURCES = SubmittedResources(JARS_RESOURCE, FILES_RESOURCE) + private val BOOTSTRAPPED_POD_ANNOTATION = "bootstrapped" + private val TRUE = "true" + private val APP_NAME = "spark-test" + private val APP_ID = "spark-app-id" + private val CUSTOM_LABEL_KEY = "customLabel" + private val CUSTOM_LABEL_VALUE = "customLabelValue" + private val ALL_EXPECTED_LABELS = Map( + CUSTOM_LABEL_KEY -> CUSTOM_LABEL_VALUE, + SPARK_APP_ID_LABEL -> APP_ID, + SPARK_APP_NAME_LABEL -> APP_NAME) + private val CUSTOM_ANNOTATION_KEY = "customAnnotation" + private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" + private val SECRET_NAME = "secret" + private val SECRET_KEY = "secret-key" + private val SECRET_DATA = "secret-data" + private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val APP_ARGS = Array("3", "20") private val SPARK_JARS = Seq( - "local:///app/jars/spark-helper.jar", "file:///var/data/spark-local-helper.jar") + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///app/jars/jar2.jar") private val RESOLVED_SPARK_JARS = Seq( - "local:///app/jars/spark-helper.jar", - "file:///var/data/spark-downloaded/spark-local-helper.jar") + "hdfs://localhost:9000/app/jars/jar1.jar", "file:///var/data/spark-jars/jar2.jar") + private val RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS = Seq( + "/var/data/spark-jars/jar1.jar", "/var/data/spark-jars/jar2.jar") private val SPARK_FILES = Seq( - "local:///app/files/spark-file.txt", "file:///var/data/spark-local-file.txt") + "hdfs://localhost:9000/app/files/file1.txt", "file:///app/files/file2.txt") private val RESOLVED_SPARK_FILES = Seq( - "local:///app/files/spark-file.txt", "file:///var/data/spark-downloaded/spark-local-file.txt") - private val DRIVER_EXTRA_CLASSPATH = "/app/jars/extra-jar1.jar:/app/jars/extra-jars2.jar" - private val DRIVER_DOCKER_IMAGE_VALUE = "spark-driver:latest" - private val DRIVER_MEMORY_OVERHEARD_MB = 128L - private val DRIVER_MEMORY_MB = 512L - private val NAMESPACE = "namespace" - private val DOWNLOAD_JARS_RESOURCE_IDENTIFIER = StagedResourceIdentifier("jarsId", "jarsSecret") - private val DOWNLOAD_FILES_RESOURCE_IDENTIFIER = StagedResourceIdentifier( - "filesId", "filesSecret") - private val MOUNTED_FILES_ANNOTATION_KEY = "mountedFiles" - - private var sparkConf: SparkConf = _ - private var submissionKubernetesClientProvider: SubmissionKubernetesClientProvider = _ - private var submissionKubernetesClient: KubernetesClient = _ - private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] - private type RESOURCES = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ - HasMetadata, Boolean] - private var podOperations: PODS = _ - private var resourceListOperations: RESOURCES = _ - private var mountedDependencyManagerProvider: MountedDependencyManagerProvider = _ - private var mountedDependencyManager: MountedDependencyManager = _ - private var captureCreatedPodAnswer: SelfArgumentCapturingAnswer[Pod] = _ - private var captureCreatedResourcesAnswer: AllArgumentsCapturingAnswer[HasMetadata, RESOURCES] = _ + "hdfs://localhost:9000/app/files/file1.txt", "file:///var/data/spark-files/file2.txt") + private val INIT_CONTAINER_SECRET = new SecretBuilder() + .withNewMetadata() + .withName(SECRET_NAME) + .endMetadata() + .addToData(SECRET_KEY, SECRET_DATA) + .build() + private val CONFIG_MAP_NAME = "config-map" + private val CONFIG_MAP_KEY = "config-map-key" + private val CONFIG_MAP_DATA = "config-map-data" + private val CUSTOM_JAVA_OPTION_KEY = "myappoption" + private val CUSTOM_JAVA_OPTION_VALUE = "myappoptionvalue" + private val DRIVER_JAVA_OPTIONS = s"-D$CUSTOM_JAVA_OPTION_KEY=$CUSTOM_JAVA_OPTION_VALUE" + private val DRIVER_EXTRA_CLASSPATH = "/var/data/spark-app-custom/custom-jar.jar" + private val INIT_CONTAINER_CONFIG_MAP = new ConfigMapBuilder() + .withNewMetadata() + .withName(CONFIG_MAP_NAME) + .endMetadata() + .addToData(CONFIG_MAP_KEY, CONFIG_MAP_DATA) + .build() + private val CUSTOM_DRIVER_IMAGE = "spark-custom-driver:latest" + private val DRIVER_MEMORY_MB = 512 + private val DRIVER_MEMORY_OVERHEAD_MB = 128 + private val SPARK_CONF = new SparkConf(true) + .set(DRIVER_DOCKER_IMAGE, CUSTOM_DRIVER_IMAGE) + .set(org.apache.spark.internal.config.DRIVER_MEMORY, DRIVER_MEMORY_MB.toLong) + .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, DRIVER_MEMORY_OVERHEAD_MB.toLong) + .set(KUBERNETES_DRIVER_LABELS, s"$CUSTOM_LABEL_KEY=$CUSTOM_LABEL_VALUE") + .set(KUBERNETES_DRIVER_ANNOTATIONS, s"$CUSTOM_ANNOTATION_KEY=$CUSTOM_ANNOTATION_VALUE") + .set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, DRIVER_EXTRA_CLASSPATH) + .set(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, DRIVER_JAVA_OPTIONS) + private val EXECUTOR_INIT_CONF_KEY = "executor-init-conf" + private val SPARK_CONF_WITH_EXECUTOR_INIT_CONF = SPARK_CONF.clone() + .set(EXECUTOR_INIT_CONF_KEY, TRUE) + private val DRIVER_POD_UID = "driver-pod-uid" + private val DRIVER_POD_KIND = "pod" + private val DRIVER_POD_API_VERSION = "v1" + @Mock + private var initContainerConfigMapBuilder: SparkInitContainerConfigMapBuilder = _ + @Mock + private var containerLocalizedFilesResolver: ContainerLocalizedFilesResolver = _ + @Mock + private var executorInitContainerConfiguration: ExecutorInitContainerConfiguration = _ + @Mock + private var submittedDependencyUploader: SubmittedDependencyUploader = _ + @Mock + private var submittedDependenciesSecretBuilder: SubmittedDependencySecretBuilder = _ + @Mock + private var initContainerBootstrap: SparkPodInitContainerBootstrap = _ + @Mock + private var initContainerComponentsProvider: DriverInitContainerComponentsProvider = _ + @Mock + private var kubernetesClientProvider: SubmissionKubernetesClientProvider = _ + @Mock + private var kubernetesClient: KubernetesClient = _ + @Mock + private var podOps: MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] = _ + private type ResourceListOps = NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable[ + HasMetadata, java.lang.Boolean] + @Mock + private var resourceListOps: ResourceListOps = _ before { - sparkConf = new SparkConf(true) - .set("spark.app.name", APP_NAME) - .set("spark.master", "k8s://https://localhost:443") - .set(DRIVER_DOCKER_IMAGE, DRIVER_DOCKER_IMAGE_VALUE) - .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, DRIVER_MEMORY_OVERHEARD_MB) - .set(KUBERNETES_NAMESPACE, NAMESPACE) - .set(org.apache.spark.internal.config.DRIVER_MEMORY, DRIVER_MEMORY_MB) - submissionKubernetesClientProvider = mock[SubmissionKubernetesClientProvider] - submissionKubernetesClient = mock[KubernetesClient] - podOperations = mock[PODS] - resourceListOperations = mock[RESOURCES] - mountedDependencyManagerProvider = mock[MountedDependencyManagerProvider] - mountedDependencyManager = mock[MountedDependencyManager] - when(submissionKubernetesClientProvider.get).thenReturn(submissionKubernetesClient) - when(submissionKubernetesClient.pods()).thenReturn(podOperations) - captureCreatedPodAnswer = new SelfArgumentCapturingAnswer[Pod] - captureCreatedResourcesAnswer = new AllArgumentsCapturingAnswer[HasMetadata, RESOURCES]( - resourceListOperations) - when(podOperations.create(any())).thenAnswer(captureCreatedPodAnswer) - when(submissionKubernetesClient.resourceList(anyVararg[HasMetadata])) - .thenAnswer(captureCreatedResourcesAnswer) - } - - // Tests w/o local dependencies, or behave independently to that configuration. - test("Simple properties and environment set on the driver pod.") { - sparkConf.set(org.apache.spark.internal.config.DRIVER_CLASS_PATH, DRIVER_EXTRA_CLASSPATH) - val createdDriverPod = createAndGetDriverPod() - val maybeDriverContainer = getDriverContainer(createdDriverPod) - maybeDriverContainer.foreach { driverContainer => - assert(driverContainer.getName === DRIVER_CONTAINER_NAME) - assert(driverContainer.getImage === DRIVER_DOCKER_IMAGE_VALUE) - assert(driverContainer.getImagePullPolicy === "IfNotPresent") - val envs = driverContainer.getEnv.asScala.map { env => - (env.getName, env.getValue) - }.toMap - assert(envs(ENV_DRIVER_MEMORY) === s"${DRIVER_MEMORY_MB + DRIVER_MEMORY_OVERHEARD_MB}m") - assert(envs(ENV_DRIVER_MAIN_CLASS) === MAIN_CLASS) - assert(envs(ENV_DRIVER_ARGS) === APP_ARGS.mkString(" ")) - assert(envs(ENV_SUBMIT_EXTRA_CLASSPATH) === DRIVER_EXTRA_CLASSPATH) - } - } - - test("Created pod should apply custom annotations and labels") { - sparkConf.set(KUBERNETES_DRIVER_LABELS, - "label1=label1value,label2=label2value") - sparkConf.set(KUBERNETES_DRIVER_ANNOTATIONS, - "annotation1=annotation1value,annotation2=annotation2value") - val createdDriverPod = createAndGetDriverPod() - val labels = createdDriverPod.getMetadata.getLabels.asScala - assert(labels.size === 4) - // App ID is non-deterministic, but just check if it's set and is prefixed with the app name - val appIdLabel = labels(SPARK_APP_ID_LABEL) - assert(appIdLabel != null && appIdLabel.startsWith(APP_NAME) && appIdLabel != APP_NAME) - val appNameLabel = labels(SPARK_APP_NAME_LABEL) - assert(appNameLabel != null && appNameLabel == APP_NAME) - assert(labels("label1") === "label1value") - assert(labels("label2") === "label2value") - val annotations = createdDriverPod.getMetadata.getAnnotations.asScala - val expectedAnnotations = Map( - "annotation1" -> "annotation1value", "annotation2" -> "annotation2value") - assert(annotations === expectedAnnotations) - } - - test("Driver JVM Options should be set in the environment.") { - sparkConf.set(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS, "-Dopt1=opt1value") - sparkConf.set("spark.logConf", "true") - val createdDriverPod = createAndGetDriverPod() - val maybeDriverContainer = getDriverContainer(createdDriverPod) - maybeDriverContainer.foreach { driverContainer => - val maybeJvmOptionsEnv = driverContainer.getEnv - .asScala - .find(_.getName == ENV_DRIVER_JAVA_OPTS) - assert(maybeJvmOptionsEnv.isDefined) - maybeJvmOptionsEnv.foreach { jvmOptionsEnv => - val jvmOptions = jvmOptionsEnv.getValue.split(" ") - jvmOptions.foreach { opt => assert(opt.startsWith("-D")) } - val optionKeyValues = jvmOptions.map { option => - val withoutDashDPrefix = option.stripPrefix("-D") - val split = withoutDashDPrefix.split('=') - assert(split.length == 2) - (split(0), split(1)) - }.toMap - assert(optionKeyValues("opt1") === "opt1value") - assert(optionKeyValues.contains("spark.app.id")) - assert(optionKeyValues("spark.jars") === MAIN_APP_RESOURCE) - assert(optionKeyValues(KUBERNETES_DRIVER_POD_NAME.key).startsWith(APP_NAME)) - assert(optionKeyValues("spark.app.name") === APP_NAME) - assert(optionKeyValues("spark.logConf") === "true") + MockitoAnnotations.initMocks(this) + when(initContainerComponentsProvider.provideInitContainerBootstrap()) + .thenReturn(initContainerBootstrap) + when(submittedDependencyUploader.uploadJars()).thenReturn(JARS_RESOURCE) + when(submittedDependencyUploader.uploadFiles()).thenReturn(FILES_RESOURCE) + when(initContainerBootstrap + .bootstrapInitContainerAndVolumes(mockitoEq(DRIVER_CONTAINER_NAME), any())) + .thenAnswer(new Answer[PodBuilder] { + override def answer(invocationOnMock: InvocationOnMock): PodBuilder = { + invocationOnMock.getArgumentAt(1, classOf[PodBuilder]).editMetadata() + .addToAnnotations(BOOTSTRAPPED_POD_ANNOTATION, TRUE) + .endMetadata() + } + }) + when(initContainerComponentsProvider.provideContainerLocalizedFilesResolver()) + .thenReturn(containerLocalizedFilesResolver) + when(initContainerComponentsProvider.provideExecutorInitContainerConfiguration()) + .thenReturn(executorInitContainerConfiguration) + when(submittedDependenciesSecretBuilder.build()) + .thenReturn(INIT_CONTAINER_SECRET) + when(initContainerConfigMapBuilder.build()) + .thenReturn(INIT_CONTAINER_CONFIG_MAP) + when(kubernetesClientProvider.get).thenReturn(kubernetesClient) + when(kubernetesClient.pods()).thenReturn(podOps) + when(podOps.create(any())).thenAnswer(new Answer[Pod] { + override def answer(invocation: InvocationOnMock): Pod = { + new PodBuilder(invocation.getArgumentAt(0, classOf[Pod])) + .editMetadata() + .withUid(DRIVER_POD_UID) + .endMetadata() + .withKind(DRIVER_POD_KIND) + .withApiVersion(DRIVER_POD_API_VERSION) + .build() } - } + }) + when(containerLocalizedFilesResolver.resolveSubmittedAndRemoteSparkJars()) + .thenReturn(RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS) + when(containerLocalizedFilesResolver.resolveSubmittedSparkJars()) + .thenReturn(RESOLVED_SPARK_JARS) + when(containerLocalizedFilesResolver.resolveSubmittedSparkFiles()) + .thenReturn(RESOLVED_SPARK_FILES) + when(executorInitContainerConfiguration.configureSparkConfForExecutorInitContainer(SPARK_CONF)) + .thenReturn(SPARK_CONF_WITH_EXECUTOR_INIT_CONF) + when(kubernetesClient.resourceList(anyVararg[HasMetadata]())).thenReturn(resourceListOps) } - // Tests with local dependencies with the mounted dependency manager. - test("Uploading local dependencies should create Kubernetes secrets and config map") { - val initContainerConfigMap = getInitContainerConfigMap() - val initContainerSecret = getInitContainerSecret() - runWithMountedDependencies(initContainerConfigMap, initContainerSecret) - val driverPod = captureCreatedPodAnswer.capturedArgument - assert(captureCreatedResourcesAnswer.capturedArguments != null) - assert(captureCreatedResourcesAnswer.capturedArguments.size === 2) - assert(captureCreatedResourcesAnswer.capturedArguments.toSet === - Set(initContainerSecret, initContainerConfigMap)) - captureCreatedResourcesAnswer.capturedArguments.foreach { resource => - val driverPodOwnerReferences = resource.getMetadata.getOwnerReferences - assert(driverPodOwnerReferences.size === 1) - val driverPodOwnerReference = driverPodOwnerReferences.asScala.head - assert(driverPodOwnerReference.getName === driverPod.getMetadata.getName) - assert(driverPodOwnerReference.getApiVersion === driverPod.getApiVersion) - assert(driverPodOwnerReference.getUid === driverPod.getMetadata.getUid) - assert(driverPodOwnerReference.getKind === driverPod.getKind) - assert(driverPodOwnerReference.getController) - } + test("Run with dependency uploader") { + when(initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) + .thenReturn(Some(submittedDependencyUploader)) + when(initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets()))) + .thenReturn(Some(submittedDependenciesSecretBuilder)) + when(initContainerComponentsProvider + .provideInitContainerConfigMapBuilder(Some(SUBMITTED_RESOURCES.ids()))) + .thenReturn(initContainerConfigMapBuilder) + runAndVerifyDriverPodHasCorrectProperties() + val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) + val createdResources = resourceListArgumentCaptor.getAllValues.asScala + assert(createdResources.size === 2) + verifyCreatedResourcesHaveOwnerReferences(createdResources) + assert(createdResources.exists { + case secret: Secret => + val expectedSecretData = Map(SECRET_KEY -> SECRET_DATA) + secret.getMetadata.getName == SECRET_NAME && secret.getData.asScala == expectedSecretData + case _ => false + }) + verifyConfigMapWasCreated(createdResources) + verify(submittedDependencyUploader).uploadJars() + verify(submittedDependencyUploader).uploadFiles() + verify(initContainerComponentsProvider) + .provideInitContainerConfigMapBuilder(Some(SUBMITTED_RESOURCES.ids())) + verify(initContainerComponentsProvider) + .provideSubmittedDependenciesSecretBuilder(Some(SUBMITTED_RESOURCES.secrets())) } - test("Uploading local resources should set classpath environment variables") { - val initContainerConfigMap = getInitContainerConfigMap() - val initContainerSecret = getInitContainerSecret() - runWithMountedDependencies(initContainerConfigMap, initContainerSecret) - val driverPod = captureCreatedPodAnswer.capturedArgument - val maybeDriverContainer = getDriverContainer(driverPod) - maybeDriverContainer.foreach { driverContainer => - val envs = driverContainer.getEnv - .asScala - .map { env => (env.getName, env.getValue) } - .toMap - val classPathEntries = envs(ENV_MOUNTED_CLASSPATH).split(File.pathSeparator).toSet - val expectedClassPathEntries = RESOLVED_SPARK_JARS - .map(Utils.resolveURI) - .map(_.getPath) - .toSet - assert(classPathEntries === expectedClassPathEntries) - } + test("Run without dependency uploader") { + when(initContainerComponentsProvider + .provideInitContainerSubmittedDependencyUploader(ALL_EXPECTED_LABELS)) + .thenReturn(None) + when(initContainerComponentsProvider + .provideSubmittedDependenciesSecretBuilder(None)) + .thenReturn(None) + when(initContainerComponentsProvider + .provideInitContainerConfigMapBuilder(None)) + .thenReturn(initContainerConfigMapBuilder) + runAndVerifyDriverPodHasCorrectProperties() + val resourceListArgumentCaptor = ArgumentCaptor.forClass(classOf[HasMetadata]) + verify(kubernetesClient).resourceList(resourceListArgumentCaptor.capture()) + val createdResources = resourceListArgumentCaptor.getAllValues.asScala + assert(createdResources.size === 1) + verifyCreatedResourcesHaveOwnerReferences(createdResources) + verifyConfigMapWasCreated(createdResources) + verify(submittedDependencyUploader, times(0)).uploadJars() + verify(submittedDependencyUploader, times(0)).uploadFiles() + verify(initContainerComponentsProvider) + .provideInitContainerConfigMapBuilder(None) + verify(initContainerComponentsProvider) + .provideSubmittedDependenciesSecretBuilder(None) } - private def getInitContainerSecret(): Secret = { - new SecretBuilder() - .withNewMetadata().withName(s"$APP_NAME-init-container-secret").endMetadata() - .addToData( - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_KEY, DOWNLOAD_JARS_RESOURCE_IDENTIFIER.resourceSecret) - .addToData(INIT_CONTAINER_DOWNLOAD_FILES_SECRET_KEY, - DOWNLOAD_FILES_RESOURCE_IDENTIFIER.resourceSecret) - .build() + private def verifyCreatedResourcesHaveOwnerReferences( + createdResources: mutable.Buffer[HasMetadata]): Unit = { + assert(createdResources.forall { resource => + val owners = resource.getMetadata.getOwnerReferences.asScala + owners.size === 1 && + owners.head.getController && + owners.head.getKind == DRIVER_POD_KIND && + owners.head.getUid == DRIVER_POD_UID && + owners.head.getName == APP_ID && + owners.head.getApiVersion == DRIVER_POD_API_VERSION + }) } - private def getInitContainerConfigMap(): ConfigMap = { - new ConfigMapBuilder() - .withNewMetadata().withName(s"$APP_NAME-init-container-conf").endMetadata() - .addToData("key", "configuration") - .build() + private def verifyConfigMapWasCreated(createdResources: mutable.Buffer[HasMetadata]): Unit = { + assert(createdResources.exists { + case configMap: ConfigMap => + val expectedConfigMapData = Map(CONFIG_MAP_KEY -> CONFIG_MAP_DATA) + configMap.getMetadata.getName == CONFIG_MAP_NAME && + configMap.getData.asScala == expectedConfigMapData + case _ => false + }) } - private def runWithMountedDependencies( - initContainerConfigMap: ConfigMap, initContainerSecret: Secret): Unit = { - sparkConf.set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) - .setJars(SPARK_JARS) - .set("spark.files", SPARK_FILES.mkString(",")) - val labelsMatcher = new BaseMatcher[Map[String, String]] { - override def matches(maybeLabels: scala.Any) = { - maybeLabels match { - case labels: Map[String, String] => - labels(SPARK_APP_ID_LABEL).startsWith(APP_NAME) && - labels(SPARK_APP_NAME_LABEL) == APP_NAME - case _ => false + private def runAndVerifyDriverPodHasCorrectProperties(): Unit = { + new Client( + APP_NAME, + APP_ID, + MAIN_CLASS, + SPARK_CONF, + APP_ARGS, + SPARK_JARS, + SPARK_FILES, + kubernetesClientProvider, + initContainerComponentsProvider).run() + val podMatcher = new BaseMatcher[Pod] { + override def matches(o: scala.Any): Boolean = { + o match { + case p: Pod => + Option(p) + .filter(_.getMetadata.getName == APP_ID) + .filter(podHasCorrectAnnotations) + .filter(_.getMetadata.getLabels.asScala == ALL_EXPECTED_LABELS) + .filter(containerHasCorrectBasicContainerConfiguration) + .filter(containerHasCorrectBasicEnvs) + .filter(containerHasCorrectMountedClasspath) + .exists(containerHasCorrectJvmOptions) + case _ => + false } } - override def describeTo(description: Description) = { - description.appendText("Checks if the labels contain the app ID and app name.") - } + override def describeTo(description: Description): Unit = {} } - when(mountedDependencyManagerProvider.getMountedDependencyManager( - startsWith(APP_NAME), - mockitoEq(STAGING_SERVER_URI), - argThat(labelsMatcher), - mockitoEq(NAMESPACE), - mockitoEq(SPARK_JARS ++ Seq(MAIN_APP_RESOURCE)), - mockitoEq(SPARK_FILES))).thenReturn(mountedDependencyManager) - when(mountedDependencyManager.uploadJars()).thenReturn(DOWNLOAD_JARS_RESOURCE_IDENTIFIER) - when(mountedDependencyManager.uploadFiles()).thenReturn(DOWNLOAD_FILES_RESOURCE_IDENTIFIER) - when(mountedDependencyManager.buildInitContainerSecret( - DOWNLOAD_JARS_RESOURCE_IDENTIFIER.resourceSecret, - DOWNLOAD_FILES_RESOURCE_IDENTIFIER.resourceSecret)) - .thenReturn(initContainerSecret) - when(mountedDependencyManager.buildInitContainerConfigMap( - DOWNLOAD_JARS_RESOURCE_IDENTIFIER.resourceId, DOWNLOAD_FILES_RESOURCE_IDENTIFIER.resourceId)) - .thenReturn(initContainerConfigMap) - when(mountedDependencyManager.resolveSparkJars()).thenReturn(RESOLVED_SPARK_JARS) - when(mountedDependencyManager.resolveSparkFiles()).thenReturn(RESOLVED_SPARK_FILES) - when(mountedDependencyManager.configurePodToMountLocalDependencies( - mockitoEq(DRIVER_CONTAINER_NAME), - mockitoEq(initContainerSecret), - mockitoEq(initContainerConfigMap), - any())).thenAnswer(new Answer[PodBuilder] { - override def answer(invocationOnMock: InvocationOnMock): PodBuilder = { - val basePod = invocationOnMock.getArgumentAt(3, classOf[PodBuilder]) - basePod.editMetadata().addToAnnotations(MOUNTED_FILES_ANNOTATION_KEY, "true").endMetadata() - } - }) - val clientUnderTest = createClient() - clientUnderTest.run() + verify(podOps).create(argThat(podMatcher)) } - private def getDriverContainer(driverPod: Pod): Option[Container] = { - val maybeDriverContainer = driverPod.getSpec - .getContainers - .asScala - .find(_.getName == DRIVER_CONTAINER_NAME) - assert(maybeDriverContainer.isDefined) - maybeDriverContainer + private def containerHasCorrectJvmOptions(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + envs.toMap.get(ENV_DRIVER_JAVA_OPTS).exists { javaOptions => + val splitOptions = javaOptions.split(" ") + val expectedOptions = SPARK_CONF.getAll + .filterNot(_._1 == org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS.key) + .toMap ++ + Map( + "spark.app.id" -> APP_ID, + KUBERNETES_DRIVER_POD_NAME.key -> APP_ID, + EXECUTOR_INIT_CONF_KEY -> TRUE, + CUSTOM_JAVA_OPTION_KEY -> CUSTOM_JAVA_OPTION_VALUE, + "spark.jars" -> RESOLVED_SPARK_JARS.mkString(","), + "spark.files" -> RESOLVED_SPARK_FILES.mkString(",")) + splitOptions.forall(_.startsWith("-D")) && + splitOptions.map { option => + val withoutPrefix = option.substring(2) + (withoutPrefix.split("=", 2)(0), withoutPrefix.split("=", 2)(1)) + }.toMap == expectedOptions + } } - private def createAndGetDriverPod(): Pod = { - val clientUnderTest = createClient() - clientUnderTest.run() - val createdDriverPod = captureCreatedPodAnswer.capturedArgument - assert(createdDriverPod != null) - createdDriverPod + private def containerHasCorrectMountedClasspath(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + envs.toMap.get(ENV_MOUNTED_CLASSPATH).exists { classpath => + val mountedClasspathEntities = classpath.split(File.pathSeparator) + mountedClasspathEntities.toSet == RESOLVED_SPARK_REMOTE_AND_LOCAL_JARS.toSet + } } - private def createClient(): Client = { - new Client( - MAIN_CLASS, - sparkConf, - APP_ARGS, - MAIN_APP_RESOURCE, - submissionKubernetesClientProvider, - mountedDependencyManagerProvider) + private def containerHasCorrectBasicEnvs(pod: Pod): Boolean = { + val driverContainer = pod.getSpec.getContainers.asScala.head + val envs = driverContainer.getEnv.asScala.map(env => (env.getName, env.getValue)) + val expectedBasicEnvs = Map( + ENV_SUBMIT_EXTRA_CLASSPATH -> DRIVER_EXTRA_CLASSPATH, + ENV_DRIVER_MEMORY -> s"${DRIVER_MEMORY_MB + DRIVER_MEMORY_OVERHEAD_MB}m", + ENV_DRIVER_MAIN_CLASS -> MAIN_CLASS, + ENV_DRIVER_ARGS -> APP_ARGS.mkString(" ")) + expectedBasicEnvs.toSet.subsetOf(envs.toSet) } - private class SelfArgumentCapturingAnswer[T: ClassTag] extends Answer[T] { - var capturedArgument: T = _ - - override def answer(invocationOnMock: InvocationOnMock): T = { - val argumentClass = implicitly[ClassTag[T]].runtimeClass.asInstanceOf[Class[T]] - val argument = invocationOnMock.getArgumentAt(0, argumentClass) - this.capturedArgument = argument - argument - } + private def containerHasCorrectBasicContainerConfiguration(pod: Pod): Boolean = { + val containers = pod.getSpec.getContainers.asScala + containers.size == 1 && + containers.head.getName == DRIVER_CONTAINER_NAME && + containers.head.getImage == CUSTOM_DRIVER_IMAGE && + containers.head.getImagePullPolicy == "IfNotPresent" } - private class AllArgumentsCapturingAnswer[I, T](returnValue: T) extends Answer[T] { - var capturedArguments: Seq[I] = _ - - override def answer(invocationOnMock: InvocationOnMock): T = { - capturedArguments = invocationOnMock.getArguments.map(_.asInstanceOf[I]).toSeq - returnValue - } + private def podHasCorrectAnnotations(pod: Pod): Boolean = { + val expectedAnnotations = Map( + CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, + BOOTSTRAPPED_POD_ANNOTATION -> TRUE) + pod.getMetadata.getAnnotations.asScala == expectedAnnotations } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolverSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolverSuite.scala new file mode 100644 index 0000000000000..6804f0010b6a5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerLocalizedFilesResolverSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import org.apache.spark.SparkFunSuite + +class ContainerLocalizedFilesResolverSuite extends SparkFunSuite { + private val SPARK_JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + "file:///app/jars/jar2.jar", + "local:///app/jars/jar3.jar", + "http://app/jars/jar4.jar") + private val SPARK_FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + "file:///app/files/file2.txt", + "local:///app/files/file3.txt", + "http://app/files/file4.txt") + private val JARS_DOWNLOAD_PATH = "/var/data/spark-jars" + private val FILES_DOWNLOAD_PATH = "/var/data/spark-files" + private val localizedFilesResolver = new ContainerLocalizedFilesResolverImpl( + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH) + + test("Submitted and remote Spark jars should resolve non-local uris to download path.") { + val resolvedJars = localizedFilesResolver.resolveSubmittedAndRemoteSparkJars() + val expectedResolvedJars = Seq( + s"$JARS_DOWNLOAD_PATH/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "/app/jars/jar3.jar", + s"$JARS_DOWNLOAD_PATH/jar4.jar") + assert(resolvedJars === expectedResolvedJars) + } + + test("Submitted Spark jars should resolve to the download path.") { + val resolvedJars = localizedFilesResolver.resolveSubmittedSparkJars() + val expectedResolvedJars = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + s"$JARS_DOWNLOAD_PATH/jar2.jar", + "local:///app/jars/jar3.jar", + "http://app/jars/jar4.jar") + assert(resolvedJars === expectedResolvedJars) + } + + test("Submitted Spark files should resolve to the download path.") { + val resolvedFiles = localizedFilesResolver.resolveSubmittedSparkFiles() + val expectedResolvedFiles = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + s"$FILES_DOWNLOAD_PATH/file2.txt", + "local:///app/files/file3.txt", + "http://app/files/file4.txt") + assert(resolvedFiles === expectedResolvedFiles) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfigurationSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfigurationSuite.scala new file mode 100644 index 0000000000000..62bfd127d17e2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ExecutorInitContainerConfigurationSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.v2 + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ + +class ExecutorInitContainerConfigurationSuite extends SparkFunSuite { + + private val SECRET_NAME = "init-container-secret" + private val SECRET_MOUNT_DIR = "/mnt/secrets/spark" + private val CONFIG_MAP_NAME = "spark-config-map" + private val CONFIG_MAP_KEY = "spark-config-map-key" + + test("Not passing a secret name should not set the secret value.") { + val baseSparkConf = new SparkConf(false) + val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( + None, + SECRET_MOUNT_DIR, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val resolvedSparkConf = configurationUnderTest + .configureSparkConfForExecutorInitContainer(baseSparkConf) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP).contains(CONFIG_MAP_NAME)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_CONFIG_MAP_KEY).contains(CONFIG_MAP_KEY)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) + .contains(SECRET_MOUNT_DIR)) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).isEmpty) + } + + test("Passing a secret name should set the secret value.") { + val baseSparkConf = new SparkConf(false) + val configurationUnderTest = new ExecutorInitContainerConfigurationImpl( + Some(SECRET_NAME), + SECRET_MOUNT_DIR, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY) + val resolvedSparkConf = configurationUnderTest + .configureSparkConfForExecutorInitContainer(baseSparkConf) + assert(resolvedSparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET).contains(SECRET_NAME)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerSuite.scala deleted file mode 100644 index 321fe1b3fd889..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerSuite.scala +++ /dev/null @@ -1,323 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.kubernetes.submit.v2 - -import java.io.{ByteArrayOutputStream, File, StringReader} -import java.util.{Properties, 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 io.fabric8.kubernetes.api.model.{ConfigMapBuilder, Container, Pod, PodBuilder, SecretBuilder} -import okhttp3.RequestBody -import okio.Okio -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer -import org.mockito.Matchers.any -import org.mockito.Mockito -import org.scalatest.BeforeAndAfter -import org.scalatest.mock.MockitoSugar._ -import retrofit2.{Call, Response} -import scala.collection.JavaConverters._ - -import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.rest.kubernetes.v2.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourceIdentifier} -import org.apache.spark.util.Utils - -private[spark] class MountedDependencyManagerSuite extends SparkFunSuite with BeforeAndAfter { - import MountedDependencyManagerSuite.createTempFile - - private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) - 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 INIT_CONTAINER_IMAGE = "spark-driver-init:latest" - private val JARS_DOWNLOAD_PATH = DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION.defaultValue.get - private val FILES_DOWNLOAD_PATH = DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION.defaultValue.get - private val DOWNLOAD_TIMEOUT_MINUTES = 5 - 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 JARS_RESOURCE_ID = "jarsId" - private val JARS_SECRET = "jarsSecret" - private val FILES_RESOURCE_ID = "filesId" - private val FILES_SECRET = "filesSecret" - private var retrofitClientFactory: RetrofitClientFactory = _ - private var retrofitClient: ResourceStagingServiceRetrofit = _ - - private var dependencyManagerUnderTest: MountedDependencyManager = _ - - before { - retrofitClientFactory = mock[RetrofitClientFactory] - retrofitClient = mock[ResourceStagingServiceRetrofit] - Mockito.when( - retrofitClientFactory.createRetrofitClient( - STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) - .thenReturn(retrofitClient) - dependencyManagerUnderTest = new MountedDependencyManagerImpl( - APP_ID, - LABELS, - NAMESPACE, - STAGING_SERVER_URI, - INIT_CONTAINER_IMAGE, - JARS_DOWNLOAD_PATH, - FILES_DOWNLOAD_PATH, - DOWNLOAD_TIMEOUT_MINUTES, - JARS, - FILES, - STAGING_SERVER_SSL_OPTIONS, - retrofitClientFactory) - } - - test("Uploading jars should contact the staging server with the appropriate parameters") { - val capturingArgumentsAnswer = new UploadDependenciesArgumentsCapturingAnswer( - StagedResourceIdentifier("resourceId", "resourceSecret")) - Mockito.when(retrofitClient.uploadResources(any(), any(), any(), any())) - .thenAnswer(capturingArgumentsAnswer) - dependencyManagerUnderTest.uploadJars() - testUploadSendsCorrectFiles(LOCAL_JARS, capturingArgumentsAnswer) - } - - test("Uploading files should contact the staging server with the appropriate parameters") { - val capturingArgumentsAnswer = new UploadDependenciesArgumentsCapturingAnswer( - StagedResourceIdentifier("resourceId", "resourceSecret")) - Mockito.when(retrofitClient.uploadResources(any(), any(), any(), any())) - .thenAnswer(capturingArgumentsAnswer) - dependencyManagerUnderTest.uploadFiles() - testUploadSendsCorrectFiles(LOCAL_FILES, capturingArgumentsAnswer) - } - - test("Init container secret should contain jars, files, and trustStore") { - val jarsSecretBase64 = BaseEncoding.base64().encode(JARS_SECRET.getBytes(Charsets.UTF_8)) - val filesSecretBase64 = BaseEncoding.base64().encode(FILES_SECRET.getBytes(Charsets.UTF_8)) - val trustStoreBase64 = BaseEncoding.base64().encode(Files.toByteArray(TRUSTSTORE_FILE)) - val secret = dependencyManagerUnderTest.buildInitContainerSecret("jarsSecret", "filesSecret") - assert(secret.getMetadata.getName === s"$APP_ID-spark-init") - val expectedSecrets = Map( - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_KEY -> jarsSecretBase64, - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_KEY -> filesSecretBase64, - INIT_CONTAINER_TRUSTSTORE_SECRET_KEY -> trustStoreBase64) - assert(secret.getData.asScala === expectedSecrets) - } - - test("Init container config map should contain parameters for downloading from staging server") { - val configMap = dependencyManagerUnderTest.buildInitContainerConfigMap( - JARS_RESOURCE_ID, FILES_RESOURCE_ID) - assert(configMap.getMetadata.getName === s"$APP_ID-init-properties") - val propertiesRawString = configMap.getData.get(INIT_CONTAINER_CONFIG_MAP_KEY) - assert(propertiesRawString != null) - val propertiesReader = new StringReader(propertiesRawString) - val properties = new Properties() - properties.load(propertiesReader) - val propertiesMap = properties.stringPropertyNames().asScala.map { prop => - (prop, properties.getProperty(prop)) - }.toMap - val expectedProperties = Map[String, String]( - RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, - DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION.key -> JARS_DOWNLOAD_PATH, - DRIVER_LOCAL_FILES_DOWNLOAD_LOCATION.key -> FILES_DOWNLOAD_PATH, - INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> - INIT_CONTAINER_DOWNLOAD_JARS_SECRET_PATH, - INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> - INIT_CONTAINER_DOWNLOAD_FILES_SECRET_PATH, - DRIVER_MOUNT_DEPENDENCIES_INIT_TIMEOUT.key -> s"${DOWNLOAD_TIMEOUT_MINUTES}m", - RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> INIT_CONTAINER_TRUSTSTORE_PATH, - RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "true", - RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASSWORD, - RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE) - assert(propertiesMap === expectedProperties) - } - - test("Resolving jars should map local paths to their mounted counterparts") { - val resolvedJars = dependencyManagerUnderTest.resolveSparkJars() - val expectedResolvedJars = Seq( - "hdfs://localhost:9000/jars/jar1.jar", - s"file://$JARS_DOWNLOAD_PATH/${new File(JARS(1)).getName}", - s"file://$JARS_DOWNLOAD_PATH/${new File(JARS(2)).getName}") - assert(resolvedJars === expectedResolvedJars) - } - - test("Resolving files should map local paths to their mounted counterparts") { - val resolvedFiles = dependencyManagerUnderTest.resolveSparkFiles() - val expectedResolvedFiles = Seq( - "hdfs://localhost:9000/files/file1.txt", - s"file://$FILES_DOWNLOAD_PATH/${new File(FILES(1)).getName}") - assert(resolvedFiles === expectedResolvedFiles) - } - - test("Downloading init container should be added to pod") { - val driverPod = configureDriverPod() - val podAnnotations = driverPod.getMetadata.getAnnotations - assert(podAnnotations.size === 1) - val initContainerRawAnnotation = podAnnotations.get(INIT_CONTAINER_ANNOTATION) - val initContainers = OBJECT_MAPPER.readValue( - initContainerRawAnnotation, classOf[Array[Container]]) - assert(initContainers.size === 1) - val initContainer = initContainers.head - assert(initContainer.getName === "spark-driver-init") - assert(initContainer.getImage === INIT_CONTAINER_IMAGE) - assert(initContainer.getImagePullPolicy === "IfNotPresent") - val volumeMounts = initContainer.getVolumeMounts - .asScala - .map(mount => (mount.getName, mount.getMountPath)) - .toMap - val expectedVolumeMounts = Map[String, String]( - DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, - DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH, - INIT_CONTAINER_PROPERTIES_FILE_VOLUME -> INIT_CONTAINER_PROPERTIES_FILE_MOUNT_PATH, - INIT_CONTAINER_SECRETS_VOLUME_NAME -> INIT_CONTAINER_SECRETS_VOLUME_MOUNT_PATH) - assert(volumeMounts === expectedVolumeMounts) - } - - test("Driver pod should have added volumes and volume mounts for file downloads") { - val driverPod = configureDriverPod() - val volumes = driverPod.getSpec.getVolumes.asScala.map(volume => (volume.getName, volume)).toMap - val initContainerPropertiesVolume = volumes(INIT_CONTAINER_PROPERTIES_FILE_VOLUME).getConfigMap - assert(initContainerPropertiesVolume != null) - assert(initContainerPropertiesVolume.getName === "config") - assert(initContainerPropertiesVolume.getItems.asScala.exists { keyToPath => - keyToPath.getKey == INIT_CONTAINER_CONFIG_MAP_KEY && - keyToPath.getPath == INIT_CONTAINER_PROPERTIES_FILE_NAME - }) - val jarsVolume = volumes(DOWNLOAD_JARS_VOLUME_NAME) - assert(jarsVolume.getEmptyDir != null) - val filesVolume = volumes(DOWNLOAD_FILES_VOLUME_NAME) - assert(filesVolume.getEmptyDir != null) - val initContainerSecretVolume = volumes(INIT_CONTAINER_SECRETS_VOLUME_NAME) - assert(initContainerSecretVolume.getSecret != null) - assert(initContainerSecretVolume.getSecret.getSecretName === "secret") - val driverContainer = driverPod.getSpec - .getContainers - .asScala - .find(_.getName == "driver-container").get - val driverContainerVolumeMounts = driverContainer.getVolumeMounts - .asScala - .map(mount => (mount.getName, mount.getMountPath)) - .toMap - val expectedVolumeMountNamesAndPaths = Map[String, String]( - DOWNLOAD_JARS_VOLUME_NAME -> JARS_DOWNLOAD_PATH, - DOWNLOAD_FILES_VOLUME_NAME -> FILES_DOWNLOAD_PATH) - assert(driverContainerVolumeMounts === expectedVolumeMountNamesAndPaths) - val envs = driverContainer.getEnv - assert(envs.size() === 1) - assert(envs.asScala.head.getName === ENV_UPLOADED_JARS_DIR) - assert(envs.asScala.head.getValue === JARS_DOWNLOAD_PATH) - } - - private def configureDriverPod(): Pod = { - val initContainerSecret = new SecretBuilder() - .withNewMetadata().withName("secret").endMetadata() - .addToData("datakey", "datavalue") - .build() - val initContainerConfigMap = new ConfigMapBuilder() - .withNewMetadata().withName("config").endMetadata() - .addToData("datakey", "datavalue") - .build() - val basePod = new PodBuilder() - .withNewMetadata() - .withName("driver-pod") - .endMetadata() - .withNewSpec() - .addNewContainer() - .withName("driver-container") - .withImage("spark-driver:latest") - .endContainer() - .endSpec() - val adjustedPod = dependencyManagerUnderTest.configurePodToMountLocalDependencies( - "driver-container", - initContainerSecret, - initContainerConfigMap, - basePod).build() - adjustedPod - } - - private def testUploadSendsCorrectFiles( - expectedFiles: Seq[String], - capturingArgumentsAnswer: UploadDependenciesArgumentsCapturingAnswer) = { - val requestLabelsBytes = requestBodyBytes(capturingArgumentsAnswer.podLabelsArg) - val requestLabelsString = new String(requestLabelsBytes, Charsets.UTF_8) - val requestLabelsMap = OBJECT_MAPPER.readValue( - requestLabelsString, classOf[Map[String, String]]) - assert(requestLabelsMap === LABELS) - val requestNamespaceBytes = requestBodyBytes(capturingArgumentsAnswer.podNamespaceArg) - val requestNamespaceString = new String(requestNamespaceBytes, Charsets.UTF_8) - assert(requestNamespaceString === NAMESPACE) - val localJarsTarStream = new ByteArrayOutputStream() - CompressionUtils.writeTarGzipToStream(localJarsTarStream, expectedFiles) - val requestResourceBytes = requestBodyBytes(capturingArgumentsAnswer.podResourcesArg) - assert(requestResourceBytes.sameElements(localJarsTarStream.toByteArray)) - } - - private def requestBodyBytes(requestBody: RequestBody): Array[Byte] = { - Utils.tryWithResource(new ByteArrayOutputStream()) { outputStream => - Utils.tryWithResource(Okio.sink(outputStream)) { sink => - Utils.tryWithResource(Okio.buffer(sink)) { bufferedSink => - requestBody.writeTo(bufferedSink) - } - } - outputStream.toByteArray - } - } -} - -private class UploadDependenciesArgumentsCapturingAnswer(returnValue: StagedResourceIdentifier) - extends Answer[Call[StagedResourceIdentifier]] { - - var podLabelsArg: RequestBody = _ - var podNamespaceArg: RequestBody = _ - var podResourcesArg: RequestBody = _ - var kubernetesCredentialsArg: RequestBody = _ - - override def answer(invocationOnMock: InvocationOnMock): Call[StagedResourceIdentifier] = { - podLabelsArg = invocationOnMock.getArgumentAt(0, classOf[RequestBody]) - podNamespaceArg = invocationOnMock.getArgumentAt(1, classOf[RequestBody]) - podResourcesArg = invocationOnMock.getArgumentAt(2, classOf[RequestBody]) - kubernetesCredentialsArg = invocationOnMock.getArgumentAt(3, classOf[RequestBody]) - val responseCall = mock[Call[StagedResourceIdentifier]] - Mockito.when(responseCall.execute()).thenReturn(Response.success(returnValue)) - responseCall - } -} - -private object MountedDependencyManagerSuite { - 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/v2/SparkInitContainerConfigMapBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SparkInitContainerConfigMapBuilderSuite.scala new file mode 100644 index 0000000000000..7c6fbf5ce6da2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SparkInitContainerConfigMapBuilderSuite.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.deploy.kubernetes.submit.v2 + +import java.io.StringReader +import java.util.Properties + +import com.google.common.collect.Maps +import org.mockito.Mockito.{verify, when} +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.kubernetes.config._ + +class SparkInitContainerConfigMapBuilderSuite extends SparkFunSuite with BeforeAndAfter { + + private val JARS = Seq( + "hdfs://localhost:9000/app/jars/jar1.jar", + "file:///app/jars/jar2.jar", + "http://localhost:9000/app/jars/jar3.jar", + "local:///app/jars/jar4.jar") + private val FILES = Seq( + "hdfs://localhost:9000/app/files/file1.txt", + "file:///app/files/file2.txt", + "http://localhost:9000/app/files/file3.txt", + "local:///app/files/file4.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" + + test("Config map without submitted dependencies sets remote download configurations") { + val configMap = new SparkInitContainerConfigMapBuilderImpl( + JARS, + FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + None).build() + assert(configMap.getMetadata.getName === CONFIG_MAP_NAME) + val maybeConfigValue = configMap.getData.asScala.get(CONFIG_MAP_KEY) + assert(maybeConfigValue.isDefined) + maybeConfigValue.foreach { configValue => + val propertiesStringReader = new StringReader(configValue) + val properties = new Properties() + properties.load(propertiesStringReader) + val propertiesMap = Maps.fromProperties(properties).asScala + val remoteJarsString = propertiesMap.get(INIT_CONTAINER_REMOTE_JARS.key) + assert(remoteJarsString.isDefined) + val remoteJars = remoteJarsString.map(_.split(",")).toSet.flatten + assert(remoteJars === + Set("hdfs://localhost:9000/app/jars/jar1.jar", "http://localhost:9000/app/jars/jar3.jar")) + val remoteFilesString = propertiesMap.get(INIT_CONTAINER_REMOTE_FILES.key) + assert(remoteFilesString.isDefined) + val remoteFiles = remoteFilesString.map(_.split(",")).toSet.flatten + assert(remoteFiles === + Set("hdfs://localhost:9000/app/files/file1.txt", + "http://localhost:9000/app/files/file3.txt")) + assert(propertiesMap(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION.key) === JARS_DOWNLOAD_PATH) + assert(propertiesMap(INIT_CONTAINER_FILES_DOWNLOAD_LOCATION.key) === FILES_DOWNLOAD_PATH) + } + } + + test("Config map with submitted dependencies adds configurations from plugin") { + val submittedDependenciesPlugin = mock[SubmittedDependencyInitContainerConfigPlugin] + when(submittedDependenciesPlugin.configurationsToFetchSubmittedDependencies()) + .thenReturn(Map("customConf" -> "customConfValue")) + val configMap = new SparkInitContainerConfigMapBuilderImpl( + JARS, + FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + CONFIG_MAP_NAME, + CONFIG_MAP_KEY, + Some(submittedDependenciesPlugin)).build() + val configValue = configMap.getData.asScala(CONFIG_MAP_KEY) + val propertiesStringReader = new StringReader(configValue) + val properties = new Properties() + properties.load(propertiesStringReader) + val propertiesMap = Maps.fromProperties(properties).asScala + assert(propertiesMap("customConf") === "customConfValue") + verify(submittedDependenciesPlugin).configurationsToFetchSubmittedDependencies() + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPluginSuite.scala new file mode 100644 index 0000000000000..11a671085c201 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyInitContainerConfigPluginSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.v2 + +import java.io.File + +import org.apache.spark.{SparkFunSuite, SSLOptions} +import org.apache.spark.deploy.kubernetes.config._ + +class SubmittedDependencyInitContainerConfigPluginSuite extends SparkFunSuite { + private val STAGING_SERVER_URI = "http://localhost:9000" + private val JARS_RESOURCE_ID = "jars-id" + private val FILES_RESOURCE_ID = "files-id" + private val JARS_SECRET_KEY = "jars" + private val FILES_SECRET_KEY = "files" + private val TRUSTSTORE_SECRET_KEY = "trustStore" + private val SECRETS_VOLUME_MOUNT_PATH = "/var/data/" + private val TRUSTSTORE_PASSWORD = "trustStore" + private val TRUSTSTORE_FILE = "/mnt/secrets/trustStore.jks" + private val TRUSTSTORE_TYPE = "jks" + private val RESOURCE_STAGING_SERVICE_SSL_OPTIONS = SSLOptions( + enabled = true, + trustStore = Some(new File(TRUSTSTORE_FILE)), + trustStorePassword = Some(TRUSTSTORE_PASSWORD), + trustStoreType = Some(TRUSTSTORE_TYPE)) + + test("Plugin should provide configuration for fetching uploaded dependencies") { + val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( + STAGING_SERVER_URI, + JARS_RESOURCE_ID, + FILES_RESOURCE_ID, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + SECRETS_VOLUME_MOUNT_PATH, + SSLOptions()) + val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() + val expectedConfigurations = Map( + RESOURCE_STAGING_SERVER_URI.key -> STAGING_SERVER_URI, + INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER.key -> JARS_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER.key -> FILES_RESOURCE_ID, + INIT_CONTAINER_DOWNLOAD_JARS_SECRET_LOCATION.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$JARS_SECRET_KEY", + INIT_CONTAINER_DOWNLOAD_FILES_SECRET_LOCATION.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$FILES_SECRET_KEY", + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "false") + assert(addedConfigurations === expectedConfigurations) + } + + test("Plugin should set up SSL with the appropriate trustStore if it's provided.") { + val configPluginUnderTest = new SubmittedDependencyInitContainerConfigPluginImpl( + STAGING_SERVER_URI, + JARS_RESOURCE_ID, + FILES_RESOURCE_ID, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + SECRETS_VOLUME_MOUNT_PATH, + RESOURCE_STAGING_SERVICE_SSL_OPTIONS) + val addedConfigurations = configPluginUnderTest.configurationsToFetchSubmittedDependencies() + val expectedSslConfigurations = Map( + RESOURCE_STAGING_SERVER_SSL_ENABLED.key -> "true", + RESOURCE_STAGING_SERVER_TRUSTSTORE_FILE.key -> + s"$SECRETS_VOLUME_MOUNT_PATH/$TRUSTSTORE_SECRET_KEY", + RESOURCE_STAGING_SERVER_TRUSTSTORE_PASSWORD.key -> TRUSTSTORE_PASSWORD, + RESOURCE_STAGING_SERVER_TRUSTSTORE_TYPE.key -> TRUSTSTORE_TYPE) + assert(expectedSslConfigurations.toSet.subsetOf(addedConfigurations.toSet)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilderSuite.scala new file mode 100644 index 0000000000000..189d87e27a28a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencySecretBuilderSuite.scala @@ -0,0 +1,83 @@ +/* + * 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.v2 + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.{BaseEncoding, Files} +import io.fabric8.kubernetes.api.model.Secret +import scala.collection.JavaConverters._ +import scala.collection.Map + +import org.apache.spark.{SparkFunSuite, SSLOptions} +import org.apache.spark.util.Utils + +class SubmittedDependencySecretBuilderSuite extends SparkFunSuite { + + private val SECRET_NAME = "submitted-dependency-secret" + private val JARS_SECRET = "jars-secret" + private val FILES_SECRET = "files-secret" + private val JARS_SECRET_KEY = "jars-secret-key" + private val FILES_SECRET_KEY = "files-secret-key" + private val TRUSTSTORE_SECRET_KEY = "truststore-secret-key" + private val TRUSTSTORE_STRING_CONTENTS = "trustStore-contents" + + test("Building the secret without a trustStore") { + val builder = new SubmittedDependencySecretBuilderImpl( + SECRET_NAME, + JARS_SECRET, + FILES_SECRET, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + SSLOptions()) + val secret = builder.build() + assert(secret.getMetadata.getName === SECRET_NAME) + val secretDecodedData = decodeSecretData(secret) + val expectedSecretData = Map(JARS_SECRET_KEY -> JARS_SECRET, FILES_SECRET_KEY -> FILES_SECRET) + assert(secretDecodedData === expectedSecretData) + } + + private def decodeSecretData(secret: Secret): Map[String, String] = { + val secretData = secret.getData.asScala + secretData.mapValues(encoded => + new String(BaseEncoding.base64().decode(encoded), Charsets.UTF_8)) + } + + test("Building the secret with a trustStore") { + val tempTrustStoreDir = Utils.createTempDir(namePrefix = "temp-truststores") + try { + val trustStoreFile = new File(tempTrustStoreDir, "trustStore.jks") + Files.write(TRUSTSTORE_STRING_CONTENTS, trustStoreFile, Charsets.UTF_8) + val builder = new SubmittedDependencySecretBuilderImpl( + SECRET_NAME, + JARS_SECRET, + FILES_SECRET, + JARS_SECRET_KEY, + FILES_SECRET_KEY, + TRUSTSTORE_SECRET_KEY, + SSLOptions(trustStore = Some(trustStoreFile))) + val secret = builder.build() + val secretDecodedData = decodeSecretData(secret) + assert(secretDecodedData(TRUSTSTORE_SECRET_KEY) === TRUSTSTORE_STRING_CONTENTS) + } finally { + tempTrustStoreDir.delete() + } + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderSuite.scala new file mode 100644 index 0000000000000..7b259aa2c3a0c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmittedDependencyUploaderSuite.scala @@ -0,0 +1,177 @@ +/* + * 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.v2 + +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.Files +import okhttp3.RequestBody +import okio.Okio +import org.mockito.Matchers.any +import org.mockito.Mockito +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +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.v2.{ResourceStagingServiceRetrofit, RetrofitClientFactory} +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 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 var retrofitClientFactory: RetrofitClientFactory = _ + private var retrofitClient: ResourceStagingServiceRetrofit = _ + + private var dependencyUploaderUnderTest: SubmittedDependencyUploader = _ + + before { + retrofitClientFactory = mock[RetrofitClientFactory] + retrofitClient = mock[ResourceStagingServiceRetrofit] + Mockito.when( + retrofitClientFactory.createRetrofitClient( + STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) + .thenReturn(retrofitClient) + dependencyUploaderUnderTest = new SubmittedDependencyUploaderImpl( + APP_ID, + LABELS, + NAMESPACE, + STAGING_SERVER_URI, + JARS, + FILES, + STAGING_SERVER_SSL_OPTIONS, + retrofitClientFactory) + } + + test("Uploading jars should contact the staging server with the appropriate parameters") { + val capturingArgumentsAnswer = new UploadDependenciesArgumentsCapturingAnswer( + SubmittedResourceIdAndSecret("resourceId", "resourceSecret")) + Mockito.when(retrofitClient.uploadResources(any(), any(), any(), any())) + .thenAnswer(capturingArgumentsAnswer) + dependencyUploaderUnderTest.uploadJars() + testUploadSendsCorrectFiles(LOCAL_JARS, capturingArgumentsAnswer) + } + + test("Uploading files should contact the staging server with the appropriate parameters") { + val capturingArgumentsAnswer = new UploadDependenciesArgumentsCapturingAnswer( + SubmittedResourceIdAndSecret("resourceId", "resourceSecret")) + Mockito.when(retrofitClient.uploadResources(any(), any(), any(), any())) + .thenAnswer(capturingArgumentsAnswer) + dependencyUploaderUnderTest.uploadFiles() + testUploadSendsCorrectFiles(LOCAL_FILES, capturingArgumentsAnswer) + } + + private def testUploadSendsCorrectFiles( + expectedFiles: Seq[String], + capturingArgumentsAnswer: UploadDependenciesArgumentsCapturingAnswer) = { + val requestLabelsBytes = requestBodyBytes(capturingArgumentsAnswer.podLabelsArg) + val requestLabelsString = new String(requestLabelsBytes, Charsets.UTF_8) + val requestLabelsMap = OBJECT_MAPPER.readValue( + requestLabelsString, classOf[Map[String, String]]) + assert(requestLabelsMap === LABELS) + val requestNamespaceBytes = requestBodyBytes(capturingArgumentsAnswer.podNamespaceArg) + val requestNamespaceString = new String(requestNamespaceBytes, Charsets.UTF_8) + assert(requestNamespaceString === NAMESPACE) + + val unpackedFilesDir = Utils.createTempDir(namePrefix = "test-unpacked-files") + val compressedBytesInput = new ByteArrayInputStream( + requestBodyBytes(capturingArgumentsAnswer.podResourcesArg)) + 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 class UploadDependenciesArgumentsCapturingAnswer(returnValue: SubmittedResourceIdAndSecret) + extends Answer[Call[SubmittedResourceIdAndSecret]] { + + var podLabelsArg: RequestBody = _ + var podNamespaceArg: RequestBody = _ + var podResourcesArg: RequestBody = _ + var kubernetesCredentialsArg: RequestBody = _ + + override def answer(invocationOnMock: InvocationOnMock): Call[SubmittedResourceIdAndSecret] = { + podLabelsArg = invocationOnMock.getArgumentAt(0, classOf[RequestBody]) + podNamespaceArg = invocationOnMock.getArgumentAt(1, classOf[RequestBody]) + podResourcesArg = invocationOnMock.getArgumentAt(2, classOf[RequestBody]) + kubernetesCredentialsArg = invocationOnMock.getArgumentAt(3, classOf[RequestBody]) + val responseCall = mock[Call[SubmittedResourceIdAndSecret]] + Mockito.when(responseCall.execute()).thenReturn(Response.success(returnValue)) + responseCall + } +} + +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/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala index 77eb7f2b9f49c..6ab37185b8d07 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -24,6 +24,7 @@ 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 @@ -31,7 +32,7 @@ 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.{SecurityManager => SparkSecurityManager, SparkConf, SparkFunSuite, SSLOptions} import org.apache.spark.deploy.kubernetes.CompressionUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.util.Utils @@ -55,7 +56,6 @@ class KubernetesSparkDependencyDownloadInitContainerSuite private val JARS_RESOURCE_ID = "jarsId" private val FILES_RESOURCE_ID = "filesId" - private var sparkConf: SparkConf = _ private var downloadJarsDir: File = _ private var downloadFilesDir: File = _ private var downloadJarsSecretValue: String = _ @@ -64,7 +64,7 @@ class KubernetesSparkDependencyDownloadInitContainerSuite private var filesCompressedBytes: Array[Byte] = _ private var retrofitClientFactory: RetrofitClientFactory = _ private var retrofitClient: ResourceStagingServiceRetrofit = _ - private var initContainerUnderTest: KubernetesSparkDependencyDownloadInitContainer = _ + private var fileFetcher: FileFetcher = _ override def beforeAll(): Unit = { jarsCompressedBytes = compressPathsToBytes(JARS) @@ -80,24 +80,10 @@ class KubernetesSparkDependencyDownloadInitContainerSuite downloadFilesDir = Utils.createTempDir() retrofitClientFactory = mock[RetrofitClientFactory] retrofitClient = mock[ResourceStagingServiceRetrofit] - 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(DRIVER_LOCAL_JARS_DOWNLOAD_LOCATION, downloadJarsDir.getAbsolutePath) - .set(DRIVER_LOCAL_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) - + fileFetcher = mock[FileFetcher] when(retrofitClientFactory.createRetrofitClient( STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) .thenReturn(retrofitClient) - initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( - sparkConf, retrofitClientFactory) } after { @@ -105,9 +91,15 @@ class KubernetesSparkDependencyDownloadInitContainerSuite downloadFilesDir.delete() } - test("Downloads should unpack response body streams to directories") { + 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, + securityManager = new SparkSecurityManager(sparkConf)) when(retrofitClient.downloadResources(JARS_RESOURCE_ID, downloadJarsSecretValue)) .thenReturn(downloadJarsCall) when(retrofitClient.downloadResources(FILES_RESOURCE_ID, downloadFilesSecretValue)) @@ -125,6 +117,46 @@ class KubernetesSparkDependencyDownloadInitContainerSuite 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, + securityManager = new SparkSecurityManager(sparkConf)) + 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( 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 index 23c6751f1b3ed..c5f1c43ff7cf4 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/executor/Dockerfile @@ -37,4 +37,7 @@ ENV SPARK_HOME /opt/spark WORKDIR /opt/spark # TODO support spark.executor.extraClassPath -CMD exec ${JAVA_HOME}/bin/java -Dspark.executor.port=$SPARK_EXECUTOR_PORT -Xms$SPARK_EXECUTOR_MEMORY -Xmx$SPARK_EXECUTOR_MEMORY -cp ${SPARK_HOME}/jars/\* org.apache.spark.executor.CoarseGrainedExecutorBackend --driver-url $SPARK_DRIVER_URL --executor-id $SPARK_EXECUTOR_ID --cores $SPARK_EXECUTOR_CORES --app-id $SPARK_APPLICATION_ID --hostname $SPARK_EXECUTOR_POD_IP +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 && \ + exec ${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/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index c94893cbce410..02904c0e5fe21 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -200,6 +200,28 @@ + + 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 + + + + + + com.googlecode.maven-download-plugin download-maven-plugin 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/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala index ae02de7937c6a..3be4507ac105a 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala @@ -18,23 +18,19 @@ package org.apache.spark.deploy.kubernetes.integrationtest import java.util.UUID -import scala.collection.JavaConverters._ - -import com.google.common.collect.ImmutableList import io.fabric8.kubernetes.client.internal.readiness.Readiness import org.scalatest.{BeforeAndAfter, DoNotDiscover} import org.scalatest.concurrent.Eventually +import scala.collection.JavaConverters._ -import org.apache.spark._ +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.IntegrationTestBackend 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.integrationtest.restapis.SparkRestApiV1 -import org.apache.spark.deploy.kubernetes.submit.v1.Client -import org.apache.spark.deploy.kubernetes.submit.v2.{MountedDependencyManagerProviderImpl, SubmissionKubernetesClientProviderImpl} -import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} +import org.apache.spark.deploy.kubernetes.submit.v2.Client +import org.apache.spark.launcher.SparkLauncher @DoNotDiscover private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) @@ -44,11 +40,14 @@ private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) private var kubernetesTestComponents: KubernetesTestComponents = _ private var sparkConf: SparkConf = _ private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ + private var staticAssetServerLauncher: StaticAssetServerLauncher = _ override def beforeAll(): Unit = { kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient) resourceStagingServerLauncher = new ResourceStagingServerLauncher( kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) + staticAssetServerLauncher = new StaticAssetServerLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) } before { @@ -98,7 +97,6 @@ private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) assume(testBackend.name == MINIKUBE_TEST_BACKEND) sparkConf.setJars(Seq( - KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE, KubernetesSuite.CONTAINER_LOCAL_HELPER_JAR_PATH)) runSparkPiAndVerifyCompletion(KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE) } @@ -118,6 +116,25 @@ private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) runSparkGroupByTestAndVerifyCompletion(KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE) } + test("Use remote resources without the resource staging server.") { + val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() + sparkConf.setJars(Seq( + s"$assetServerUri/${KubernetesSuite.EXAMPLES_JAR_FILE.getName}", + s"$assetServerUri/${KubernetesSuite.HELPER_JAR_FILE.getName}" + )) + runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) + } + + test("Mix remote resources with submitted ones.") { + launchStagingServer(SSLOptions()) + val assetServerUri = staticAssetServerLauncher.launchStaticAssetServer() + sparkConf.setJars(Seq( + KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + s"$assetServerUri/${KubernetesSuite.HELPER_JAR_FILE.getName}" + )) + runSparkPiAndVerifyCompletion(SparkLauncher.NO_RESOURCE) + } + private def launchStagingServer(resourceStagingServerSslOptions: SSLOptions): Unit = { assume(testBackend.name == MINIKUBE_TEST_BACKEND) @@ -134,16 +151,7 @@ private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) } private def runSparkPiAndVerifyCompletion(appResource: String): Unit = { - val client = new org.apache.spark.deploy.kubernetes.submit.v2.Client( - sparkConf = sparkConf, - mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, - appArgs = Array.empty[String], - mainAppResource = appResource, - kubernetesClientProvider = - new SubmissionKubernetesClientProviderImpl(sparkConf), - mountedDependencyManagerProvider = - new MountedDependencyManagerProviderImpl(sparkConf)) - client.run() + Client.run(sparkConf, appResource, KubernetesSuite.SPARK_PI_MAIN_CLASS, Array.empty[String]) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) @@ -160,16 +168,11 @@ private[spark] class KubernetesV2Suite(testBackend: IntegrationTestBackend) } private def runSparkGroupByTestAndVerifyCompletion(appResource: String): Unit = { - val client = new org.apache.spark.deploy.kubernetes.submit.v2.Client( + Client.run( sparkConf = sparkConf, - mainClass = KubernetesSuite.GROUP_BY_MAIN_CLASS, appArgs = Array.empty[String], - mainAppResource = appResource, - kubernetesClientProvider = - new SubmissionKubernetesClientProviderImpl(sparkConf), - mountedDependencyManagerProvider = - new MountedDependencyManagerProviderImpl(sparkConf)) - client.run() + mainClass = KubernetesSuite.GROUP_BY_MAIN_CLASS, + mainAppResource = appResource) val driverPod = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", APP_LOCATOR_LABEL) 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 index ca549fa27d630..3a99f907d15fd 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala @@ -43,7 +43,6 @@ private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesC 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" - private var activeResources = Seq.empty[HasMetadata] // Returns the NodePort the staging server is listening on def launchStagingServer(sslOptions: SSLOptions): Int = { @@ -146,8 +145,8 @@ private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesC .endPort() .endSpec() .build() - val stagingServerPodReadyWatcher = new ReadinessWatcher[Pod] - val serviceReadyWatcher = new ReadinessWatcher[Endpoints] + val stagingServerPodReadyWatcher = new SparkReadinessWatcher[Pod] + val serviceReadyWatcher = new SparkReadinessWatcher[Endpoints] val allResources = Seq( stagingServerService, stagingServerConfigMap, @@ -159,9 +158,7 @@ private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesC Utils.tryWithResource(kubernetesClient.endpoints() .withName(stagingServerService.getMetadata.getName) .watch(serviceReadyWatcher)) { _ => - activeResources = kubernetesClient.resourceList(allResources: _*) - .createOrReplace() - .asScala + kubernetesClient.resourceList(allResources: _*).createOrReplace() stagingServerPodReadyWatcher.waitUntilReady() serviceReadyWatcher.waitUntilReady() } @@ -172,25 +169,4 @@ private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesC .get(0) .getNodePort } - - def tearDownStagingServer(): Unit = { - kubernetesClient.resourceList(activeResources: _*).delete() - activeResources = Seq.empty[HasMetadata] - } - - private class ReadinessWatcher[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/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/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala index 52b8c7d7359a6..0692cf55db848 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala @@ -34,6 +34,8 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, private val SHUFFLE_SERVICE_DOCKER_FILE = "dockerfiles/shuffle-service/Dockerfile" private val DRIVER_INIT_DOCKER_FILE = "dockerfiles/driver-init/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", @@ -65,6 +67,7 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, buildImage("spark-driver-v2", DRIVER_V2_DOCKER_FILE) buildImage("spark-resource-staging-server", STAGING_SERVER_DOCKER_FILE) buildImage("spark-driver-init", DRIVER_INIT_DOCKER_FILE) + buildImage("spark-integration-test-asset-server", STATIC_ASSET_SERVER_DOCKER_FILE) } private def buildImage(name: String, dockerFile: String): Unit = {