From 04afcf81df3ea09a85f7e4825d6bd2907bc8fe34 Mon Sep 17 00:00:00 2001 From: mccheah Date: Mon, 24 Apr 2017 18:15:11 -0700 Subject: [PATCH] Driver submission with mounting dependencies from the staging server (#227) --- resource-managers/kubernetes/core/pom.xml | 8 +- .../{submit/v1 => }/CompressionUtils.scala | 123 +++--- .../spark/deploy/kubernetes/config.scala | 107 ++++- .../spark/deploy/kubernetes/constants.scala | 42 +- .../deploy/kubernetes/submit/v1/Client.scala | 17 +- ...iverPodKubernetesCredentialsProvider.scala | 1 - .../deploy/kubernetes/submit/v2/Client.scala | 249 ++++++++++++ .../v2/ContainerNameEqualityPredicate.scala | 29 ++ .../v2/MountedDependencyManagerImpl.scala | 324 +++++++++++++++ .../v2/MountedDependencyManagerProvider.scala | 58 +++ .../SubmissionKubernetesClientProvider.scala | 55 +++ .../v1/KubernetesSparkRestServer.scala | 2 +- ...SparkDependencyDownloadInitContainer.scala | 127 ++++++ .../kubernetes/v2/ResourceStagingServer.scala | 24 +- .../v2/ResourceStagingService.scala | 13 +- .../v2/ResourceStagingServiceImpl.scala | 2 + .../v2/ResourceStagingServiceRetrofit.scala | 8 +- ...tils.scala => RetrofitClientFactory.scala} | 13 +- .../v2/SparkConfPropertiesParser.scala | 46 +++ .../DriverPodKubernetesClientProvider.scala | 83 ++++ .../KubernetesClusterSchedulerBackend.scala | 14 +- .../kubernetes/submit/v2/ClientV2Suite.scala | 328 ++++++++++++++++ .../v2/MountedDependencyManagerSuite.scala | 323 +++++++++++++++ ...DependencyDownloadInitContainerSuite.scala | 165 ++++++++ .../v2/ResourceStagingServerSuite.scala | 2 +- .../kubernetes/docker-minimal-bundle/pom.xml | 17 +- ...river-assembly.xml => docker-assembly.xml} | 6 +- .../src/main/assembly/executor-assembly.xml | 84 ---- .../src/main/docker/driver-init/Dockerfile | 38 ++ .../src/main/docker/driver-v2/Dockerfile | 43 ++ .../docker/resource-staging-server/Dockerfile | 38 ++ .../kubernetes/integration-tests/pom.xml | 65 +--- .../integrationtest/KubernetesSuite.scala | 368 ++---------------- .../KubernetesTestComponents.scala | 72 ++++ .../integrationtest/KubernetesV1Suite.scala | 306 +++++++++++++++ .../integrationtest/KubernetesV2Suite.scala | 127 ++++++ .../ResourceStagingServerLauncher.scala | 196 ++++++++++ .../docker/SparkDockerImageBuilder.scala | 25 +- 38 files changed, 2932 insertions(+), 616 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/{submit/v1 => }/CompressionUtils.scala (58%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerNameEqualityPredicate.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerProvider.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmissionKubernetesClientProvider.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/{RetrofitUtils.scala => RetrofitClientFactory.scala} (85%) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/SparkConfPropertiesParser.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/DriverPodKubernetesClientProvider.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala rename resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/{driver-assembly.xml => docker-assembly.xml} (95%) delete mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-init/Dockerfile create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-v2/Dockerfile create mode 100644 resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV1Suite.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala create mode 100644 resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 8856339d4f6d9..70c252009c9b4 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -108,6 +108,8 @@ com.google.guava guava + + org.bouncycastle bcpkix-jdk15on @@ -116,7 +118,11 @@ org.bouncycastle bcprov-jdk15on - + + org.mockito + mockito-core + test + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala similarity index 58% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/CompressionUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala index 8296218ba1f70..03991ba26a6f7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.v1 +package org.apache.spark.deploy.kubernetes -import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream, InputStream, OutputStream} import java.util.zip.{GZIPInputStream, GZIPOutputStream} import com.google.common.io.Files @@ -48,40 +48,7 @@ private[spark] object CompressionUtils extends Logging { */ def createTarGzip(paths: Iterable[String]): TarGzippedData = { val compressedBytesStream = Utils.tryWithResource(new ByteBufferOutputStream()) { raw => - Utils.tryWithResource(new GZIPOutputStream(raw)) { gzipping => - Utils.tryWithResource(new TarArchiveOutputStream( - gzipping, - BLOCK_SIZE, - RECORD_SIZE, - ENCODING)) { tarStream => - val usedFileNames = mutable.HashSet.empty[String] - for (path <- paths) { - val file = new File(path) - if (!file.isFile) { - throw new IllegalArgumentException(s"Cannot add $path to tarball; either does" + - s" not exist or is a directory.") - } - var resolvedFileName = file.getName - val extension = Files.getFileExtension(file.getName) - val nameWithoutExtension = Files.getNameWithoutExtension(file.getName) - var deduplicationCounter = 1 - while (usedFileNames.contains(resolvedFileName)) { - val oldResolvedFileName = resolvedFileName - resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" - logWarning(s"File with name $oldResolvedFileName already exists. Trying to add" + - s" with file name $resolvedFileName instead.") - deduplicationCounter += 1 - } - usedFileNames += resolvedFileName - val tarEntry = new TarArchiveEntry(file, resolvedFileName) - tarStream.putArchiveEntry(tarEntry) - Utils.tryWithResource(new FileInputStream(file)) { fileInput => - IOUtils.copy(fileInput, tarStream) - } - tarStream.closeArchiveEntry() - } - } - } + writeTarGzipToStream(raw, paths) raw } val compressedAsBase64 = Base64.encodeBase64String(compressedBytesStream.toByteBuffer.array) @@ -93,6 +60,44 @@ private[spark] object CompressionUtils extends Logging { ) } + def writeTarGzipToStream(outputStream: OutputStream, paths: Iterable[String]): Unit = { + Utils.tryWithResource(new GZIPOutputStream(outputStream)) { gzipping => + Utils.tryWithResource(new TarArchiveOutputStream( + gzipping, + BLOCK_SIZE, + RECORD_SIZE, + ENCODING)) { tarStream => + val usedFileNames = mutable.HashSet.empty[String] + for (path <- paths) { + val file = new File(path) + if (!file.isFile) { + throw new IllegalArgumentException(s"Cannot add $path to tarball; either does" + + s" not exist or is a directory.") + } + var resolvedFileName = file.getName + val extension = Files.getFileExtension(file.getName) + val nameWithoutExtension = Files.getNameWithoutExtension(file.getName) + var deduplicationCounter = 1 + while (usedFileNames.contains(resolvedFileName)) { + val oldResolvedFileName = resolvedFileName + resolvedFileName = s"$nameWithoutExtension-$deduplicationCounter.$extension" + logWarning(s"File with name $oldResolvedFileName already exists. Trying to add" + + s" with file name $resolvedFileName instead.") + deduplicationCounter += 1 + } + usedFileNames += resolvedFileName + val tarEntry = new TarArchiveEntry(resolvedFileName) + tarEntry.setSize(file.length()); + tarStream.putArchiveEntry(tarEntry) + Utils.tryWithResource(new FileInputStream(file)) { fileInput => + IOUtils.copy(fileInput, tarStream) + } + tarStream.closeArchiveEntry() + } + } + } + } + /** * Decompresses the provided tar archive to a directory. * @param compressedData In-memory representation of the compressed data, ideally created via @@ -104,7 +109,6 @@ private[spark] object CompressionUtils extends Logging { def unpackAndWriteCompressedFiles( compressedData: TarGzippedData, rootOutputDir: File): Seq[String] = { - val paths = mutable.Buffer.empty[String] val compressedBytes = Base64.decodeBase64(compressedData.dataBase64) if (!rootOutputDir.exists) { if (!rootOutputDir.mkdirs) { @@ -116,24 +120,39 @@ private[spark] object CompressionUtils extends Logging { s"${rootOutputDir.getAbsolutePath} exists and is not a directory.") } Utils.tryWithResource(new ByteArrayInputStream(compressedBytes)) { compressedBytesStream => - Utils.tryWithResource(new GZIPInputStream(compressedBytesStream)) { gzipped => - Utils.tryWithResource(new TarArchiveInputStream( - gzipped, - compressedData.blockSize, - compressedData.recordSize, - compressedData.encoding)) { tarInputStream => - var nextTarEntry = tarInputStream.getNextTarEntry - while (nextTarEntry != null) { - val outputFile = new File(rootOutputDir, nextTarEntry.getName) - Utils.tryWithResource(new FileOutputStream(outputFile)) { fileOutputStream => - IOUtils.copy(tarInputStream, fileOutputStream) - } - paths += outputFile.getAbsolutePath - nextTarEntry = tarInputStream.getNextTarEntry + unpackTarStreamToDirectory( + compressedBytesStream, + rootOutputDir, + compressedData.blockSize, + compressedData.recordSize, + compressedData.encoding) + } + } + + def unpackTarStreamToDirectory( + inputStream: InputStream, + outputDir: File, + blockSize: Int = BLOCK_SIZE, + recordSize: Int = RECORD_SIZE, + encoding: String = ENCODING): Seq[String] = { + val paths = mutable.Buffer.empty[String] + Utils.tryWithResource(new GZIPInputStream(inputStream)) { gzipped => + Utils.tryWithResource(new TarArchiveInputStream( + gzipped, + blockSize, + recordSize, + encoding)) { tarInputStream => + var nextTarEntry = tarInputStream.getNextTarEntry + while (nextTarEntry != null) { + val outputFile = new File(outputDir, nextTarEntry.getName) + Utils.tryWithResource(new FileOutputStream(outputFile)) { fileOutputStream => + IOUtils.copy(tarInputStream, fileOutputStream) } + paths += outputFile.getAbsolutePath + nextTarEntry = tarInputStream.getNextTarEntry } } } - paths.toSeq + paths } } 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 15f7a17857f1f..1c8b6798bbdd5 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 @@ -19,11 +19,13 @@ package org.apache.spark.deploy.kubernetes import java.util.concurrent.TimeUnit import org.apache.spark.{SPARK_VERSION => sparkVersion} +import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.v1.NodePortUrisDriverServiceManager +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -package object config { +package object config extends Logging { private[spark] val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") @@ -321,4 +323,107 @@ package object config { .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") + .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") + .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") + .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") + .doc("Type of trustStore for communicating with the dependency server.") + .stringConf + .createOptional + + // Driver and Init-Container parameters for submission v2 + private[spark] val RESOURCE_STAGING_SERVER_URI = + ConfigBuilder("spark.kubernetes.resourceStagingServer.uri") + .doc("Base URI for the Spark resource staging server") + .stringConf + .createOptional + + private[spark] val INIT_CONTAINER_DOWNLOAD_JARS_RESOURCE_IDENTIFIER = + ConfigBuilder("spark.kubernetes.driver.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") + .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) + + private[spark] val INIT_CONTAINER_DOWNLOAD_FILES_RESOURCE_IDENTIFIER = + ConfigBuilder("spark.kubernetes.driver.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") + .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) + + 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.") + .stringConf + .createWithDefault(s"spark-driver-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.") + .stringConf + .createWithDefault("/var/spark-data/spark-local-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.") + .stringConf + .createWithDefault("/var/spark-data/spark-local-files") + + private[spark] val DRIVER_MOUNT_DEPENDENCIES_INIT_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.") + .timeConf(TimeUnit.MINUTES) + .createWithDefault(5) + + private[spark] def resolveK8sMaster(rawMasterString: String): String = { + if (!rawMasterString.startsWith("k8s://")) { + throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") + } + val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") + if (masterWithoutK8sPrefix.startsWith("http://") + || masterWithoutK8sPrefix.startsWith("https://")) { + masterWithoutK8sPrefix + } else { + val resolvedURL = s"https://$masterWithoutK8sPrefix" + logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + + s" URL is $resolvedURL") + resolvedURL + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 03b3d21ac9c45..f82cb88b4c622 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 @@ -30,9 +30,9 @@ package object constants { private[spark] val SUBMISSION_APP_SECRET_PREFIX = "spark-submission-server-secret" private[spark] val SUBMISSION_APP_SECRET_VOLUME_NAME = "spark-submission-secret-volume" private[spark] val SUBMISSION_SSL_KEY_PASSWORD_SECRET_NAME = - "spark-submission-server-key-password" + "spark-submission-server-key-password" private[spark] val SUBMISSION_SSL_KEYSTORE_PASSWORD_SECRET_NAME = - "spark-submission-server-keystore-password" + "spark-submission-server-keystore-password" private[spark] val SUBMISSION_SSL_KEYSTORE_SECRET_NAME = "spark-submission-server-keystore" private[spark] val SUBMISSION_SSL_SECRETS_PREFIX = "spark-submission-server-ssl" private[spark] val SUBMISSION_SSL_SECRETS_VOLUME_NAME = "spark-submission-server-ssl-secrets" @@ -55,9 +55,9 @@ package object constants { private[spark] val ENV_SUBMISSION_SERVER_PORT = "SPARK_SUBMISSION_SERVER_PORT" private[spark] val ENV_SUBMISSION_KEYSTORE_FILE = "SPARK_SUBMISSION_KEYSTORE_FILE" private[spark] val ENV_SUBMISSION_KEYSTORE_PASSWORD_FILE = - "SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE" + "SPARK_SUBMISSION_KEYSTORE_PASSWORD_FILE" private[spark] val ENV_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE = - "SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE" + "SPARK_SUBMISSION_KEYSTORE_KEY_PASSWORD_FILE" private[spark] val ENV_SUBMISSION_KEYSTORE_TYPE = "SPARK_SUBMISSION_KEYSTORE_TYPE" private[spark] val ENV_SUBMISSION_KEY_PEM_FILE = "SPARK_SUBMISSION_KEY_PEM_FILE" private[spark] val ENV_SUBMISSION_CERT_PEM_FILE = "SPARK_SUBMISSION_CERT_PEM_FILE" @@ -70,12 +70,18 @@ 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_MOUNTED_CLASSPATH = "SPARK_MOUNTED_CLASSPATH" + private[spark] val ENV_DRIVER_MAIN_CLASS = "SPARK_DRIVER_CLASS" + private[spark] val ENV_DRIVER_ARGS = "SPARK_DRIVER_ARGS" + private[spark] val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS" // Annotation keys private[spark] val ANNOTATION_PROVIDE_EXTERNAL_URI = - "spark-job.alpha.apache.org/provideExternalUri" + "spark-job.alpha.apache.org/provideExternalUri" private[spark] val ANNOTATION_RESOLVED_EXTERNAL_URI = - "spark-job.alpha.apache.org/resolvedExternalUri" + "spark-job.alpha.apache.org/resolvedExternalUri" // Miscellaneous private[spark] val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" @@ -83,4 +89,28 @@ package object constants { private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 private[spark] val MEMORY_OVERHEAD_MIN = 384L + + // 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_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" } 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 72d24f7bf8342..e1cfac8feba37 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 @@ -30,6 +30,7 @@ import org.apache.commons.codec.binary.Base64 import scala.collection.JavaConverters._ 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} @@ -733,20 +734,4 @@ private[spark] object Client extends Logging { sparkConf = sparkConf, appArgs = appArgs).run() } - - def resolveK8sMaster(rawMasterString: String): String = { - if (!rawMasterString.startsWith("k8s://")) { - throw new IllegalArgumentException("Master URL should start with k8s:// in Kubernetes mode.") - } - val masterWithoutK8sPrefix = rawMasterString.replaceFirst("k8s://", "") - if (masterWithoutK8sPrefix.startsWith("http://") - || masterWithoutK8sPrefix.startsWith("https://")) { - masterWithoutK8sPrefix - } else { - val resolvedURL = s"https://$masterWithoutK8sPrefix" - logDebug(s"No scheme specified for kubernetes master URL, so defaulting to https. Resolved" + - s" URL is $resolvedURL") - resolvedURL - } - } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverPodKubernetesCredentialsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverPodKubernetesCredentialsProvider.scala index bc7490ef9ec4a..112226dbe3fc1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverPodKubernetesCredentialsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v1/DriverPodKubernetesCredentialsProvider.scala @@ -45,7 +45,6 @@ private[spark] class DriverPodKubernetesCredentialsProvider(sparkConf: SparkConf s"Driver client key file provided at %s does not exist or is not a file.") val clientCertDataBase64 = safeFileConfToBase64(KUBERNETES_DRIVER_CLIENT_CERT_FILE, s"Driver client cert file provided at %s does not exist or is not a file.") - val serviceAccountName = sparkConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) KubernetesCredentials( oauthToken = oauthToken, caCertDataBase64 = caCertDataBase64, 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 new file mode 100644 index 0000000000000..69dbfd041bb86 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/Client.scala @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 java.util.Collections + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata, OwnerReferenceBuilder, PodBuilder} +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.util.Utils + +/** + * Submission client for launching Spark applications on Kubernetes clusters. + * + * 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). + */ +private[spark] class Client( + mainClass: String, + sparkConf: SparkConf, + appArgs: Array[String], + mainAppResource: String, + kubernetesClientProvider: SubmissionKubernetesClientProvider, + mountedDependencyManagerProvider: MountedDependencyManagerProvider) 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 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) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMb).toInt, + MEMORY_OVERHEAD_MIN)) + 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 = { + val parsedCustomLabels = parseKeyValuePairs(customLabels, KUBERNETES_DRIVER_LABELS.key, + "labels") + require(!parsedCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping operations.") + require(!parsedCustomLabels.contains(SPARK_APP_NAME_LABEL), s"Label with key" + + s" $SPARK_APP_NAME_LABEL is not allowed as it is reserved for Spark bookkeeping operations.") + val allLabels = parsedCustomLabels ++ + Map(SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_APP_NAME_LABEL -> appName) + val parsedCustomAnnotations = parseKeyValuePairs( + customAnnotations, + KUBERNETES_DRIVER_ANNOTATIONS.key, + "annotations") + Utils.tryWithResource(kubernetesClientProvider.get) { kubernetesClient => + val driverExtraClasspathEnv = driverExtraClasspath.map { classPath => + new EnvVarBuilder() + .withName(ENV_SUBMIT_EXTRA_CLASSPATH) + .withValue(classPath) + .build() + } + val driverContainer = new ContainerBuilder() + .withName(DRIVER_CONTAINER_NAME) + .withImage(driverDockerImage) + .withImagePullPolicy("IfNotPresent") + .addToEnv(driverExtraClasspathEnv.toSeq: _*) + .addNewEnv() + .withName(ENV_DRIVER_MEMORY) + .withValue(driverContainerMemoryWithOverhead + "m") + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_MAIN_CLASS) + .withValue(mainClass) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_ARGS) + .withValue(appArgs.mkString(" ")) + .endEnv() + .build() + val basePod = new PodBuilder() + .withNewMetadata() + .withName(kubernetesAppId) + .addToLabels(allLabels.asJava) + .addToAnnotations(parsedCustomAnnotations.asJava) + .endMetadata() + .withNewSpec() + .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 resolvedSparkConf = sparkConf.clone() + if (resolvedJars.nonEmpty) { + resolvedSparkConf.set("spark.jars", resolvedJars.mkString(",")) + } + if (resolvedFiles.nonEmpty) { + resolvedSparkConf.set("spark.files", resolvedFiles.mkString(",")) + } + resolvedSparkConf.set(KUBERNETES_DRIVER_POD_NAME, kubernetesAppId) + resolvedSparkConf.set("spark.app.id", kubernetesAppId) + // We don't need this anymore since we just set the JVM options on the environment + resolvedSparkConf.remove(org.apache.spark.internal.config.DRIVER_JAVA_OPTIONS) + resolvedSparkConf.get(KUBERNETES_SUBMIT_OAUTH_TOKEN).foreach { _ => + resolvedSparkConf.set(KUBERNETES_SUBMIT_OAUTH_TOKEN.key, "") + } + 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" + }.mkString(" ") + driverJavaOptions.map(" " + _).getOrElse("") + val resolvedDriverPod = driverPodWithMountedDeps.editSpec() + .editMatchingContainer(new ContainerNameEqualityPredicate(driverContainer.getName)) + .addNewEnv() + .withName(ENV_MOUNTED_CLASSPATH) + .withValue(mountedClassPath) + .endEnv() + .addNewEnv() + .withName(ENV_DRIVER_JAVA_OPTS) + .withValue(resolvedDriverJavaOpts) + .endEnv() + .endContainer() + .endSpec() + .build() + val createdDriverPod = kubernetesClient.pods().create(resolvedDriverPod) + try { + val driverPodOwnerReference = new OwnerReferenceBuilder() + .withName(createdDriverPod.getMetadata.getName) + .withApiVersion(createdDriverPod.getApiVersion) + .withUid(createdDriverPod.getMetadata.getUid) + .withKind(createdDriverPod.getKind) + .withController(true) + .build() + nonDriverPodKubernetesResources.foreach { resource => + val originalMetadata = resource.getMetadata + originalMetadata.setOwnerReferences(Collections.singletonList(driverPodOwnerReference)) + } + kubernetesClient.resourceList(nonDriverPodKubernetesResources: _*).createOrReplace() + } catch { + case e: Throwable => + kubernetesClient.pods().delete(createdDriverPod) + throw e + } + } + } + + private def parseKeyValuePairs( + maybeKeyValues: Option[String], + configKey: String, + keyValueType: String): Map[String, String] = { + maybeKeyValues.map(keyValues => { + keyValues.split(",").map(_.trim).filterNot(_.isEmpty).map(keyValue => { + keyValue.split("=", 2).toSeq match { + case Seq(k, v) => + (k, v) + case _ => + throw new SparkException(s"Custom $keyValueType set by $configKey must be a" + + s" comma-separated list of key-value pairs, with format =." + + s" Got value: $keyValue. All values: $keyValues") + } + }).toMap + }).getOrElse(Map.empty[String, String]) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerNameEqualityPredicate.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerNameEqualityPredicate.scala new file mode 100644 index 0000000000000..5101e1506e4d5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/ContainerNameEqualityPredicate.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import java.lang.Boolean + +import io.fabric8.kubernetes.api.builder.Predicate +import io.fabric8.kubernetes.api.model.ContainerBuilder + +private[spark] class ContainerNameEqualityPredicate(containerName: String) + extends Predicate[ContainerBuilder] { + override def apply(item: ContainerBuilder): Boolean = { + item.getName == containerName + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala new file mode 100644 index 0000000000000..9dbbcd0d56a3b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerImpl.scala @@ -0,0 +1,324 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 new file mode 100644 index 0000000000000..8f09112132b2c --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerProvider.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.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/SubmissionKubernetesClientProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmissionKubernetesClientProvider.scala new file mode 100644 index 0000000000000..af3de6ce85026 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/v2/SubmissionKubernetesClientProvider.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.v2 + +import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging + +trait SubmissionKubernetesClientProvider { + def get: KubernetesClient +} + +private[spark] class SubmissionKubernetesClientProviderImpl(sparkConf: SparkConf) + extends SubmissionKubernetesClientProvider with Logging { + + private val namespace = sparkConf.get(KUBERNETES_NAMESPACE) + private val master = resolveK8sMaster(sparkConf.get("spark.master")) + + override def get: KubernetesClient = { + var k8ConfBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(master) + .withNamespace(namespace) + sparkConf.get(KUBERNETES_SUBMIT_CA_CERT_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withCaCertFile(f) + } + sparkConf.get(KUBERNETES_SUBMIT_CLIENT_KEY_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientKeyFile(f) + } + sparkConf.get(KUBERNETES_SUBMIT_CLIENT_CERT_FILE).foreach { + f => k8ConfBuilder = k8ConfBuilder.withClientCertFile(f) + } + sparkConf.get(KUBERNETES_SUBMIT_OAUTH_TOKEN).foreach { token => + k8ConfBuilder = k8ConfBuilder.withOauthToken(token) + } + val k8ClientConfig = k8ConfBuilder.build + new DefaultKubernetesClient(k8ClientConfig) + } +} 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 048427fa4ec23..ca05fe767146b 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 @@ -33,8 +33,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SecurityManager, SPARK_VERSION => sparkVersion, SparkConf, SparkException, SSLOptions} 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.v1.CompressionUtils 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 new file mode 100644 index 0000000000000..680d305985cc0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainer.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.rest.kubernetes.v2 + +import java.io.File +import java.util.concurrent.TimeUnit + +import com.google.common.base.Charsets +import com.google.common.io.Files +import com.google.common.util.concurrent.SettableFuture +import okhttp3.ResponseBody +import retrofit2.{Call, Callback, Response} + +import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SparkException} +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 + +private trait WaitableCallback[T] extends Callback[T] { + private val complete = SettableFuture.create[Boolean] + + override final def onFailure(call: Call[T], t: Throwable): Unit = complete.setException(t) + + override final def onResponse(call: Call[T], response: Response[T]): Unit = { + require(response.code() >= 200 && response.code() < 300, Option(response.errorBody()) + .map(_.string()) + .getOrElse(s"Error executing HTTP request, but error body was not provided.")) + handleResponse(response.body()) + complete.set(true) + } + + protected def handleResponse(body: T): Unit + + final def waitForCompletion(time: Long, timeUnit: TimeUnit): Unit = { + complete.get(time, timeUnit) + } +} + +private class DownloadTarGzCallback(downloadDir: File) extends WaitableCallback[ResponseBody] { + + override def handleResponse(responseBody: ResponseBody): Unit = { + Utils.tryWithResource(responseBody.byteStream()) { responseStream => + CompressionUtils.unpackTarStreamToDirectory(responseStream, downloadDir) + } + } +} + +private[spark] class KubernetesSparkDependencyDownloadInitContainer( + sparkConf: SparkConf, retrofitClientFactory: RetrofitClientFactory) extends Logging { + + private val resourceStagingServerUri = sparkConf.get(RESOURCE_STAGING_SERVER_URI) + .getOrElse(throw new SparkException("No dependency server URI was provided.")) + + private val downloadJarsResourceIdentifier = 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 + .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 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) + + 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}") + } +} + +object KubernetesSparkDependencyDownloadInitContainer extends Logging { + def main(args: Array[String]): Unit = { + logInfo("Starting init-container to download Spark application dependencies.") + val sparkConf = if (args.nonEmpty) { + SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) + } else { + new SparkConf(true) + } + new KubernetesSparkDependencyDownloadInitContainer(sparkConf, RetrofitClientFactoryImpl).run() + logInfo("Finished downloading application dependencies.") + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServer.scala index 8ca13da545d5d..4ecb6369ff3b0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServer.scala @@ -16,13 +16,11 @@ */ package org.apache.spark.deploy.rest.kubernetes.v2 -import java.io.{File, FileInputStream} -import java.util.Properties +import java.io.File import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider import com.fasterxml.jackson.module.scala.DefaultScalaModule -import com.google.common.collect.Maps import org.eclipse.jetty.http.HttpVersion import org.eclipse.jetty.server.{HttpConfiguration, HttpConnectionFactory, Server, ServerConnector, SslConnectionFactory} import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} @@ -30,12 +28,10 @@ import org.eclipse.jetty.util.thread.{QueuedThreadPool, ScheduledExecutorSchedul import org.glassfish.jersey.media.multipart.MultiPartFeature import org.glassfish.jersey.server.ResourceConfig import org.glassfish.jersey.servlet.ServletContainer -import scala.collection.JavaConverters._ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider} import org.apache.spark.util.Utils private[spark] class ResourceStagingServer( @@ -97,20 +93,10 @@ private[spark] class ResourceStagingServer( object ResourceStagingServer { def main(args: Array[String]): Unit = { - val sparkConf = new SparkConf(true) - if (args.nonEmpty) { - val propertiesFile = new File(args(0)) - if (!propertiesFile.isFile) { - throw new IllegalArgumentException(s"Server properties file given at" + - s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.") - } - val properties = new Properties - Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load) - val propertiesMap = Maps.fromProperties(properties) - val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap)) - propertiesMap.asScala.keys.foreach { key => - configReader.get(key).foreach(sparkConf.set(key, _)) - } + val sparkConf = if (args.nonEmpty) { + SparkConfPropertiesParser.getSparkConfFromPropertiesFile(new File(args(0))) + } else { + new SparkConf(true) } val dependenciesRootDir = Utils.createTempDir(namePrefix = "local-application-dependencies") val serviceInstance = new ResourceStagingServiceImpl(dependenciesRootDir) 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 95cc6ab949d5c..844809dec995c 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 @@ -80,6 +80,15 @@ private[spark] trait ResourceStagingService { @Produces(Array(MediaType.APPLICATION_OCTET_STREAM)) @Path("/resources/{resourceId}") def downloadResources( - @PathParam("resourceId") resourceId: String, - @HeaderParam("Authorization") resourceSecret: String): StreamingOutput + @PathParam("resourceId") resourceId: String, + @HeaderParam("Authorization") resourceSecret: String): StreamingOutput + + /** + * Health check. + */ + @GET + @Consumes(Array(MediaType.APPLICATION_JSON)) + @Produces(Array(MediaType.TEXT_PLAIN)) + @Path("/ping") + def ping(): String } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServiceImpl.scala index 732969cd67d89..cf6180fbf53d4 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 @@ -88,6 +88,8 @@ private[spark] class ResourceStagingServiceImpl(dependenciesRootDir: File) } } } + + override def ping(): String = "pong" } private case class StagedResources( 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 daf03f764b35a..b1a3cc0676757 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 @@ -37,6 +37,10 @@ private[spark] trait ResourceStagingServiceRetrofit { @Streaming @retrofit2.http.GET("/api/v0/resources/{resourceId}") - def downloadResources(@Path("resourceId") resourceId: String, - @retrofit2.http.Header("Authorization") resourceSecret: String): Call[ResponseBody] + def downloadResources( + @Path("resourceId") resourceId: String, + @retrofit2.http.Header("Authorization") resourceSecret: String): Call[ResponseBody] + + @retrofit2.http.GET("/api/ping") + def ping(): String } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitClientFactory.scala similarity index 85% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitClientFactory.scala index 7416c624e97f6..f906423524944 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/RetrofitClientFactory.scala @@ -22,21 +22,26 @@ import javax.net.ssl.{SSLContext, TrustManagerFactory, X509TrustManager} import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import okhttp3.OkHttpClient +import okhttp3.{Dispatcher, OkHttpClient} import retrofit2.Retrofit import retrofit2.converter.jackson.JacksonConverterFactory import retrofit2.converter.scalars.ScalarsConverterFactory import org.apache.spark.SSLOptions -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} -private[spark] object RetrofitUtils { +private[spark] trait RetrofitClientFactory { + def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T +} + +private[spark] object RetrofitClientFactoryImpl extends RetrofitClientFactory { private val OBJECT_MAPPER = new ObjectMapper().registerModule(new DefaultScalaModule) private val SECURE_RANDOM = new SecureRandom() def createRetrofitClient[T](baseUrl: String, serviceType: Class[T], sslOptions: SSLOptions): T = { - val okHttpClientBuilder = new OkHttpClient.Builder() + val dispatcher = new Dispatcher(ThreadUtils.newDaemonCachedThreadPool(s"http-client-$baseUrl")) + val okHttpClientBuilder = new OkHttpClient.Builder().dispatcher(dispatcher) sslOptions.trustStore.foreach { trustStoreFile => require(trustStoreFile.isFile, s"TrustStore provided at ${trustStoreFile.getAbsolutePath}" + " does not exist, or is not a file.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/SparkConfPropertiesParser.scala new file mode 100644 index 0000000000000..cf9decab127c5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/v2/SparkConfPropertiesParser.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.rest.kubernetes.v2 + +import java.io.{File, FileInputStream} +import java.util.Properties + +import com.google.common.collect.Maps +import scala.collection.JavaConverters.mapAsScalaMapConverter + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.{ConfigReader, SparkConfigProvider} +import org.apache.spark.util.Utils + +private[spark] object SparkConfPropertiesParser { + + def getSparkConfFromPropertiesFile(propertiesFile: File): SparkConf = { + val sparkConf = new SparkConf(true) + if (!propertiesFile.isFile) { + throw new IllegalArgumentException(s"Server properties file given at" + + s" ${propertiesFile.getAbsoluteFile} does not exist or is not a file.") + } + val properties = new Properties + Utils.tryWithResource(new FileInputStream(propertiesFile))(properties.load) + val propertiesMap = Maps.fromProperties(properties) + val configReader = new ConfigReader(new SparkConfigProvider(propertiesMap)) + propertiesMap.asScala.keys.foreach { key => + configReader.get(key).foreach(sparkConf.set(key, _)) + } + sparkConf + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/DriverPodKubernetesClientProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/DriverPodKubernetesClientProvider.scala new file mode 100644 index 0000000000000..b8c2b0c91bbeb --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/DriverPodKubernetesClientProvider.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.scheduler.cluster.kubernetes + +import java.io.File + +import com.google.common.base.Charsets +import com.google.common.io.Files +import io.fabric8.kubernetes.client.{Config, ConfigBuilder, DefaultKubernetesClient} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ + +private[spark] class DriverPodKubernetesClientProvider(sparkConf: SparkConf, namespace: String) { + private val SERVICE_ACCOUNT_TOKEN = new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH) + private val SERVICE_ACCOUNT_CA_CERT = new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH) + private val oauthTokenFile = sparkConf.get(KUBERNETES_DRIVER_MOUNTED_OAUTH_TOKEN) + private val caCertFile = sparkConf.get(KUBERNETES_DRIVER_MOUNTED_CA_CERT_FILE) + private val clientKeyFile = sparkConf.get(KUBERNETES_DRIVER_MOUNTED_CLIENT_KEY_FILE) + private val clientCertFile = sparkConf.get(KUBERNETES_DRIVER_MOUNTED_CLIENT_CERT_FILE) + + /** + * Creates a {@link KubernetesClient}, expecting to be from within the context of a pod. When + * doing so, service account token files can be picked up from canonical locations. + */ + def get: DefaultKubernetesClient = { + val baseClientConfigBuilder = new ConfigBuilder() + .withApiVersion("v1") + .withMasterUrl(KUBERNETES_MASTER_INTERNAL_URL) + .withNamespace(namespace) + + val configBuilder = oauthTokenFile + .orElse(caCertFile) + .orElse(clientKeyFile) + .orElse(clientCertFile) + .map { _ => + var mountedAuthConfigBuilder = baseClientConfigBuilder + oauthTokenFile.foreach { tokenFilePath => + val tokenFile = new File(tokenFilePath) + mountedAuthConfigBuilder = mountedAuthConfigBuilder + .withOauthToken(Files.toString(tokenFile, Charsets.UTF_8)) + } + caCertFile.foreach { caFile => + mountedAuthConfigBuilder = mountedAuthConfigBuilder.withCaCertFile(caFile) + } + clientKeyFile.foreach { keyFile => + mountedAuthConfigBuilder = mountedAuthConfigBuilder.withClientKeyFile(keyFile) + } + clientCertFile.foreach { certFile => + mountedAuthConfigBuilder = mountedAuthConfigBuilder.withClientCertFile(certFile) + } + mountedAuthConfigBuilder + }.getOrElse { + var serviceAccountConfigBuilder = baseClientConfigBuilder + if (SERVICE_ACCOUNT_CA_CERT.isFile) { + serviceAccountConfigBuilder = serviceAccountConfigBuilder.withCaCertFile( + SERVICE_ACCOUNT_CA_CERT.getAbsolutePath) + } + + if (SERVICE_ACCOUNT_TOKEN.isFile) { + serviceAccountConfigBuilder = serviceAccountConfigBuilder.withOauthToken( + Files.toString(SERVICE_ACCOUNT_TOKEN, Charsets.UTF_8)) + } + serviceAccountConfigBuilder + } + new DefaultKubernetesClient(configBuilder.build) + } +} 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 130b143c7e92b..15457db7e1459 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 @@ -47,11 +47,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private val blockmanagerPort = conf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - private val kubernetesDriverServiceName = conf - .get(KUBERNETES_DRIVER_SERVICE_NAME) - .getOrElse( - throw new SparkException("Must specify the service name the driver is running with")) - private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) .getOrElse( @@ -73,8 +68,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) - private val kubernetesClient = new KubernetesClientBuilder(conf, kubernetesNamespace) - .buildFromWithinPod() + private val kubernetesClient = new DriverPodKubernetesClientProvider(conf, kubernetesNamespace) + .get private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). @@ -142,11 +137,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } catch { case e: Throwable => logError("Uncaught exception while shutting down controllers.", e) } - try { - kubernetesClient.services().withName(kubernetesDriverServiceName).delete() - } catch { - case e: Throwable => logError("Uncaught exception while shutting down driver service.", e) - } try { kubernetesClient.close() } catch { 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 new file mode 100644 index 0000000000000..9e2ab26460412 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/ClientV2Suite.scala @@ -0,0 +1,328 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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 io.fabric8.kubernetes.api.model.{ConfigMap, ConfigMapBuilder, Container, 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, startsWith, eq => mockitoEq} +import org.mockito.Mockito.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 org.apache.spark.{SparkConf, SparkFunSuite} +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 SPARK_JARS = Seq( + "local:///app/jars/spark-helper.jar", "file:///var/data/spark-local-helper.jar") + private val RESOLVED_SPARK_JARS = Seq( + "local:///app/jars/spark-helper.jar", + "file:///var/data/spark-downloaded/spark-local-helper.jar") + private val SPARK_FILES = Seq( + "local:///app/files/spark-file.txt", "file:///var/data/spark-local-file.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] = _ + + 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") + } + } + } + + // 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("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) + } + } + + 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 getInitContainerConfigMap(): ConfigMap = { + new ConfigMapBuilder() + .withNewMetadata().withName(s"$APP_NAME-init-container-conf").endMetadata() + .addToData("key", "configuration") + .build() + } + + 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 + } + } + + override def describeTo(description: Description) = { + description.appendText("Checks if the labels contain the app ID and app name.") + } + } + 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() + } + + private def getDriverContainer(driverPod: Pod): Option[Container] = { + val maybeDriverContainer = driverPod.getSpec + .getContainers + .asScala + .find(_.getName == DRIVER_CONTAINER_NAME) + assert(maybeDriverContainer.isDefined) + maybeDriverContainer + } + + private def createAndGetDriverPod(): Pod = { + val clientUnderTest = createClient() + clientUnderTest.run() + val createdDriverPod = captureCreatedPodAnswer.capturedArgument + assert(createdDriverPod != null) + createdDriverPod + } + + private def createClient(): Client = { + new Client( + MAIN_CLASS, + sparkConf, + APP_ARGS, + MAIN_APP_RESOURCE, + submissionKubernetesClientProvider, + mountedDependencyManagerProvider) + } + + 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 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 + } + } +} 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 new file mode 100644 index 0000000000000..321fe1b3fd889 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/v2/MountedDependencyManagerSuite.scala @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala new file mode 100644 index 0000000000000..77eb7f2b9f49c --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.rest.kubernetes.v2 + +import java.io.{ByteArrayOutputStream, File} +import java.util.UUID +import javax.ws.rs.core + +import com.google.common.base.Charsets +import com.google.common.io.Files +import okhttp3.{MediaType, ResponseBody} +import org.mockito.Matchers.any +import org.mockito.Mockito.{doAnswer, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import retrofit2.{Call, Callback, Response} + +import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} +import org.apache.spark.deploy.kubernetes.CompressionUtils +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.util.Utils + +class KubernetesSparkDependencyDownloadInitContainerSuite + extends SparkFunSuite with BeforeAndAfter { + import KubernetesSparkDependencyDownloadInitContainerSuite.createTempFile + private val STAGING_SERVER_URI = "http://localhost:8000" + private val TRUSTSTORE_FILE = new File(createTempFile(".jks")) + private val TRUSTSTORE_PASSWORD = "trustStorePassword" + private val TRUSTSTORE_TYPE = "jks" + private val STAGING_SERVER_SSL_OPTIONS = SSLOptions( + enabled = true, + trustStore = Some(TRUSTSTORE_FILE), + trustStorePassword = Some(TRUSTSTORE_PASSWORD), + trustStoreType = Some(TRUSTSTORE_TYPE)) + private val JARS = Seq(createTempFile("jar"), createTempFile("jar")) + private val FILES = Seq(createTempFile("txt"), createTempFile("csv")) + private val DOWNLOAD_JARS_SECRET_LOCATION = createTempFile("txt") + private val DOWNLOAD_FILES_SECRET_LOCATION = createTempFile("txt") + private val JARS_RESOURCE_ID = "jarsId" + private val FILES_RESOURCE_ID = "filesId" + + private var sparkConf: SparkConf = _ + private var downloadJarsDir: File = _ + private var downloadFilesDir: File = _ + private var downloadJarsSecretValue: String = _ + private var downloadFilesSecretValue: String = _ + private var jarsCompressedBytes: Array[Byte] = _ + private var filesCompressedBytes: Array[Byte] = _ + private var retrofitClientFactory: RetrofitClientFactory = _ + private var retrofitClient: ResourceStagingServiceRetrofit = _ + private var initContainerUnderTest: KubernetesSparkDependencyDownloadInitContainer = _ + + override def beforeAll(): Unit = { + jarsCompressedBytes = compressPathsToBytes(JARS) + filesCompressedBytes = compressPathsToBytes(FILES) + downloadJarsSecretValue = Files.toString( + new File(DOWNLOAD_JARS_SECRET_LOCATION), Charsets.UTF_8) + downloadFilesSecretValue = Files.toString( + new File(DOWNLOAD_FILES_SECRET_LOCATION), Charsets.UTF_8) + } + + before { + downloadJarsDir = Utils.createTempDir() + downloadFilesDir = Utils.createTempDir() + retrofitClientFactory = mock[RetrofitClientFactory] + retrofitClient = mock[ResourceStagingServiceRetrofit] + 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) + + when(retrofitClientFactory.createRetrofitClient( + STAGING_SERVER_URI, classOf[ResourceStagingServiceRetrofit], STAGING_SERVER_SSL_OPTIONS)) + .thenReturn(retrofitClient) + initContainerUnderTest = new KubernetesSparkDependencyDownloadInitContainer( + sparkConf, retrofitClientFactory) + } + + after { + downloadJarsDir.delete() + downloadFilesDir.delete() + } + + test("Downloads should unpack response body streams to directories") { + val downloadJarsCall = mock[Call[ResponseBody]] + val downloadFilesCall = mock[Call[ResponseBody]] + when(retrofitClient.downloadResources(JARS_RESOURCE_ID, downloadJarsSecretValue)) + .thenReturn(downloadJarsCall) + when(retrofitClient.downloadResources(FILES_RESOURCE_ID, downloadFilesSecretValue)) + .thenReturn(downloadFilesCall) + val jarsResponseBody = ResponseBody.create( + MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), jarsCompressedBytes) + val filesResponseBody = ResponseBody.create( + MediaType.parse(core.MediaType.APPLICATION_OCTET_STREAM), filesCompressedBytes) + doAnswer(new InvokeCallbackAnswer(downloadJarsCall, jarsResponseBody)) + .when(downloadJarsCall) + .enqueue(any()) + doAnswer(new InvokeCallbackAnswer(downloadFilesCall, filesResponseBody)) + .when(downloadFilesCall) + .enqueue(any()) + initContainerUnderTest.run() + checkWrittenFilesAreTheSameAsOriginal(JARS, downloadJarsDir) + checkWrittenFilesAreTheSameAsOriginal(FILES, downloadFilesDir) + } + + private def checkWrittenFilesAreTheSameAsOriginal( + originalFiles: Iterable[String], downloadDir: File): Unit = { + originalFiles.map(new File(_)).foreach { file => + val writtenFile = new File(downloadDir, file.getName) + assert(writtenFile.exists) + val originalJarContents = Seq(Files.toByteArray(file): _*) + val writtenJarContents = Seq(Files.toByteArray(writtenFile): _*) + assert(writtenJarContents === originalJarContents) + } + } + + private def compressPathsToBytes(paths: Iterable[String]): Array[Byte] = { + Utils.tryWithResource(new ByteArrayOutputStream()) { compressedBytes => + CompressionUtils.writeTarGzipToStream (compressedBytes, paths) + compressedBytes.toByteArray + } + } +} + +private object KubernetesSparkDependencyDownloadInitContainerSuite { + def createTempFile(extension: String): String = { + val dir = Utils.createTempDir() + val file = new File(dir, s"${UUID.randomUUID().toString}.$extension") + Files.write(UUID.randomUUID().toString, file, Charsets.UTF_8) + file.getAbsolutePath + } +} + +private class InvokeCallbackAnswer(call: Call[ResponseBody], responseBody: ResponseBody) + extends Answer[Unit] { + override def answer(invocationOnMock: InvocationOnMock): Unit = { + val callback = invocationOnMock.getArgumentAt(0, classOf[Callback[ResponseBody]]) + val response = Response.success(responseBody) + callback.onResponse(call, response) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServerSuite.scala index 51c5e43af1124..08be8af30b3bc 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/v2/ResourceStagingServerSuite.scala @@ -77,7 +77,7 @@ class ResourceStagingServerSuite extends SparkFunSuite with BeforeAndAfter { private def runUploadAndDownload(sslOptions: SSLOptions): Unit = { val scheme = if (sslOptions.enabled) "https" else "http" - val retrofitService = RetrofitUtils.createRetrofitClient( + val retrofitService = RetrofitClientFactoryImpl.createRetrofitClient( s"$scheme://127.0.0.1:$serverPort/", classOf[ResourceStagingServiceRetrofit], sslOptions) diff --git a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml index e9f88e37a5f89..a10fe8fb58408 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/pom.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/pom.xml @@ -72,27 +72,14 @@ maven-assembly-plugin - driver-docker-dist + docker-dist pre-integration-test single - src/main/assembly/driver-assembly.xml - - posix - - - - executor-docker-dist - pre-integration-test - - single - - - - src/main/assembly/executor-assembly.xml + src/main/assembly/docker-assembly.xml posix diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml similarity index 95% rename from resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml rename to resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml index b5fcaa75f049c..2b48d366256fe 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/driver-assembly.xml +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/docker-assembly.xml @@ -15,7 +15,7 @@ ~ limitations under the License. --> - driver-docker-dist + docker-dist tar.gz dir @@ -51,9 +51,9 @@ - src/main/docker/driver + src/main/docker/ - + dockerfiles **/* diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml b/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml deleted file mode 100644 index d97ba56562a12..0000000000000 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/assembly/executor-assembly.xml +++ /dev/null @@ -1,84 +0,0 @@ - - - executor-docker-dist - - tar.gz - dir - - false - - - - ${project.parent.basedir}/core/src/main/resources/org/apache/spark/ui/static/ - - ui-resources/org/apache/spark/ui/static - - **/* - - - - - ${project.parent.basedir}/sbin/ - - sbin - - **/* - - - - - ${project.parent.basedir}/bin/ - - bin - - **/* - - - - - ${project.parent.basedir}/conf/ - - conf - - **/* - - - - - src/main/docker/executor - - - - **/* - - - - - - jars - true - false - runtime - false - - org.apache.spark:spark-assembly_${scala.binary.version}:pom - org.spark-project.spark:unused - - - - diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-init/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-init/Dockerfile new file mode 100644 index 0000000000000..59029a6c08b4a --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-init/Dockerfile @@ -0,0 +1,38 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FROM openjdk:8-alpine + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . + +RUN apk upgrade --update +RUN apk add --update bash +RUN mkdir -p /opt/spark +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark + +ENTRYPOINT [ "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.v2.KubernetesSparkDependencyDownloadInitContainer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-v2/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-v2/Dockerfile new file mode 100644 index 0000000000000..40f9459dc06dc --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/driver-v2/Dockerfile @@ -0,0 +1,43 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FROM openjdk:8-alpine + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-driver:latest -f dockerfiles/driver/Dockerfile . + +RUN apk upgrade --update +RUN apk add --update bash +RUN mkdir -p /opt/spark +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD examples /opt/spark/examples +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark + +CMD SPARK_CLASSPATH="${SPARK_HOME}/jars/*" && \ + if ! [ -z ${SPARK_MOUNTED_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_MOUNTED_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_SUBMIT_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_SUBMIT_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + if ! [ -z ${SPARK_EXTRA_CLASSPATH+x} ]; then SPARK_CLASSPATH="$SPARK_EXTRA_CLASSPATH:$SPARK_CLASSPATH"; fi && \ + exec ${JAVA_HOME}/bin/java $SPARK_DRIVER_JAVA_OPTS -cp $SPARK_CLASSPATH -Xms$SPARK_DRIVER_MEMORY -Xmx$SPARK_DRIVER_MEMORY $SPARK_DRIVER_CLASS $SPARK_DRIVER_ARGS diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile new file mode 100644 index 0000000000000..15e1ce75815df --- /dev/null +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile @@ -0,0 +1,38 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +FROM openjdk:8-alpine + +# If this docker file is being used in the context of building your images from a Spark distribution, the docker build +# command should be invoked from the top level directory of the Spark distribution. E.g.: +# docker build -t spark-executor:latest -f dockerfiles/executor/Dockerfile . + +RUN apk upgrade --update +RUN apk add --update bash +RUN mkdir -p /opt/spark +RUN touch /opt/spark/RELEASE + +ADD jars /opt/spark/jars +ADD bin /opt/spark/bin +ADD sbin /opt/spark/sbin +ADD conf /opt/spark/conf + +ENV SPARK_HOME /opt/spark + +WORKDIR /opt/spark + +ENTRYPOINT [ "bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.v2.ResourceStagingServer" ] diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 5418afa25ca85..ac7a549c9b483 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -33,7 +33,11 @@ org.apache.spark spark-kubernetes_${scala.binary.version} ${project.version} - test + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} org.apache.spark @@ -66,7 +70,7 @@ spark-docker-minimal-bundle_${scala.binary.version} ${project.version} tar.gz - driver-docker-dist + docker-dist test @@ -147,7 +151,7 @@ - copy-test-spark-jobs-to-docker-driver + copy-test-spark-jobs-to-docker-dist pre-integration-test copy @@ -159,65 +163,20 @@ spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} ${project.version} jar - ${project.build.directory}/docker/driver/examples/integration-tests-jars + ${project.build.directory}/docker/examples/integration-tests-jars org.apache.spark spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} ${project.version} jar - ${project.build.directory}/docker/driver/examples/integration-tests-jars - - - - - - copy-test-spark-jobs-to-docker-executor - pre-integration-test - - copy - - - - - org.apache.spark - spark-kubernetes-integration-tests-spark-jobs_${scala.binary.version} - ${project.version} - jar - ${project.build.directory}/docker/executor/examples/integration-tests-jars - - - org.apache.spark - spark-kubernetes-integration-tests-spark-jobs-helpers_${scala.binary.version} - ${project.version} - jar - ${project.build.directory}/docker/executor/examples/integration-tests-jars - - - - - - unpack-docker-driver-bundle - pre-integration-test - - unpack - - - - - org.apache.spark - spark-docker-minimal-bundle_${scala.binary.version} - ${project.version} - driver-docker-dist - tar.gz - true - ${project.build.directory}/docker/driver + ${project.build.directory}/docker/examples/integration-tests-jars - unpack-docker-executor-bundle + unpack-docker-bundle pre-integration-test unpack @@ -228,10 +187,10 @@ org.apache.spark spark-docker-minimal-bundle_${scala.binary.version} ${project.version} - executor-docker-dist + docker-dist tar.gz true - ${project.build.directory}/docker/executor + ${project.build.directory}/docker/ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala index 750e7668b9912..abbf7e4d5ce1b 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala @@ -16,119 +16,23 @@ */ package org.apache.spark.deploy.kubernetes.integrationtest -import java.io.File import java.nio.file.Paths -import java.util.UUID -import java.util.concurrent.TimeUnit import com.google.common.base.Charsets -import com.google.common.collect.ImmutableList import com.google.common.io.Files -import com.google.common.util.concurrent.SettableFuture -import io.fabric8.kubernetes.api.model.Pod -import io.fabric8.kubernetes.client.{Config, KubernetesClient, KubernetesClientException, Watcher} -import io.fabric8.kubernetes.client.Watcher.Action -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.Suite +import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.{Minutes, Seconds, Span} -import scala.collection.JavaConverters._ -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -import org.apache.spark.deploy.SparkSubmit -import org.apache.spark.deploy.kubernetes.SSLUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube -import org.apache.spark.deploy.kubernetes.integrationtest.restapis.SparkRestApiV1 -import org.apache.spark.deploy.kubernetes.submit.v1.{Client, ExternalSuppliedUrisDriverServiceManager} -import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} -import org.apache.spark.util.Utils -private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { - - private val EXAMPLES_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs") - .toFile - .listFiles()(0) - - private val HELPER_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs-helpers") - .toFile - .listFiles()(0) - private val SUBMITTER_LOCAL_MAIN_APP_RESOURCE = s"file://${EXAMPLES_JAR_FILE.getAbsolutePath}" - private val CONTAINER_LOCAL_MAIN_APP_RESOURCE = s"local:///opt/spark/examples/" + - s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" - private val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + - s"integration-tests-jars/${HELPER_JAR_FILE.getName}" - - private val TEST_EXISTENCE_FILE = Paths.get("test-data", "input.txt").toFile - private val TEST_EXISTENCE_FILE_CONTENTS = Files.toString(TEST_EXISTENCE_FILE, Charsets.UTF_8) - private val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) - private val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) - private val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + - ".integrationtest.jobs.SparkPiWithInfiniteWait" - private val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + - ".integrationtest.jobs.FileExistenceTest" - private val NAMESPACE = UUID.randomUUID().toString.replaceAll("-", "") - private var minikubeKubernetesClient: KubernetesClient = _ - private var clientConfig: Config = _ - private var sparkConf: SparkConf = _ +private[spark] class KubernetesSuite extends SparkFunSuite { override def beforeAll(): Unit = { Minikube.startMinikube() new SparkDockerImageBuilder(Minikube.getDockerEnv).buildSparkDockerImages() - Minikube.getKubernetesClient.namespaces.createNew() - .withNewMetadata() - .withName(NAMESPACE) - .endMetadata() - .done() - minikubeKubernetesClient = Minikube.getKubernetesClient.inNamespace(NAMESPACE) - clientConfig = minikubeKubernetesClient.getConfiguration - } - - before { - Eventually.eventually(TIMEOUT, INTERVAL) { - val podsList = minikubeKubernetesClient.pods().list() - assert(podsList == null - || podsList.getItems == null - || podsList.getItems.isEmpty - ) - val servicesList = minikubeKubernetesClient.services().list() - assert(servicesList == null - || servicesList.getItems == null - || servicesList.getItems.isEmpty) - } - sparkConf = new SparkConf(true) - .setMaster(s"k8s://https://${Minikube.getMinikubeIp}:8443") - .set(KUBERNETES_SUBMIT_CA_CERT_FILE, clientConfig.getCaCertFile) - .set(KUBERNETES_SUBMIT_CLIENT_KEY_FILE, clientConfig.getClientKeyFile) - .set(KUBERNETES_SUBMIT_CLIENT_CERT_FILE, clientConfig.getClientCertFile) - .set(KUBERNETES_NAMESPACE, NAMESPACE) - .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") - .set(EXECUTOR_DOCKER_IMAGE, "spark-executor:latest") - .setJars(Seq(HELPER_JAR_FILE.getAbsolutePath)) - .set("spark.executor.memory", "500m") - .set("spark.executor.cores", "1") - .set("spark.executors.instances", "1") - .set("spark.app.name", "spark-pi") - .set("spark.ui.enabled", "true") - .set("spark.testing", "false") - .set(WAIT_FOR_APP_COMPLETION, false) - } - - after { - val pods = minikubeKubernetesClient.pods().list().getItems.asScala - pods.par.foreach(pod => { - minikubeKubernetesClient - .pods() - .withName(pod.getMetadata.getName) - .withGracePeriod(60) - .delete - }) - // spark-submit sets system properties so we have to clear them - new SparkConf(true) - .getAll.map(_._1) - .filter(_ != "spark.docker.test.persistMinikube") - .foreach { System.clearProperty } } override def afterAll(): Unit = { @@ -137,247 +41,33 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { } } - private def getSparkMetricsService(sparkBaseAppName: String): SparkRestApiV1 = { - val serviceName = minikubeKubernetesClient.services() - .withLabel("spark-app-name", sparkBaseAppName) - .list() - .getItems - .get(0) - .getMetadata - .getName - Minikube.getService[SparkRestApiV1](serviceName, NAMESPACE, "spark-ui-port") - } - - private def expectationsForStaticAllocation(sparkMetricsService: SparkRestApiV1): Unit = { - val apps = Eventually.eventually(TIMEOUT, INTERVAL) { - val result = sparkMetricsService - .getApplications(ImmutableList.of(ApplicationStatus.RUNNING, ApplicationStatus.COMPLETED)) - assert(result.size == 1 - && !result.head.id.equalsIgnoreCase("appid") - && !result.head.id.equalsIgnoreCase("{appId}")) - result - } - Eventually.eventually(TIMEOUT, INTERVAL) { - val result = sparkMetricsService.getExecutors(apps.head.id) - assert(result.size == 2) - assert(result.count(exec => exec.id != "driver") == 1) - result - } - Eventually.eventually(TIMEOUT, INTERVAL) { - val result = sparkMetricsService.getStages( - apps.head.id, Seq(StageStatus.COMPLETE).asJava) - assert(result.size == 1) - result - } - } - - test("Run a simple example") { - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - val sparkMetricsService = getSparkMetricsService("spark-pi") - expectationsForStaticAllocation(sparkMetricsService) + override def nestedSuites: scala.collection.immutable.IndexedSeq[Suite] = { + Vector( + new KubernetesV1Suite, + new KubernetesV2Suite) } +} - test("Run using spark-submit") { - val args = Array( - "--master", s"k8s://https://${Minikube.getMinikubeIp}:8443", - "--deploy-mode", "cluster", - "--kubernetes-namespace", NAMESPACE, - "--name", "spark-pi", - "--executor-memory", "512m", - "--executor-cores", "1", - "--num-executors", "1", - "--jars", HELPER_JAR_FILE.getAbsolutePath, - "--class", SPARK_PI_MAIN_CLASS, - "--conf", "spark.ui.enabled=true", - "--conf", "spark.testing=false", - "--conf", s"${KUBERNETES_SUBMIT_CA_CERT_FILE.key}=${clientConfig.getCaCertFile}", - "--conf", s"${KUBERNETES_SUBMIT_CLIENT_KEY_FILE.key}=${clientConfig.getClientKeyFile}", - "--conf", s"${KUBERNETES_SUBMIT_CLIENT_CERT_FILE.key}=${clientConfig.getClientCertFile}", - "--conf", s"${EXECUTOR_DOCKER_IMAGE.key}=spark-executor:latest", - "--conf", s"${DRIVER_DOCKER_IMAGE.key}=spark-driver:latest", - "--conf", s"${WAIT_FOR_APP_COMPLETION.key}=false", - EXAMPLES_JAR_FILE.getAbsolutePath) - SparkSubmit.main(args) - val sparkMetricsService = getSparkMetricsService("spark-pi") - expectationsForStaticAllocation(sparkMetricsService) - } - - test("Run with the examples jar on the docker image") { - sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = CONTAINER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - val sparkMetricsService = getSparkMetricsService("spark-pi") - expectationsForStaticAllocation(sparkMetricsService) - } - - test("Run with custom labels and annotations") { - sparkConf.set(KUBERNETES_DRIVER_LABELS, "label1=label1value,label2=label2value") - sparkConf.set(KUBERNETES_DRIVER_ANNOTATIONS, "annotation1=annotation1value," + - "annotation2=annotation2value") - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - val driverPodMetadata = minikubeKubernetesClient - .pods - .withLabel("spark-app-name", "spark-pi") - .list() - .getItems - .get(0) - .getMetadata - val driverPodLabels = driverPodMetadata.getLabels - // We can't match all of the selectors directly since one of the selectors is based on the - // launch time. - assert(driverPodLabels.size === 5, "Unexpected number of pod labels.") - assert(driverPodLabels.get("spark-app-name") === "spark-pi", "Unexpected value for" + - " spark-app-name label.") - assert(driverPodLabels.get("spark-app-id").startsWith("spark-pi"), "Unexpected value for" + - " spark-app-id label (should be prefixed with the app name).") - assert(driverPodLabels.get("label1") === "label1value", "Unexpected value for label1") - assert(driverPodLabels.get("label2") === "label2value", "Unexpected value for label2") - val driverPodAnnotations = driverPodMetadata.getAnnotations - assert(driverPodAnnotations.size === 2, "Unexpected number of pod annotations.") - assert(driverPodAnnotations.get("annotation1") === "annotation1value", - "Unexpected value for annotation1") - assert(driverPodAnnotations.get("annotation2") === "annotation2value", - "Unexpected value for annotation2") - } - - test("Enable SSL on the driver submit server") { - val (keyStoreFile, trustStoreFile) = SSLUtils.generateKeyStoreTrustStorePair( - Minikube.getMinikubeIp, - "changeit", - "changeit", - "changeit") - sparkConf.set(KUBERNETES_DRIVER_SUBMIT_SSL_KEYSTORE, s"file://${keyStoreFile.getAbsolutePath}") - sparkConf.set("spark.ssl.kubernetes.driversubmitserver.keyStorePassword", "changeit") - sparkConf.set("spark.ssl.kubernetes.driversubmitserver.keyPassword", "changeit") - sparkConf.set(KUBERNETES_DRIVER_SUBMIT_SSL_TRUSTSTORE, - s"file://${trustStoreFile.getAbsolutePath}") - sparkConf.set("spark.ssl.kubernetes.driversubmitserver.trustStorePassword", "changeit") - sparkConf.set(DRIVER_SUBMIT_SSL_ENABLED, true) - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - } - - test("Enable SSL on the driver submit server using PEM files") { - val (keyPem, certPem) = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp) - sparkConf.set(DRIVER_SUBMIT_SSL_KEY_PEM, s"file://${keyPem.getAbsolutePath}") - sparkConf.set(DRIVER_SUBMIT_SSL_CLIENT_CERT_PEM, s"file://${certPem.getAbsolutePath}") - sparkConf.set(DRIVER_SUBMIT_SSL_SERVER_CERT_PEM, s"file://${certPem.getAbsolutePath}") - sparkConf.set(DRIVER_SUBMIT_SSL_ENABLED, true) - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - } - - test("Added files should exist on the driver.") { - sparkConf.set("spark.files", TEST_EXISTENCE_FILE.getAbsolutePath) - sparkConf.setAppName("spark-file-existence-test") - val podCompletedFuture = SettableFuture.create[Boolean] - val watch = new Watcher[Pod] { - override def eventReceived(action: Action, pod: Pod): Unit = { - val containerStatuses = pod.getStatus.getContainerStatuses.asScala - val allSuccessful = containerStatuses.nonEmpty && containerStatuses - .forall(status => { - status.getState.getTerminated != null && status.getState.getTerminated.getExitCode == 0 - }) - if (allSuccessful) { - podCompletedFuture.set(true) - } else { - val failedContainers = containerStatuses.filter(container => { - container.getState.getTerminated != null && - container.getState.getTerminated.getExitCode != 0 - }) - if (failedContainers.nonEmpty) { - podCompletedFuture.setException(new SparkException( - "One or more containers in the driver failed with a nonzero exit code.")) - } - } - } - - override def onClose(e: KubernetesClientException): Unit = { - logWarning("Watch closed", e) - } - } - Utils.tryWithResource(minikubeKubernetesClient - .pods - .withLabel("spark-app-name", "spark-file-existence-test") - .watch(watch)) { _ => - new Client( - sparkConf = sparkConf, - mainClass = FILE_EXISTENCE_MAIN_CLASS, - mainAppResource = CONTAINER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array(TEST_EXISTENCE_FILE.getName, TEST_EXISTENCE_FILE_CONTENTS)).run() - assert(podCompletedFuture.get(60, TimeUnit.SECONDS), "Failed to run driver pod") - val driverPod = minikubeKubernetesClient - .pods - .withLabel("spark-app-name", "spark-file-existence-test") - .list() - .getItems - .get(0) - val podLog = minikubeKubernetesClient - .pods - .withName(driverPod.getMetadata.getName) - .getLog - assert(podLog.contains(s"File found at /opt/spark/${TEST_EXISTENCE_FILE.getName}" + - s" with correct contents."), "Job did not find the file as expected.") - } - } +private[spark] object KubernetesSuite { + val EXAMPLES_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs") + .toFile + .listFiles()(0) - test("Use external URI provider") { - val externalUriProviderWatch = new ExternalUriProviderWatch(minikubeKubernetesClient) - Utils.tryWithResource(minikubeKubernetesClient.services() - .withLabel("spark-app-name", "spark-pi") - .watch(externalUriProviderWatch)) { _ => - sparkConf.set(DRIVER_SERVICE_MANAGER_TYPE, ExternalSuppliedUrisDriverServiceManager.TYPE) - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - val sparkMetricsService = getSparkMetricsService("spark-pi") - expectationsForStaticAllocation(sparkMetricsService) - assert(externalUriProviderWatch.annotationSet.get) - val driverService = minikubeKubernetesClient - .services() - .withLabel("spark-app-name", "spark-pi") - .list() - .getItems - .asScala(0) - assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_PROVIDE_EXTERNAL_URI), - "External URI request annotation was not set on the driver service.") - // Unfortunately we can't check the correctness of the actual value of the URI, as it depends - // on the driver submission port set on the driver service but we remove that port from the - // service once the submission is complete. - assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_RESOLVED_EXTERNAL_URI), - "Resolved URI annotation not set on driver service.") - } - } + val HELPER_JAR_FILE = Paths.get("target", "integration-tests-spark-jobs-helpers") + .toFile + .listFiles()(0) + val SUBMITTER_LOCAL_MAIN_APP_RESOURCE = s"file://${EXAMPLES_JAR_FILE.getAbsolutePath}" + val CONTAINER_LOCAL_MAIN_APP_RESOURCE = s"local:///opt/spark/examples/" + + s"integration-tests-jars/${EXAMPLES_JAR_FILE.getName}" + val CONTAINER_LOCAL_HELPER_JAR_PATH = s"local:///opt/spark/examples/" + + s"integration-tests-jars/${HELPER_JAR_FILE.getName}" - test("Mount the Kubernetes credentials onto the driver pod") { - sparkConf.set(KUBERNETES_DRIVER_CA_CERT_FILE, clientConfig.getCaCertFile) - sparkConf.set(KUBERNETES_DRIVER_CLIENT_KEY_FILE, clientConfig.getClientKeyFile) - sparkConf.set(KUBERNETES_DRIVER_CLIENT_CERT_FILE, clientConfig.getClientCertFile) - new Client( - sparkConf = sparkConf, - mainClass = SPARK_PI_MAIN_CLASS, - mainAppResource = SUBMITTER_LOCAL_MAIN_APP_RESOURCE, - appArgs = Array.empty[String]).run() - val sparkMetricsService = getSparkMetricsService("spark-pi") - expectationsForStaticAllocation(sparkMetricsService) - } + val TEST_EXISTENCE_FILE = Paths.get("test-data", "input.txt").toFile + val TEST_EXISTENCE_FILE_CONTENTS = Files.toString(TEST_EXISTENCE_FILE, Charsets.UTF_8) + val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) + val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) + val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.SparkPiWithInfiniteWait" + val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + ".integrationtest.jobs.FileExistenceTest" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala new file mode 100644 index 0000000000000..53e02f9e479c1 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.integrationtest + +import java.util.UUID + +import org.scalatest.concurrent.Eventually +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube + +private[spark] class KubernetesTestComponents { + + val namespace = UUID.randomUUID().toString.replaceAll("-", "") + val kubernetesClient = Minikube.getKubernetesClient.inNamespace(namespace) + val clientConfig = kubernetesClient.getConfiguration + + def createNamespace(): Unit = { + Minikube.getKubernetesClient.namespaces.createNew() + .withNewMetadata() + .withName(namespace) + .endMetadata() + .done() + } + + def deleteNamespace(): Unit = { + Minikube.getKubernetesClient.namespaces.withName(namespace).delete() + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val namespaceList = Minikube.getKubernetesClient + .namespaces() + .list() + .getItems() + .asScala + require(!namespaceList.exists(_.getMetadata.getName == namespace)) + } + } + + def newSparkConf(): SparkConf = { + new SparkConf(true) + .setMaster(s"k8s://https://${Minikube.getMinikubeIp}:8443") + .set(KUBERNETES_SUBMIT_CA_CERT_FILE, clientConfig.getCaCertFile) + .set(KUBERNETES_SUBMIT_CLIENT_KEY_FILE, clientConfig.getClientKeyFile) + .set(KUBERNETES_SUBMIT_CLIENT_CERT_FILE, clientConfig.getClientCertFile) + .set(KUBERNETES_NAMESPACE, namespace) + .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") + .set(EXECUTOR_DOCKER_IMAGE, "spark-executor:latest") + .setJars(Seq(KubernetesSuite.HELPER_JAR_FILE.getAbsolutePath)) + .set("spark.executor.memory", "500m") + .set("spark.executor.cores", "1") + .set("spark.executors.instances", "1") + .set("spark.app.name", "spark-pi") + .set("spark.ui.enabled", "true") + .set("spark.testing", "false") + .set(WAIT_FOR_APP_COMPLETION, false) + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV1Suite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV1Suite.scala new file mode 100644 index 0000000000000..a4e3353032b71 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV1Suite.scala @@ -0,0 +1,306 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT 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.collect.ImmutableList +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import org.scalatest.{BeforeAndAfter, DoNotDiscover} +import org.scalatest.concurrent.Eventually +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.integrationtest.minikube.Minikube +import org.apache.spark.deploy.kubernetes.integrationtest.restapis.SparkRestApiV1 +import org.apache.spark.deploy.kubernetes.submit.v1.{Client, ExternalSuppliedUrisDriverServiceManager} +import org.apache.spark.status.api.v1.{ApplicationStatus, StageStatus} +import org.apache.spark.util.Utils + +@DoNotDiscover +private[spark] class KubernetesV1Suite extends SparkFunSuite with BeforeAndAfter { + + private var kubernetesTestComponents: KubernetesTestComponents = _ + private var sparkConf: SparkConf = _ + + override def beforeAll(): Unit = { + kubernetesTestComponents = new KubernetesTestComponents() + kubernetesTestComponents.createNamespace() + } + + override def afterAll(): Unit = { + kubernetesTestComponents.deleteNamespace() + } + + before { + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val podsList = kubernetesTestComponents.kubernetesClient.pods().list() + assert(podsList == null + || podsList.getItems == null + || podsList.getItems.isEmpty + ) + val servicesList = kubernetesTestComponents.kubernetesClient.services().list() + assert(servicesList == null + || servicesList.getItems == null + || servicesList.getItems.isEmpty) + } + sparkConf = kubernetesTestComponents.newSparkConf() + } + + after { + val pods = kubernetesTestComponents.kubernetesClient.pods().list().getItems.asScala + pods.par.foreach(pod => { + kubernetesTestComponents.kubernetesClient.pods() + .withName(pod.getMetadata.getName) + .withGracePeriod(60) + .delete + }) + } + + private def getSparkMetricsService(sparkBaseAppName: String): SparkRestApiV1 = { + val serviceName = kubernetesTestComponents.kubernetesClient.services() + .withLabel("spark-app-name", sparkBaseAppName) + .list() + .getItems + .get(0) + .getMetadata + .getName + Minikube.getService[SparkRestApiV1](serviceName, + kubernetesTestComponents.namespace, "spark-ui-port") + } + + private def expectationsForStaticAllocation(sparkMetricsService: SparkRestApiV1): Unit = { + val apps = Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val result = sparkMetricsService + .getApplications(ImmutableList.of(ApplicationStatus.RUNNING, ApplicationStatus.COMPLETED)) + assert(result.size == 1 + && !result.head.id.equalsIgnoreCase("appid") + && !result.head.id.equalsIgnoreCase("{appId}")) + result + } + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val result = sparkMetricsService.getExecutors(apps.head.id) + assert(result.size == 2) + assert(result.count(exec => exec.id != "driver") == 1) + result + } + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + val result = sparkMetricsService.getStages( + apps.head.id, Seq(StageStatus.COMPLETE).asJava) + assert(result.size == 1) + result + } + } + + test("Run a simple example") { + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run with the examples jar on the docker image") { + sparkConf.setJars(Seq(KubernetesSuite.CONTAINER_LOCAL_HELPER_JAR_PATH)) + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + + test("Run with custom labels and annotations") { + sparkConf.set(KUBERNETES_DRIVER_LABELS, "label1=label1value,label2=label2value") + sparkConf.set(KUBERNETES_DRIVER_ANNOTATIONS, "annotation1=annotation1value," + + "annotation2=annotation2value") + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + val driverPodMetadata = kubernetesTestComponents.kubernetesClient + .pods + .withLabel("spark-app-name", "spark-pi") + .list() + .getItems + .get(0) + .getMetadata + val driverPodLabels = driverPodMetadata.getLabels + // We can't match all of the selectors directly since one of the selectors is based on the + // launch time. + assert(driverPodLabels.size === 5, "Unexpected number of pod labels.") + assert(driverPodLabels.get("spark-app-name") === "spark-pi", "Unexpected value for" + + " spark-app-name label.") + assert(driverPodLabels.get("spark-app-id").startsWith("spark-pi"), "Unexpected value for" + + " spark-app-id label (should be prefixed with the app name).") + assert(driverPodLabels.get("label1") === "label1value", "Unexpected value for label1") + assert(driverPodLabels.get("label2") === "label2value", "Unexpected value for label2") + val driverPodAnnotations = driverPodMetadata.getAnnotations + assert(driverPodAnnotations.size === 2, "Unexpected number of pod annotations.") + assert(driverPodAnnotations.get("annotation1") === "annotation1value", + "Unexpected value for annotation1") + assert(driverPodAnnotations.get("annotation2") === "annotation2value", + "Unexpected value for annotation2") + } + + test("Enable SSL on the driver submit server") { + val (keyStoreFile, trustStoreFile) = SSLUtils.generateKeyStoreTrustStorePair( + Minikube.getMinikubeIp, + "changeit", + "changeit", + "changeit") + sparkConf.set(KUBERNETES_DRIVER_SUBMIT_SSL_KEYSTORE, s"file://${keyStoreFile.getAbsolutePath}") + sparkConf.set("spark.ssl.kubernetes.driversubmitserver.keyStorePassword", "changeit") + sparkConf.set("spark.ssl.kubernetes.driversubmitserver.keyPassword", "changeit") + sparkConf.set(KUBERNETES_DRIVER_SUBMIT_SSL_TRUSTSTORE, + s"file://${trustStoreFile.getAbsolutePath}") + sparkConf.set("spark.ssl.kubernetes.driversubmitserver.trustStorePassword", "changeit") + sparkConf.set(DRIVER_SUBMIT_SSL_ENABLED, true) + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + } + + test("Enable SSL on the driver submit server using PEM files") { + val (keyPem, certPem) = SSLUtils.generateKeyCertPemPair(Minikube.getMinikubeIp) + sparkConf.set(DRIVER_SUBMIT_SSL_KEY_PEM, s"file://${keyPem.getAbsolutePath}") + sparkConf.set(DRIVER_SUBMIT_SSL_CLIENT_CERT_PEM, s"file://${certPem.getAbsolutePath}") + sparkConf.set(DRIVER_SUBMIT_SSL_SERVER_CERT_PEM, s"file://${certPem.getAbsolutePath}") + sparkConf.set(DRIVER_SUBMIT_SSL_ENABLED, true) + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + } + + test("Added files should exist on the driver.") { + sparkConf.set("spark.files", KubernetesSuite.TEST_EXISTENCE_FILE.getAbsolutePath) + sparkConf.setAppName("spark-file-existence-test") + val podCompletedFuture = SettableFuture.create[Boolean] + val watch = new Watcher[Pod] { + override def eventReceived(action: Action, pod: Pod): Unit = { + val containerStatuses = pod.getStatus.getContainerStatuses.asScala + val allSuccessful = containerStatuses.nonEmpty && containerStatuses + .forall(status => { + status.getState.getTerminated != null && status.getState.getTerminated.getExitCode == 0 + }) + if (allSuccessful) { + podCompletedFuture.set(true) + } else { + val failedContainers = containerStatuses.filter(container => { + container.getState.getTerminated != null && + container.getState.getTerminated.getExitCode != 0 + }) + if (failedContainers.nonEmpty) { + podCompletedFuture.setException(new SparkException( + "One or more containers in the driver failed with a nonzero exit code.")) + } + } + } + + override def onClose(e: KubernetesClientException): Unit = { + logWarning("Watch closed", e) + } + } + Utils.tryWithResource(kubernetesTestComponents.kubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .watch(watch)) { _ => + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.FILE_EXISTENCE_MAIN_CLASS, + mainAppResource = KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array(KubernetesSuite.TEST_EXISTENCE_FILE.getName, + KubernetesSuite.TEST_EXISTENCE_FILE_CONTENTS)).run() + assert(podCompletedFuture.get(60, TimeUnit.SECONDS), "Failed to run driver pod") + val driverPod = kubernetesTestComponents.kubernetesClient + .pods + .withLabel("spark-app-name", "spark-file-existence-test") + .list() + .getItems + .get(0) + val podLog = kubernetesTestComponents.kubernetesClient + .pods + .withName(driverPod.getMetadata.getName) + .getLog + assert(podLog.contains(s"File found at" + + s" /opt/spark/${KubernetesSuite.TEST_EXISTENCE_FILE.getName} with correct contents."), + "Job did not find the file as expected.") + } + } + + test("Use external URI provider") { + val externalUriProviderWatch = + new ExternalUriProviderWatch(kubernetesTestComponents.kubernetesClient) + Utils.tryWithResource(kubernetesTestComponents.kubernetesClient.services() + .withLabel("spark-app-name", "spark-pi") + .watch(externalUriProviderWatch)) { _ => + sparkConf.set(DRIVER_SERVICE_MANAGER_TYPE, ExternalSuppliedUrisDriverServiceManager.TYPE) + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + assert(externalUriProviderWatch.annotationSet.get) + val driverService = kubernetesTestComponents.kubernetesClient + .services() + .withLabel("spark-app-name", "spark-pi") + .list() + .getItems + .asScala(0) + assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_PROVIDE_EXTERNAL_URI), + "External URI request annotation was not set on the driver service.") + // Unfortunately we can't check the correctness of the actual value of the URI, as it depends + // on the driver submission port set on the driver service but we remove that port from the + // service once the submission is complete. + assert(driverService.getMetadata.getAnnotations.containsKey(ANNOTATION_RESOLVED_EXTERNAL_URI), + "Resolved URI annotation not set on driver service.") + } + } + + test("Mount the Kubernetes credentials onto the driver pod") { + sparkConf.set(KUBERNETES_DRIVER_CA_CERT_FILE, + kubernetesTestComponents.clientConfig.getCaCertFile) + sparkConf.set(KUBERNETES_DRIVER_CLIENT_KEY_FILE, + kubernetesTestComponents.clientConfig.getClientKeyFile) + sparkConf.set(KUBERNETES_DRIVER_CLIENT_CERT_FILE, + kubernetesTestComponents.clientConfig.getClientCertFile) + new Client( + sparkConf = sparkConf, + mainClass = KubernetesSuite.SPARK_PI_MAIN_CLASS, + mainAppResource = KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE, + appArgs = Array.empty[String]).run() + val sparkMetricsService = getSparkMetricsService("spark-pi") + expectationsForStaticAllocation(sparkMetricsService) + } + +} 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 new file mode 100644 index 0000000000000..0d74067334028 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesV2Suite.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.integrationtest + +import java.util.UUID + +import org.scalatest.{BeforeAndAfter, DoNotDiscover} +import org.scalatest.concurrent.Eventually + +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.minikube.Minikube +import org.apache.spark.deploy.kubernetes.submit.v2.{MountedDependencyManagerProviderImpl, SubmissionKubernetesClientProviderImpl} + +@DoNotDiscover +private[spark] class KubernetesV2Suite extends SparkFunSuite with BeforeAndAfter { + + private val APP_LOCATOR_LABEL = UUID.randomUUID().toString.replaceAll("-", "") + private var kubernetesTestComponents: KubernetesTestComponents = _ + private var sparkConf: SparkConf = _ + private var resourceStagingServerLauncher: ResourceStagingServerLauncher = _ + + override def beforeAll(): Unit = { + kubernetesTestComponents = new KubernetesTestComponents + resourceStagingServerLauncher = new ResourceStagingServerLauncher( + kubernetesTestComponents.kubernetesClient.inNamespace(kubernetesTestComponents.namespace)) + } + + before { + sparkConf = kubernetesTestComponents.newSparkConf() + .set(INIT_CONTAINER_DOCKER_IMAGE, s"spark-driver-init:latest") + .set(DRIVER_DOCKER_IMAGE, s"spark-driver-v2:latest") + .set(KUBERNETES_DRIVER_LABELS, s"spark-app-locator=$APP_LOCATOR_LABEL") + kubernetesTestComponents.createNamespace() + } + + after { + kubernetesTestComponents.deleteNamespace() + } + + test("Use submission v2.") { + launchStagingServer(SSLOptions()) + runSparkAppAndVerifyCompletion(KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Enable SSL on the submission server") { + val (keyStore, trustStore) = SSLUtils.generateKeyStoreTrustStorePair( + ipAddress = Minikube.getMinikubeIp, + keyStorePassword = "keyStore", + keyPassword = "key", + trustStorePassword = "trustStore") + sparkConf.set(RESOURCE_STAGING_SERVER_SSL_ENABLED, true) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStore", keyStore.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.trustStore", trustStore.getAbsolutePath) + .set("spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", "keyStore") + .set("spark.ssl.kubernetes.resourceStagingServer.keyPassword", "key") + .set("spark.ssl.kubernetes.resourceStagingServer.trustStorePassword", "trustStore") + launchStagingServer(SSLOptions( + enabled = true, + keyStore = Some(keyStore), + trustStore = Some(trustStore), + keyStorePassword = Some("keyStore"), + keyPassword = Some("key"), + trustStorePassword = Some("trustStore"))) + runSparkAppAndVerifyCompletion(KubernetesSuite.SUBMITTER_LOCAL_MAIN_APP_RESOURCE) + } + + test("Use container-local resources without the resource staging server") { + sparkConf.setJars(Seq( + KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE, + KubernetesSuite.CONTAINER_LOCAL_HELPER_JAR_PATH)) + runSparkAppAndVerifyCompletion(KubernetesSuite.CONTAINER_LOCAL_MAIN_APP_RESOURCE) + } + + private def launchStagingServer(resourceStagingServerSslOptions: SSLOptions): Unit = { + val resourceStagingServerPort = resourceStagingServerLauncher.launchStagingServer( + resourceStagingServerSslOptions) + val resourceStagingServerUriScheme = if (resourceStagingServerSslOptions.enabled) { + "https" + } else { + "http" + } + sparkConf.set(RESOURCE_STAGING_SERVER_URI, + s"$resourceStagingServerUriScheme://${Minikube.getMinikubeIp}:$resourceStagingServerPort") + } + + private def runSparkAppAndVerifyCompletion(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() + val driverPod = kubernetesTestComponents.kubernetesClient + .pods() + .withLabel("spark-app-locator", APP_LOCATOR_LABEL) + .list() + .getItems + .get(0) + Eventually.eventually(KubernetesSuite.TIMEOUT, KubernetesSuite.INTERVAL) { + assert(kubernetesTestComponents.kubernetesClient + .pods() + .withName(driverPod.getMetadata.getName) + .getLog + .contains("Pi is roughly 3"), "The application did not compute the value of pi.") + } + } +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala new file mode 100644 index 0000000000000..ca549fa27d630 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.integrationtest + +import java.io.StringWriter +import java.util.Properties +import java.util.concurrent.TimeUnit + +import com.google.common.io.{BaseEncoding, Files} +import com.google.common.util.concurrent.SettableFuture +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, Endpoints, HasMetadata, HTTPGetActionBuilder, KeyToPathBuilder, Pod, PodBuilder, SecretBuilder, ServiceBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.internal.readiness.Readiness +import scala.collection.JavaConverters._ + +import org.apache.spark.SSLOptions +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.submit.v2.ContainerNameEqualityPredicate +import org.apache.spark.util.Utils + +/** + * Launches a pod that runs the resource staging server, exposing it over a NodePort. + */ +private[spark] class ResourceStagingServerLauncher(kubernetesClient: KubernetesClient) { + + private val KEYSTORE_DIR = "/mnt/secrets/spark-staging" + private val KEYSTORE_FILE = s"$KEYSTORE_DIR/keyStore" + 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 = { + val stagingServerProperties = new Properties() + val stagingServerSecret = sslOptions.keyStore.map { keyStore => + val keyStoreBytes = Files.toByteArray(keyStore) + val keyStoreBase64 = BaseEncoding.base64().encode(keyStoreBytes) + new SecretBuilder() + .withNewMetadata() + .withName("resource-staging-server-keystore") + .endMetadata() + .addToData("keyStore", keyStoreBase64) + .build() + } + stagingServerProperties.setProperty( + RESOURCE_STAGING_SERVER_SSL_ENABLED.key, sslOptions.enabled.toString) + sslOptions.keyStorePassword.foreach { password => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyStorePassword", password) + } + sslOptions.keyPassword.foreach { password => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyPassword", password) + } + stagingServerSecret.foreach { _ => + stagingServerProperties.setProperty( + "spark.ssl.kubernetes.resourceStagingServer.keyStore", KEYSTORE_FILE) + } + val propertiesWriter = new StringWriter() + stagingServerProperties.store(propertiesWriter, "Resource staging server properties.") + val stagingServerConfigMap = new ConfigMapBuilder() + .withNewMetadata() + .withName(s"staging-server-properties") + .endMetadata() + .addToData("staging-server", propertiesWriter.toString) + .build() + val probePingHttpGet = new HTTPGetActionBuilder() + .withScheme(if (sslOptions.enabled) "HTTPS" else "HTTP") + .withPath("/api/v0/ping") + .withNewPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .build() + val basePod = new PodBuilder() + .withNewMetadata() + .withName("resource-staging-server") + .addToLabels("resource-staging-server", "staging-server") + .endMetadata() + .withNewSpec() + .addNewVolume() + .withName("staging-server-properties") + .withNewConfigMap() + .withName(stagingServerConfigMap.getMetadata.getName) + .withItems( + new KeyToPathBuilder() + .withKey("staging-server") + .withPath(PROPERTIES_FILE_NAME) + .build()) + .endConfigMap() + .endVolume() + .addNewContainer() + .withName("staging-server-container") + .withImage("spark-resource-staging-server:latest") + .withImagePullPolicy("IfNotPresent") + .withNewReadinessProbe() + .withHttpGet(probePingHttpGet) + .endReadinessProbe() + .addNewVolumeMount() + .withName("staging-server-properties") + .withMountPath(PROPERTIES_DIR) + .endVolumeMount() + .addToArgs(PROPERTIES_FILE_PATH) + .endContainer() + .endSpec() + val withMountedKeyStorePod = stagingServerSecret.map { secret => + basePod.editSpec() + .addNewVolume() + .withName("keystore-volume") + .withNewSecret() + .withSecretName(secret.getMetadata.getName) + .endSecret() + .endVolume() + .editMatchingContainer(new ContainerNameEqualityPredicate("staging-server-container")) + .addNewVolumeMount() + .withName("keystore-volume") + .withMountPath(KEYSTORE_DIR) + .endVolumeMount() + .endContainer() + .endSpec() + }.getOrElse(basePod).build() + val stagingServerService = new ServiceBuilder() + .withNewMetadata() + .withName("resource-staging-server") + .endMetadata() + .withNewSpec() + .withType("NodePort") + .addToSelector("resource-staging-server", "staging-server") + .addNewPort() + .withName("staging-server-port") + .withPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .withNewTargetPort(RESOURCE_STAGING_SERVER_PORT.defaultValue.get) + .endPort() + .endSpec() + .build() + val stagingServerPodReadyWatcher = new ReadinessWatcher[Pod] + val serviceReadyWatcher = new ReadinessWatcher[Endpoints] + val allResources = Seq( + stagingServerService, + stagingServerConfigMap, + withMountedKeyStorePod) ++ + stagingServerSecret.toSeq + Utils.tryWithResource(kubernetesClient.pods() + .withName(withMountedKeyStorePod.getMetadata.getName) + .watch(stagingServerPodReadyWatcher)) { _ => + Utils.tryWithResource(kubernetesClient.endpoints() + .withName(stagingServerService.getMetadata.getName) + .watch(serviceReadyWatcher)) { _ => + activeResources = kubernetesClient.resourceList(allResources: _*) + .createOrReplace() + .asScala + stagingServerPodReadyWatcher.waitUntilReady() + serviceReadyWatcher.waitUntilReady() + } + } + kubernetesClient.services().withName(stagingServerService.getMetadata.getName).get() + .getSpec + .getPorts + .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/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala index 1aa6a7b7e70c2..d807c4d81009b 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 @@ -19,14 +19,20 @@ package org.apache.spark.deploy.kubernetes.integrationtest.docker import java.net.URI import java.nio.file.Paths -import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider -import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates} +import com.spotify.docker.client.{DefaultDockerClient, DockerCertificates, LoggingBuildHandler} import org.apache.http.client.utils.URIBuilder import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, String]) { + private val DOCKER_BUILD_PATH = Paths.get("target", "docker") + // Dockerfile paths must be relative to the build path. + private val DRIVER_V1_DOCKER_FILE = "dockerfiles/driver/Dockerfile" + private val DRIVER_V2_DOCKER_FILE = "dockerfiles/driver-v2/Dockerfile" + private val EXECUTOR_DOCKER_FILE = "dockerfiles/executor/Dockerfile" + private val DRIVER_INIT_DOCKER_FILE = "dockerfiles/driver-init/Dockerfile" + private val STAGING_SERVER_DOCKER_FILE = "dockerfiles/resource-staging-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", @@ -52,7 +58,18 @@ private[spark] class SparkDockerImageBuilder(private val dockerEnv: Map[String, def buildSparkDockerImages(): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { dockerClient.ping() } - dockerClient.build(Paths.get("target", "docker", "driver"), "spark-driver") - dockerClient.build(Paths.get("target", "docker", "executor"), "spark-executor") + buildImage("spark-driver", DRIVER_V1_DOCKER_FILE) + buildImage("spark-executor", EXECUTOR_DOCKER_FILE) + 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) + } + + private def buildImage(name: String, dockerFile: String): Unit = { + dockerClient.build( + DOCKER_BUILD_PATH, + name, + dockerFile, + new LoggingBuildHandler()) } }