From fb5b9ed83d4e5ed73bc44b9d719ac0e52702655e Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 15 Apr 2018 23:23:43 -0400 Subject: [PATCH 01/16] initial architecture for PySpark w/o dockerfile work --- .../org/apache/spark/deploy/SparkSubmit.scala | 12 ++-- .../org/apache/spark/deploy/k8s/Config.scala | 22 ++++++ .../apache/spark/deploy/k8s/Constants.scala | 5 +- .../spark/deploy/k8s/KubernetesConf.scala | 54 +++++++++++---- .../spark/deploy/k8s/KubernetesUtils.scala | 2 +- .../k8s/features/BasicDriverFeatureStep.scala | 5 +- .../features/BasicExecutorFeatureStep.scala | 2 +- .../bindings/PythonDriverFeatureStep.scala | 49 +++++++++++++ .../submit/KubernetesClientApplication.scala | 16 ++++- .../k8s/submit/KubernetesDriverBuilder.scala | 22 ++++-- .../deploy/k8s/submit/MainAppResource.scala | 5 ++ .../k8s/KubernetesExecutorBuilder.scala | 9 ++- .../deploy/k8s/KubernetesConfSuite.scala | 39 +++++++++-- .../BasicDriverFeatureStepSuite.scala | 8 ++- .../BasicExecutorFeatureStepSuite.scala | 9 ++- ...ubernetesCredentialsFeatureStepSuite.scala | 9 ++- .../DriverServiceFeatureStepSuite.scala | 18 +++-- .../MountSecretsFeatureStepSuite.scala | 3 +- .../PythonDriverFeatureStepSuite.scala | 68 +++++++++++++++++++ .../spark/deploy/k8s/submit/ClientSuite.scala | 3 +- .../submit/KubernetesDriverBuilderSuite.scala | 37 +++++++++- .../k8s/KubernetesExecutorBuilderSuite.scala | 6 +- 22 files changed, 342 insertions(+), 61 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 427c797755b84..4820224b45150 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -284,8 +284,6 @@ private[spark] class SparkSubmit extends Logging { case (STANDALONE, CLUSTER) if args.isR => error("Cluster deploy mode is currently not supported for R " + "applications on standalone clusters.") - case (KUBERNETES, _) if args.isPython => - error("Python applications are currently not supported for Kubernetes.") case (KUBERNETES, _) if args.isR => error("R applications are currently not supported for Kubernetes.") case (LOCAL, CLUSTER) => @@ -695,9 +693,15 @@ private[spark] class SparkSubmit extends Logging { if (isKubernetesCluster) { childMainClass = KUBERNETES_CLUSTER_SUBMIT_CLASS if (args.primaryResource != SparkLauncher.NO_RESOURCE) { - childArgs ++= Array("--primary-java-resource", args.primaryResource) + if (args.isPython) { + childArgs ++= Array("--primary-py-file", args.primaryResource) + childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") + childArgs ++= Array("--other-py-files", args.pyFiles) + } else { + childArgs ++= Array("--primary-java-resource", args.primaryResource) + childArgs ++= Array("--main-class", args.mainClass) + } } - childArgs ++= Array("--main-class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 4086970ffb256..1128267eb8f50 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -117,6 +117,21 @@ private[spark] object Config extends Logging { .stringConf .createWithDefault("spark") + val KUBERNETES_PYSPARK_PY_FILES = + ConfigBuilder("spark.kubernetes.python.pyFiles") + .doc("The PyFiles that are distributed via client arguments") + .internal() + .stringConf + .createOptional + + val KUBERNETES_PYSPARK_MAIN_APP_RESOURCE = + ConfigBuilder("spark.kubernetes.python.mainAppResource") + .doc("The main app resource for pyspark jobs") + .internal() + .stringConf + .createOptional + + val KUBERNETES_ALLOCATION_BATCH_SIZE = ConfigBuilder("spark.kubernetes.allocation.batch.size") .doc("Number of pods to launch at once in each round of executor allocation.") @@ -154,6 +169,13 @@ private[spark] object Config extends Logging { .checkValue(interval => interval > 0, s"Logging interval must be a positive time value.") .createWithDefaultString("1s") + val MEMORY_OVERHEAD_FACTOR = + ConfigBuilder("spark.kubernetes.memoryOverheadFactor") + .doc("This sets the Memory Overhead Factor that will allocate memory to non-JVM jobs " + + "which in the case of JVM tasks will default to 0.10 and 0.40 for non-JVM jobs") + .doubleConf + .createWithDefault(0.10) + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 8da5f24044aad..1ea5c481e0cea 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -71,9 +71,12 @@ private[spark] object Constants { val SPARK_CONF_FILE_NAME = "spark.properties" val SPARK_CONF_PATH = s"$SPARK_CONF_DIR_INTERNAL/$SPARK_CONF_FILE_NAME" + // BINDINGS + val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" + val ENV_PYSPARK_FILES = "PYSPARK_FILES" + // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" - val MEMORY_OVERHEAD_FACTOR = 0.10 val MEMORY_OVERHEAD_MIN_MIB = 384L } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 77b634ddfabcc..2e50f44039b9e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -16,14 +16,17 @@ */ package org.apache.spark.deploy.k8s +import scala.collection.mutable + import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferenceBuilder, Pod} import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.{JavaMainAppResource, MainAppResource} +import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config.ConfigEntry + private[spark] sealed trait KubernetesRoleSpecificConf /* @@ -54,7 +57,8 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( roleLabels: Map[String, String], roleAnnotations: Map[String, String], roleSecretNamesToMountPaths: Map[String, String], - roleEnvs: Map[String, String]) { + roleEnvs: Map[String, String], + sparkFiles: Seq[String]) { def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) @@ -63,11 +67,14 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) - def sparkFiles(): Seq[String] = sparkConf - .getOption("spark.files") + def pyFiles(): Seq[String] = sparkConf + .get(KUBERNETES_PYSPARK_PY_FILES) .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) + def pySparkMainResource(): Option[String] = sparkConf + .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) + def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) def imagePullSecrets(): Seq[LocalObjectReference] = { @@ -101,17 +108,27 @@ private[spark] object KubernetesConf { appId: String, mainAppResource: Option[MainAppResource], mainClass: String, - appArgs: Array[String]): KubernetesConf[KubernetesDriverSpecificConf] = { + appArgs: Array[String], + maybePyFiles: Seq[String]): KubernetesConf[KubernetesDriverSpecificConf] = { val sparkConfWithMainAppJar = sparkConf.clone() + val additionalFiles = mutable.ArrayBuffer.empty[String] mainAppResource.foreach { - case JavaMainAppResource(res) => - val previousJars = sparkConf - .getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty) - if (!previousJars.contains(res)) { - sparkConfWithMainAppJar.setJars(previousJars ++ Seq(res)) - } + case JavaMainAppResource(res) => + val previousJars = sparkConf + .getOption("spark.jars") + .map(_.split(",")) + .getOrElse(Array.empty) + if (!previousJars.contains(res)) { + sparkConfWithMainAppJar.setJars(previousJars ++ Seq(res)) + } + case nonJVM: NonJVMResource => + nonJVM match { + case PythonMainAppResource(res) => + additionalFiles += res + additionalFiles.appendAll(maybePyFiles) + sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) + } + sparkConfWithMainAppJar.set(MEMORY_OVERHEAD_FACTOR, 0.4) } val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( @@ -132,6 +149,11 @@ private[spark] object KubernetesConf { val driverEnvs = KubernetesUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) + val sparkFiles = sparkConf + .getOption("spark.files") + .map(str => str.split(",").toSeq) + .getOrElse(Seq.empty[String]) ++ additionalFiles + KubernetesConf( sparkConfWithMainAppJar, KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs), @@ -140,7 +162,8 @@ private[spark] object KubernetesConf { driverLabels, driverAnnotations, driverSecretNamesToMountPaths, - driverEnvs) + driverEnvs, + sparkFiles) } def createExecutorConf( @@ -179,6 +202,7 @@ private[spark] object KubernetesConf { executorLabels, executorAnnotations, executorSecrets, - executorEnv) + executorEnv, + Seq.empty[String]) } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index ee629068ad90d..593fb531a004d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -52,7 +52,7 @@ private[spark] object KubernetesUtils { } } - private def resolveFileUri(uri: String): String = { + def resolveFileUri(uri: String): String = { val fileUri = Utils.resolveURI(uri) val fileScheme = Option(fileUri.getScheme).getOrElse("file") fileScheme match { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 07bdccbe0479d..5102f17151d9f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -48,7 +48,8 @@ private[spark] class BasicDriverFeatureStep( private val driverMemoryMiB = conf.get(DRIVER_MEMORY) private val memoryOverheadMiB = conf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) + .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR) * driverMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configurePod(pod: SparkPod): SparkPod = { @@ -122,7 +123,7 @@ private[spark] class BasicDriverFeatureStep( val resolvedSparkJars = KubernetesUtils.resolveFileUrisAndPath( conf.sparkJars()) val resolvedSparkFiles = KubernetesUtils.resolveFileUrisAndPath( - conf.sparkFiles()) + conf.sparkFiles) if (resolvedSparkJars.nonEmpty) { additionalProps.put("spark.jars", resolvedSparkJars.mkString(",")) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index d22097587aafe..79d467eadd30f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -54,7 +54,7 @@ private[spark] class BasicExecutorFeatureStep( private val memoryOverheadMiB = kubernetesConf .get(EXECUTOR_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, + .getOrElse(math.max((kubernetesConf.get(MEMORY_OVERHEAD_FACTOR) * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala new file mode 100644 index 0000000000000..79cf0ed84de40 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.bindings + +import io.fabric8.kubernetes.api.model.ContainerBuilder +import io.fabric8.kubernetes.api.model.HasMetadata + +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants.{ENV_PYSPARK_FILES, ENV_PYSPARK_PRIMARY} +import org.apache.spark.deploy.k8s.KubernetesUtils +import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep + +private[spark] class PythonDriverFeatureStep( + kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + extends KubernetesFeatureConfigStep { + override def configurePod(pod: SparkPod): SparkPod = { + val mainResource = kubernetesConf.pySparkMainResource() + require(mainResource.isDefined, "PySpark Main Resource must be defined") + val otherPyFiles = kubernetesConf.pyFiles() + val withPythonPrimaryFileContainer = new ContainerBuilder(pod.container) + .addNewEnv() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(KubernetesUtils.resolveFileUri(mainResource.get)) + .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_FILES) + .withValue(KubernetesUtils.resolveFileUrisAndPath(otherPyFiles).mkString(",")) + .endEnv() + .build() + SparkPod(pod.pod, withPythonPrimaryFileContainer) + } + override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index a97f5650fb869..2fc217e8be6e7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -39,11 +39,13 @@ import org.apache.spark.util.Utils * @param mainAppResource the main application resource if any * @param mainClass the main class of the application to run * @param driverArgs arguments to the driver + * @param maybePyFiles additional Python files via --py-files */ private[spark] case class ClientArguments( mainAppResource: Option[MainAppResource], mainClass: String, - driverArgs: Array[String]) + driverArgs: Array[String], + maybePyFiles: Seq[String]) private[spark] object ClientArguments { @@ -51,10 +53,15 @@ private[spark] object ClientArguments { var mainAppResource: Option[MainAppResource] = None var mainClass: Option[String] = None val driverArgs = mutable.ArrayBuffer.empty[String] + var maybePyFiles : Seq[String] = Seq.empty[String] args.sliding(2, 2).toList.foreach { case Array("--primary-java-resource", primaryJavaResource: String) => mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) + case Array("--primary-py-file", primaryPythonResource: String) => + mainAppResource = Some(PythonMainAppResource(primaryPythonResource)) + case Array("--other-py-files", pyFiles: String) => + maybePyFiles = pyFiles.split(",") case Array("--main-class", clazz: String) => mainClass = Some(clazz) case Array("--arg", arg: String) => @@ -69,7 +76,8 @@ private[spark] object ClientArguments { ClientArguments( mainAppResource, mainClass.get, - driverArgs.toArray) + driverArgs.toArray, + maybePyFiles) } } @@ -206,6 +214,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val kubernetesResourceNamePrefix = { s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") } + sparkConf.set("spark.kubernetes.python.pyFiles", clientArguments.maybePyFiles.mkString(",")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, appName, @@ -213,7 +222,8 @@ private[spark] class KubernetesClientApplication extends SparkApplication { kubernetesAppId, clientArguments.mainAppResource, clientArguments.mainClass, - clientArguments.driverArgs) + clientArguments.driverArgs, + clientArguments.maybePyFiles) val builder = new KubernetesDriverBuilder val namespace = kubernetesConf.namespace() // The master URL has been checked for validity already in SparkSubmit. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index c7579ed8cb689..1a0e70299d6ad 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -18,6 +18,8 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf} import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep +import org.apache.spark.deploy.k8s.features.bindings.PythonDriverFeatureStep private[spark] class KubernetesDriverBuilder( provideBasicStep: (KubernetesConf[KubernetesDriverSpecificConf]) => BasicDriverFeatureStep = @@ -29,7 +31,11 @@ private[spark] class KubernetesDriverBuilder( new DriverServiceFeatureStep(_), provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountSecretsFeatureStep) = - new MountSecretsFeatureStep(_)) { + new MountSecretsFeatureStep(_), + providePythonStep: ( + KubernetesConf[_ <: KubernetesRoleSpecificConf] + => PythonDriverFeatureStep) = + new PythonDriverFeatureStep(_)) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { @@ -37,10 +43,16 @@ private[spark] class KubernetesDriverBuilder( provideBasicStep(kubernetesConf), provideCredentialsStep(kubernetesConf), provideServiceStep(kubernetesConf)) - val allFeatures = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { - baseFeatures ++ Seq(provideSecretsStep(kubernetesConf)) - } else baseFeatures - + val maybeRoleSecretNamesStep = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + Some(provideSecretsStep(kubernetesConf)) } else None + val maybeNonJVMBindings = kubernetesConf.roleSpecificConf.mainAppResource.map { + case PythonMainAppResource(_) => + providePythonStep(kubernetesConf) + } + val allFeatures: Seq[KubernetesFeatureConfigStep] = + baseFeatures ++ + maybeRoleSecretNamesStep.toSeq ++ + maybeNonJVMBindings.toSeq var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { val configuredPod = feature.configurePod(spec.pod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala index cca9f4627a1f6..cbe081ae35683 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -18,4 +18,9 @@ package org.apache.spark.deploy.k8s.submit private[spark] sealed trait MainAppResource +private[spark] sealed trait NonJVMResource + private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource + +private[spark] case class PythonMainAppResource(primaryResource: String) + extends MainAppResource with NonJVMResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 22568fe7ea3be..837fb503f63ba 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster.k8s import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep private[spark] class KubernetesExecutorBuilder( provideBasicStep: (KubernetesConf[KubernetesExecutorSpecificConf]) => BasicExecutorFeatureStep = @@ -29,9 +30,11 @@ private[spark] class KubernetesExecutorBuilder( def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { val baseFeatures = Seq(provideBasicStep(kubernetesConf)) - val allFeatures = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { - baseFeatures ++ Seq(provideSecretsStep(kubernetesConf)) - } else baseFeatures + val maybeRoleSecretNamesStep = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { + Some(provideSecretsStep(kubernetesConf)) } else None + val allFeatures: Seq[KubernetesFeatureConfigStep] = + baseFeatures ++ + maybeRoleSecretNamesStep.toSeq var executorPod = SparkPod.initialPod() for (feature <- allFeatures) { executorPod = feature.configurePod(executorPod) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index f10202f7a3546..dbac40f408abf 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{LocalObjectReferenceBuilder, PodBuilder} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.deploy.k8s.submit._ class KubernetesConfSuite extends SparkFunSuite { @@ -55,7 +55,8 @@ class KubernetesConfSuite extends SparkFunSuite { APP_ID, None, MAIN_CLASS, - APP_ARGS) + APP_ARGS, + Seq.empty[String]) assert(conf.appId === APP_ID) assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) @@ -76,7 +77,8 @@ class KubernetesConfSuite extends SparkFunSuite { APP_ID, mainAppJar, MAIN_CLASS, - APP_ARGS) + APP_ARGS, + Seq.empty[String]) assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") .split(",") === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) @@ -87,11 +89,37 @@ class KubernetesConfSuite extends SparkFunSuite { APP_ID, None, MAIN_CLASS, - APP_ARGS) + APP_ARGS, + Seq.empty[String]) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) + assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) } + test("Creating driver conf with a python primary file") { + val mainResourceFile = "local:///opt/spark/main.py" + val inputPyFiles = Array("local:///opt/spark/example2.py", "local:///example3.py") + val sparkConf = new SparkConf(false) + .setJars(Seq("local:///opt/spark/jar1.jar")) + .set("spark.files", "local:///opt/spark/example4.py") + val mainAppResource = Some(PythonMainAppResource(mainResourceFile)) + val kubernetesConfWithMainResource = KubernetesConf.createDriverConf( + sparkConf, + APP_NAME, + RESOURCE_NAME_PREFIX, + APP_ID, + mainAppResource, + MAIN_CLASS, + APP_ARGS, + inputPyFiles) + assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") + === Array("local:///opt/spark/jar1.jar")) + assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) + assert(kubernetesConfWithMainResource.sparkFiles + === Array("local:///opt/spark/example4.py", mainResourceFile) ++ inputPyFiles) + } + + test("Resolve driver labels, annotations, secret mount paths, and envs.") { val sparkConf = new SparkConf(false) CUSTOM_LABELS.foreach { case (key, value) => @@ -114,7 +142,8 @@ class KubernetesConfSuite extends SparkFunSuite { APP_ID, None, MAIN_CLASS, - APP_ARGS) + APP_ARGS, + Seq.empty[String]) assert(conf.roleLabels === Map( SPARK_APP_ID_LABEL -> APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index eee85b8baa730..2cae4ba7c95d9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -33,6 +33,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent" private val APP_NAME = "spark-test" private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" + private val PYTHON_MAIN_CLASS = "example.py" + private val EXAMPLE_PYTHON_FILES = Seq("example2.py", "example3.py") private val APP_ARGS = Array("arg1", "arg2", "\"arg 3\"") private val CUSTOM_ANNOTATION_KEY = "customAnnotation" private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" @@ -69,7 +71,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_LABELS, DRIVER_ANNOTATIONS, Map.empty, - DRIVER_ENVS) + DRIVER_ENVS, + Seq.empty[String]) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() @@ -138,7 +141,8 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_LABELS, DRIVER_ANNOTATIONS, Map.empty, - Map.empty) + Map.empty, + allFiles) val step = new BasicDriverFeatureStep(kubernetesConf) val additionalProperties = step.getAdditionalPodSystemProperties() val expectedSparkConf = Map( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index a764f7630b5c8..e1a6b7a105b2b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -87,7 +87,8 @@ class BasicExecutorFeatureStepSuite LABELS, ANNOTATIONS, Map.empty, - Map.empty)) + Map.empty, + Seq.empty[String])) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. @@ -124,7 +125,8 @@ class BasicExecutorFeatureStepSuite LABELS, ANNOTATIONS, Map.empty, - Map.empty)) + Map.empty, + Seq.empty[String])) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } @@ -142,7 +144,8 @@ class BasicExecutorFeatureStepSuite LABELS, ANNOTATIONS, Map.empty, - Map("qux" -> "quux"))) + Map("qux" -> "quux"), + Seq.empty[String])) val executor = step.configurePod(SparkPod.initialPod()) checkEnv(executor, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala index 9f817d3bfc79a..775d78ecf649d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala @@ -59,7 +59,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -88,7 +89,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) @@ -124,7 +126,8 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index c299d56865ec0..498730d5c6d40 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -65,7 +65,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty)) + Map.empty, + Seq.empty[String])) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -94,7 +95,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty)) + Map.empty, + Seq.empty[String])) val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" @@ -113,7 +115,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty)) + Map.empty, + Seq.empty[String])) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -141,7 +144,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty), + Map.empty, + Seq.empty[String]), clock) val driverService = configurationStep .getAdditionalKubernetesResources() @@ -166,7 +170,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty), + Map.empty, + Seq.empty[String]), clock) fail("The driver bind address should not be allowed.") } catch { @@ -189,7 +194,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { DRIVER_LABELS, Map.empty, Map.empty, - Map.empty), + Map.empty, + Seq.empty[String]), clock) fail("The driver host address should not be allowed.") } catch { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index 9d02f56cc206d..847a313b619b5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -41,7 +41,8 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { Map.empty, Map.empty, secretNamesToMountPaths, - Map.empty) + Map.empty, + Seq.empty[String]) val step = new MountSecretsFeatureStep(kubernetesConf) val driverPodWithSecretsMounted = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..4103c4a36cc90 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.bindings + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.submit.PythonMainAppResource +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +class PythonDriverFeatureStepSuite extends SparkFunSuite { + + + test("Python Step modifies container correctly") { + val expectedMainResource = "/main.py" + val mainResource = "local:///main.py" + val pyFiles = Seq("local:///example2.py", "local:///example3.py") + val expectedPySparkFiles = + "/example2.py,/example3.py" + val baseDriverPod = SparkPod.initialPod() + val sparkConf = new SparkConf(false) + .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) + .set(KUBERNETES_PYSPARK_PY_FILES, pyFiles.mkString(",")) + .set("spark.files", "local:///example.py") + val kubernetesConf = KubernetesConf( + sparkConf, + KubernetesDriverSpecificConf( + Some(PythonMainAppResource("local:///main.py")), + "test-app", + "python-runner", + Seq.empty[String]), + "", + "", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String]) + + val step = new PythonDriverFeatureStep(kubernetesConf) + val driverPod = step.configurePod(baseDriverPod).pod + val driverContainerwithPySpark = step.configurePod(baseDriverPod).container + assert(driverContainerwithPySpark.getEnv.size === 2) + val envs = driverContainerwithPySpark + .getEnv + .asScala + .map(env => (env.getName, env.getValue)) + .toMap + assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) + assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index c1b203e03a357..d66f7ef2e0b27 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -142,7 +142,8 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 161f9afe7bba9..f0b1b1b82b7a3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf} import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, KubernetesFeaturesTestUtils, MountSecretsFeatureStep} +import org.apache.spark.deploy.k8s.features.bindings.PythonDriverFeatureStep class KubernetesDriverBuilderSuite extends SparkFunSuite { @@ -26,6 +27,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val CREDENTIALS_STEP_TYPE = "credentials" private val SERVICE_STEP_TYPE = "service" private val SECRETS_STEP_TYPE = "mount-secrets" + private val PYSPARK_STEP_TYPE = "pyspark-bindings" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( BASIC_STEP_TYPE, classOf[BasicDriverFeatureStep]) @@ -39,12 +41,16 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val secretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) + private val pythonStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + PYSPARK_STEP_TYPE, classOf[PythonDriverFeatureStep]) + private val builderUnderTest: KubernetesDriverBuilder = new KubernetesDriverBuilder( _ => basicFeatureStep, _ => credentialsStep, _ => serviceStep, - _ => secretsStep) + _ => secretsStep, + _ => pythonStep) test("Apply fundamental steps all the time.") { val conf = KubernetesConf( @@ -59,7 +65,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -80,7 +87,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map("secret" -> "secretMountPath"), - Map.empty) + Map.empty, + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, @@ -89,6 +97,29 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { SECRETS_STEP_TYPE) } + test("Apply Python step if main resource is python.") { + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf( + Some(PythonMainAppResource("example.py")), + "test-app", + "main", + Seq.empty), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String]) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + CREDENTIALS_STEP_TYPE, + SERVICE_STEP_TYPE, + PYSPARK_STEP_TYPE) + } + private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) : Unit = { assert(resolvedSpec.systemProperties.size === stepTypes.size) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index f5270623f8acc..7878e0e91e45a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -45,7 +45,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) validateStepTypesApplied(builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE) } @@ -59,7 +60,8 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Map.empty, Map("secret" -> "secretMountPath"), - Map.empty) + Map.empty, + Seq.empty[String]) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, From b7b3db0abfbf425120fa21cc61e603c5d766f8af Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 17 Apr 2018 15:13:45 -0400 Subject: [PATCH 02/16] included entrypoint logic --- bin/docker-image-tool.sh | 23 ++++++--- .../k8s/features/BasicDriverFeatureStep.scala | 7 ++- .../BasicDriverFeatureStepSuite.scala | 51 +++++++++++++++++-- .../spark/bindings/python/Dockerfile | 33 ++++++++++++ .../src/main/dockerfiles/spark/entrypoint.sh | 8 +++ 5 files changed, 110 insertions(+), 12 deletions(-) create mode 100644 resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index f090240065bf1..a871ab5d448c3 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -63,12 +63,20 @@ function build { if [ ! -d "$IMG_PATH" ]; then error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark." fi - - local DOCKERFILE=${DOCKERFILE:-"$IMG_PATH/spark/Dockerfile"} + local BINDING_BUILD_ARGS=( + --build-arg + base_img=$(image_ref spark) + ) + local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"} + local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"} docker build "${BUILD_ARGS[@]}" \ -t $(image_ref spark) \ - -f "$DOCKERFILE" . + -f "$BASEDOCKERFILE" . + + docker build "${BINDING_BUILD_ARGS[@]}" \ + -t $(image_ref spark-py) \ + -f "$PYDOCKERFILE" . } function push { @@ -86,7 +94,8 @@ Commands: push Push a pre-built image to a registry. Requires a repository address to be provided. Options: - -f file Dockerfile to build. By default builds the Dockerfile shipped with Spark. + -f file Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark. + -p file Dockerfile with Python baked in. By default builds the Dockerfile shipped with Spark. -r repo Repository address. -t tag Tag to apply to the built image, or to identify the image to be pushed. -m Use minikube's Docker daemon. @@ -116,12 +125,14 @@ fi REPO= TAG= -DOCKERFILE= +BASEDOCKERFILE= +PYDOCKERFILE= while getopts f:mr:t: option do case "${option}" in - f) DOCKERFILE=${OPTARG};; + f) BASEDOCKERFILE=${OPTARG};; + p) PYDOCKERFILE=${OPTARG};; r) REPO=${OPTARG};; t) TAG=${OPTARG};; m) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 5102f17151d9f..386978100bb12 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -25,6 +25,7 @@ import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ import org.apache.spark.launcher.SparkLauncher @@ -44,6 +45,10 @@ private[spark] class BasicDriverFeatureStep( private val driverCpuCores = conf.get("spark.driver.cores", "1") private val driverLimitCores = conf.get(KUBERNETES_DRIVER_LIMIT_CORES) + private val driverDockerContainer = conf.roleSpecificConf.mainAppResource.map { + case JavaMainAppResource(_) => "driver" + case PythonMainAppResource(_) => "driver-py" + }.getOrElse(throw new SparkException("Must specify a JVM or Python Resource")) // Memory settings private val driverMemoryMiB = conf.get(DRIVER_MEMORY) private val memoryOverheadMiB = conf @@ -89,7 +94,7 @@ private[spark] class BasicDriverFeatureStep( .addToRequests("memory", driverMemoryQuantity) .addToLimits("memory", driverMemoryQuantity) .endResources() - .addToArgs("driver") + .addToArgs(driverDockerContainer) .addToArgs("--properties-file", SPARK_CONF_PATH) .addToArgs("--class", conf.roleSpecificConf.mainClass) // The user application jar is merged into the spark.jars list and managed through that diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 2cae4ba7c95d9..23fb27cc666b2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -24,6 +24,8 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.deploy.k8s.submit.PythonMainAppResource class BasicDriverFeatureStepSuite extends SparkFunSuite { @@ -33,8 +35,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent" private val APP_NAME = "spark-test" private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PYTHON_MAIN_CLASS = "example.py" - private val EXAMPLE_PYTHON_FILES = Seq("example2.py", "example3.py") + private val PY_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" private val APP_ARGS = Array("arg1", "arg2", "\"arg 3\"") private val CUSTOM_ANNOTATION_KEY = "customAnnotation" private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" @@ -62,7 +63,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( - None, + Some(JavaMainAppResource("")), APP_NAME, MAIN_CLASS, APP_ARGS), @@ -112,7 +113,6 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) assert(driverPodMetadata.getAnnotations.asScala === DRIVER_ANNOTATIONS) assert(configuredPod.pod.getSpec.getRestartPolicy === "Never") - val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", "spark.app.id" -> APP_ID, @@ -121,6 +121,47 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { assert(featureStep.getAdditionalPodSystemProperties() === expectedSparkConf) } + test("Check appropriate entrypoint rerouting for various bindings") { + val sparkConf = new SparkConf() + .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") + .set(CONTAINER_IMAGE, "spark-driver:latest") + val javaKubernetesConf = KubernetesConf( + sparkConf, + KubernetesDriverSpecificConf( + Some(JavaMainAppResource("")), + APP_NAME, + PY_MAIN_CLASS, + APP_ARGS), + RESOURCE_NAME_PREFIX, + APP_ID, + DRIVER_LABELS, + DRIVER_ANNOTATIONS, + Map.empty, + DRIVER_ENVS, + Seq.empty[String]) + val pythonKubernetesConf = KubernetesConf( + sparkConf, + KubernetesDriverSpecificConf( + Some(PythonMainAppResource("")), + APP_NAME, + PY_MAIN_CLASS, + APP_ARGS), + RESOURCE_NAME_PREFIX, + APP_ID, + DRIVER_LABELS, + DRIVER_ANNOTATIONS, + Map.empty, + DRIVER_ENVS, + Seq.empty[String]) + val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) + val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) + val basePod = SparkPod.initialPod() + val configuredJavaPod = javaFeatureStep.configurePod(basePod) + val configuredPythonPod = pythonFeatureStep.configurePod(basePod) + assert(configuredJavaPod.container.getArgs.get(0) === "driver") + assert(configuredPythonPod.container.getArgs.get(0) === "driver-py") + } + test("Additional system properties resolve jars and set cluster-mode confs.") { val allJars = Seq("local:///opt/spark/jar1.jar", "hdfs:///opt/spark/jar2.jar") val allFiles = Seq("https://localhost:9000/file1.txt", "local:///opt/spark/file2.txt") @@ -132,7 +173,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( - None, + Some(JavaMainAppResource("")), APP_NAME, MAIN_CLASS, APP_ARGS), diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile new file mode 100644 index 0000000000000..a36dc87dd8027 --- /dev/null +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -0,0 +1,33 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +ARG base_img +FROM $base_img +WORKDIR / +COPY python /opt/spark/python +RUN apk add --no-cache python && \ + python -m ensurepip && \ + rm -r /usr/lib/python*/ensurepip && \ + pip install --upgrade pip setuptools && \ + rm -r /root/.cache +ENV PYTHON_VERSION 2.7.13 +ENV PYSPARK_PYTHON python +ENV PYSPARK_DRIVER_PYTHON python +ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:${PYTHONPATH} + +WORKDIR /opt/spark/work-dir +ENTRYPOINT [ "/opt/entrypoint.sh" ] diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 3e166116aa3fd..54f4f34e08c73 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -62,6 +62,14 @@ case "$SPARK_K8S_CMD" in "$@" ) ;; + driver-py) + CMD=( + "$SPARK_HOME/bin/spark-submit" + --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" + --deploy-mode client + $PYSPARK_PRIMARY $PYSPARK_FILES "$@" + ) + ;; executor) CMD=( From 98cef8ceb0f04cfcefbc482c2a0fe39c75f620c4 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 17 Apr 2018 22:22:55 -0400 Subject: [PATCH 03/16] satisfying integration tests --- .../org/apache/spark/deploy/SparkSubmit.scala | 4 ++- .../spark/deploy/k8s/KubernetesConf.scala | 4 +-- .../k8s/features/BasicDriverFeatureStep.scala | 20 ++++++++---- .../bindings/PythonDriverFeatureStep.scala | 6 ++-- .../k8s/submit/KubernetesDriverBuilder.scala | 8 +++-- .../PythonDriverFeatureStepSuite.scala | 31 +++++++++++++++++++ 6 files changed, 58 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 4820224b45150..793c37143391b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -696,7 +696,9 @@ private[spark] class SparkSubmit extends Logging { if (args.isPython) { childArgs ++= Array("--primary-py-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") - childArgs ++= Array("--other-py-files", args.pyFiles) + if (args.pyFiles != null) { + childArgs ++= Array("--other-py-files", args.pyFiles) + } } else { childArgs ++= Array("--primary-java-resource", args.primaryResource) childArgs ++= Array("--main-class", args.mainClass) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 2e50f44039b9e..7e42661bfeaca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -67,10 +67,8 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) - def pyFiles(): Seq[String] = sparkConf + def pyFiles(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_PY_FILES) - .map(str => str.split(",").toSeq) - .getOrElse(Seq.empty[String]) def pySparkMainResource(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 386978100bb12..b7deec8657d6c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -77,7 +77,7 @@ private[spark] class BasicDriverFeatureStep( ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) } - val driverContainer = new ContainerBuilder(pod.container) + val withoutArgsDriverContainer: ContainerBuilder = new ContainerBuilder(pod.container) .withName(DRIVER_CONTAINER_NAME) .withImage(driverContainerImage) .withImagePullPolicy(conf.imagePullPolicy()) @@ -97,12 +97,20 @@ private[spark] class BasicDriverFeatureStep( .addToArgs(driverDockerContainer) .addToArgs("--properties-file", SPARK_CONF_PATH) .addToArgs("--class", conf.roleSpecificConf.mainClass) - // The user application jar is merged into the spark.jars list and managed through that - // property, so there is no need to reference it explicitly here. - .addToArgs(SparkLauncher.NO_RESOURCE) - .addToArgs(conf.roleSpecificConf.appArgs: _*) - .build() + val driverContainer = + if (driverDockerContainer == "driver-py") { + withoutArgsDriverContainer + .addToArgs(conf.roleSpecificConf.appArgs: _*) + .build() + } else { + // The user application jar is merged into the spark.jars list and managed through that + // property, so there is no need to reference it explicitly here. + withoutArgsDriverContainer + .addToArgs(SparkLauncher.NO_RESOURCE) + .addToArgs(conf.roleSpecificConf.appArgs: _*) + .build() + } val driverPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(driverPodName) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 79cf0ed84de40..efb522f8dd6b0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -30,7 +30,9 @@ private[spark] class PythonDriverFeatureStep( override def configurePod(pod: SparkPod): SparkPod = { val mainResource = kubernetesConf.pySparkMainResource() require(mainResource.isDefined, "PySpark Main Resource must be defined") - val otherPyFiles = kubernetesConf.pyFiles() + val otherPyFiles = kubernetesConf.pyFiles().map(pyFile => + KubernetesUtils.resolveFileUrisAndPath(pyFile.split(",")) + .mkString(",")).getOrElse("null") val withPythonPrimaryFileContainer = new ContainerBuilder(pod.container) .addNewEnv() .withName(ENV_PYSPARK_PRIMARY) @@ -38,7 +40,7 @@ private[spark] class PythonDriverFeatureStep( .endEnv() .addNewEnv() .withName(ENV_PYSPARK_FILES) - .withValue(KubernetesUtils.resolveFileUrisAndPath(otherPyFiles).mkString(",")) + .withValue(if (otherPyFiles == "") {"null"} else otherPyFiles) .endEnv() .build() SparkPod(pod.pod, withPythonPrimaryFileContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 1a0e70299d6ad..6d7b644a2ee77 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -45,9 +45,11 @@ private[spark] class KubernetesDriverBuilder( provideServiceStep(kubernetesConf)) val maybeRoleSecretNamesStep = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { Some(provideSecretsStep(kubernetesConf)) } else None - val maybeNonJVMBindings = kubernetesConf.roleSpecificConf.mainAppResource.map { - case PythonMainAppResource(_) => - providePythonStep(kubernetesConf) + val maybeNonJVMBindings = kubernetesConf.roleSpecificConf.mainAppResource.getOrElse(None) + match { + case PythonMainAppResource(_) => + Some(providePythonStep(kubernetesConf)) + case _ => None } val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 4103c4a36cc90..c695e4faaf1e5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -65,4 +65,35 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) } + test("Python Step testing empty pyfiles") { + val mainResource = "local:///main.py" + val baseDriverPod = SparkPod.initialPod() + val sparkConf = new SparkConf(false) + .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) + .set(KUBERNETES_PYSPARK_PY_FILES, "") + val kubernetesConf = KubernetesConf( + sparkConf, + KubernetesDriverSpecificConf( + Some(PythonMainAppResource("local:///main.py")), + "test-app", + "python-runner", + Seq.empty[String]), + "", + "", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String]) + val step = new PythonDriverFeatureStep(kubernetesConf) + val driverPod = step.configurePod(baseDriverPod).pod + val driverContainerwithPySpark = step.configurePod(baseDriverPod).container + assert(driverContainerwithPySpark.getEnv.size === 2) + val envs = driverContainerwithPySpark + .getEnv + .asScala + .map(env => (env.getName, env.getValue)) + .toMap + assert(envs(ENV_PYSPARK_FILES) === "null") + } } From dc670dcd07944ae30b9b425c26250a21986b2699 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 18 Apr 2018 01:20:12 -0400 Subject: [PATCH 04/16] end-to-end working pyspark --- .../scala/org/apache/spark/deploy/k8s/Config.scala | 7 +++++++ .../scala/org/apache/spark/deploy/k8s/Constants.scala | 1 + .../org/apache/spark/deploy/k8s/KubernetesConf.scala | 9 +++++++-- .../deploy/k8s/features/BasicDriverFeatureStep.scala | 1 - .../features/bindings/PythonDriverFeatureStep.scala | 10 +++++++--- .../k8s/submit/KubernetesClientApplication.scala | 8 ++++---- .../apache/spark/deploy/k8s/KubernetesConfSuite.scala | 10 +++++----- .../bindings/PythonDriverFeatureStepSuite.scala | 10 ++++++---- .../docker/src/main/dockerfiles/spark/entrypoint.sh | 9 ++++++++- 9 files changed, 45 insertions(+), 20 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 1128267eb8f50..f9a4205b3a6e9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -131,6 +131,13 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_PYSPARK_APP_ARGS = + ConfigBuilder("spark.kubernetes.python.appArgs") + .doc("The app arguments for PySpark Jobs") + .internal() + .stringConf + .createOptional + val KUBERNETES_ALLOCATION_BATCH_SIZE = ConfigBuilder("spark.kubernetes.allocation.batch.size") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 1ea5c481e0cea..3884755e3aca7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -74,6 +74,7 @@ private[spark] object Constants { // BINDINGS val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" val ENV_PYSPARK_FILES = "PYSPARK_FILES" + val ENV_PYSPARK_ARGS = "PYSPARK_APP_ARGS" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 7e42661bfeaca..0e28de97b0567 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -73,6 +73,9 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def pySparkMainResource(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) + def pySparkAppArgs(): Option[String] = sparkConf + .get(KUBERNETES_PYSPARK_APP_ARGS) + def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) def imagePullSecrets(): Seq[LocalObjectReference] = { @@ -107,7 +110,7 @@ private[spark] object KubernetesConf { mainAppResource: Option[MainAppResource], mainClass: String, appArgs: Array[String], - maybePyFiles: Seq[String]): KubernetesConf[KubernetesDriverSpecificConf] = { + maybePyFiles: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { val sparkConfWithMainAppJar = sparkConf.clone() val additionalFiles = mutable.ArrayBuffer.empty[String] mainAppResource.foreach { @@ -123,8 +126,10 @@ private[spark] object KubernetesConf { nonJVM match { case PythonMainAppResource(res) => additionalFiles += res - additionalFiles.appendAll(maybePyFiles) + maybePyFiles.foreach{maybePyFiles => + additionalFiles.appendAll(maybePyFiles.split(","))} sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) + sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_APP_ARGS, appArgs.mkString(" ")) } sparkConfWithMainAppJar.set(MEMORY_OVERHEAD_FACTOR, 0.4) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index b7deec8657d6c..0e2f279c189c9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -101,7 +101,6 @@ private[spark] class BasicDriverFeatureStep( val driverContainer = if (driverDockerContainer == "driver-py") { withoutArgsDriverContainer - .addToArgs(conf.roleSpecificConf.appArgs: _*) .build() } else { // The user application jar is merged into the spark.jars list and managed through that diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index efb522f8dd6b0..295d7093c5952 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model.ContainerBuilder import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants.{ENV_PYSPARK_FILES, ENV_PYSPARK_PRIMARY} +import org.apache.spark.deploy.k8s.Constants.{ENV_PYSPARK_ARGS, ENV_PYSPARK_FILES, ENV_PYSPARK_PRIMARY} import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep @@ -32,15 +32,19 @@ private[spark] class PythonDriverFeatureStep( require(mainResource.isDefined, "PySpark Main Resource must be defined") val otherPyFiles = kubernetesConf.pyFiles().map(pyFile => KubernetesUtils.resolveFileUrisAndPath(pyFile.split(",")) - .mkString(",")).getOrElse("null") + .mkString(",")).getOrElse("") val withPythonPrimaryFileContainer = new ContainerBuilder(pod.container) + .addNewEnv() + .withName(ENV_PYSPARK_ARGS) + .withValue(kubernetesConf.pySparkAppArgs().getOrElse("")) + .endEnv() .addNewEnv() .withName(ENV_PYSPARK_PRIMARY) .withValue(KubernetesUtils.resolveFileUri(mainResource.get)) .endEnv() .addNewEnv() .withName(ENV_PYSPARK_FILES) - .withValue(if (otherPyFiles == "") {"null"} else otherPyFiles) + .withValue(if (otherPyFiles == "") {""} else otherPyFiles) .endEnv() .build() SparkPod(pod.pod, withPythonPrimaryFileContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 2fc217e8be6e7..30475108fd2c0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -45,7 +45,7 @@ private[spark] case class ClientArguments( mainAppResource: Option[MainAppResource], mainClass: String, driverArgs: Array[String], - maybePyFiles: Seq[String]) + maybePyFiles: Option[String]) private[spark] object ClientArguments { @@ -53,7 +53,7 @@ private[spark] object ClientArguments { var mainAppResource: Option[MainAppResource] = None var mainClass: Option[String] = None val driverArgs = mutable.ArrayBuffer.empty[String] - var maybePyFiles : Seq[String] = Seq.empty[String] + var maybePyFiles : Option[String] = None args.sliding(2, 2).toList.foreach { case Array("--primary-java-resource", primaryJavaResource: String) => @@ -61,7 +61,7 @@ private[spark] object ClientArguments { case Array("--primary-py-file", primaryPythonResource: String) => mainAppResource = Some(PythonMainAppResource(primaryPythonResource)) case Array("--other-py-files", pyFiles: String) => - maybePyFiles = pyFiles.split(",") + maybePyFiles = Some(pyFiles) case Array("--main-class", clazz: String) => mainClass = Some(clazz) case Array("--arg", arg: String) => @@ -214,7 +214,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val kubernetesResourceNamePrefix = { s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") } - sparkConf.set("spark.kubernetes.python.pyFiles", clientArguments.maybePyFiles.mkString(",")) + sparkConf.set("spark.kubernetes.python.pyFiles", clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, appName, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index dbac40f408abf..a4587cbe37f32 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -56,7 +56,7 @@ class KubernetesConfSuite extends SparkFunSuite { None, MAIN_CLASS, APP_ARGS, - Seq.empty[String]) + None) assert(conf.appId === APP_ID) assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) @@ -78,7 +78,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppJar, MAIN_CLASS, APP_ARGS, - Seq.empty[String]) + None) assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") .split(",") === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) @@ -90,7 +90,7 @@ class KubernetesConfSuite extends SparkFunSuite { None, MAIN_CLASS, APP_ARGS, - Seq.empty[String]) + None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) @@ -111,7 +111,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppResource, MAIN_CLASS, APP_ARGS, - inputPyFiles) + Some(inputPyFiles.mkString(","))) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) @@ -143,7 +143,7 @@ class KubernetesConfSuite extends SparkFunSuite { None, MAIN_CLASS, APP_ARGS, - Seq.empty[String]) + None) assert(conf.roleLabels === Map( SPARK_APP_ID_LABEL -> APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index c695e4faaf1e5..f84d5e2adb35f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -38,6 +38,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) .set(KUBERNETES_PYSPARK_PY_FILES, pyFiles.mkString(",")) .set("spark.files", "local:///example.py") + .set(KUBERNETES_PYSPARK_APP_ARGS, "5 7") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( @@ -56,7 +57,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 2) + assert(driverContainerwithPySpark.getEnv.size === 3) val envs = driverContainerwithPySpark .getEnv .asScala @@ -64,13 +65,13 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .toMap assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) + assert(envs(ENV_PYSPARK_ARGS) === "5 7") } test("Python Step testing empty pyfiles") { val mainResource = "local:///main.py" val baseDriverPod = SparkPod.initialPod() val sparkConf = new SparkConf(false) .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) - .set(KUBERNETES_PYSPARK_PY_FILES, "") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( @@ -88,12 +89,13 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 2) + assert(driverContainerwithPySpark.getEnv.size === 3) val envs = driverContainerwithPySpark .getEnv .asScala .map(env => (env.getName, env.getValue)) .toMap - assert(envs(ENV_PYSPARK_FILES) === "null") + assert(envs(ENV_PYSPARK_FILES) === "") + assert(envs(ENV_PYSPARK_ARGS) === "") } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 54f4f34e08c73..27840bfe635e5 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -53,6 +53,13 @@ if [ -n "$SPARK_MOUNTED_FILES_DIR" ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." . fi +PYSPARK_SECONDARY="$PYSPARK_APP_ARGS" +if [ ! -z "$PYSPARK_FILES" ]; then + PYSPARK_SECONDARY="$PYSPARK_FILES $PYSPARK_APP_ARGS" +fi + + + case "$SPARK_K8S_CMD" in driver) CMD=( @@ -67,7 +74,7 @@ case "$SPARK_K8S_CMD" in "$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client - $PYSPARK_PRIMARY $PYSPARK_FILES "$@" + "$@" $PYSPARK_PRIMARY $PYSPARK_SECONDARY ) ;; From 8d3debb88d065df152ec84a900f963fa361e2bb1 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 2 May 2018 03:56:44 -0400 Subject: [PATCH 05/16] resolved comments and fixed --pyfiles issue and allowed for python2 or python3 to be specified --- .../org/apache/spark/deploy/k8s/Config.scala | 13 ++++++++++++- .../apache/spark/deploy/k8s/Constants.scala | 1 + .../spark/deploy/k8s/KubernetesConf.scala | 5 ++++- .../k8s/features/BasicDriverFeatureStep.scala | 2 +- .../features/BasicExecutorFeatureStep.scala | 3 ++- .../bindings/PythonDriverFeatureStep.scala | 8 ++++++-- .../deploy/k8s/KubernetesConfSuite.scala | 8 +++++--- .../PythonDriverFeatureStepSuite.scala | 9 ++++++--- .../spark/bindings/python/Dockerfile | 11 ++++++----- .../src/main/dockerfiles/spark/entrypoint.sh | 19 ++++++++++++++----- 10 files changed, 57 insertions(+), 22 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index f9a4205b3a6e9..db6148ab036cb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -181,7 +181,18 @@ private[spark] object Config extends Logging { .doc("This sets the Memory Overhead Factor that will allocate memory to non-JVM jobs " + "which in the case of JVM tasks will default to 0.10 and 0.40 for non-JVM jobs") .doubleConf - .createWithDefault(0.10) + .checkValue(mem_overhead => mem_overhead >= 0 && mem_overhead < 1, + "Ensure that memory overhead is a double between 0 --> 1.0") + .createOptional + + val PYSPARK_PYTHON_VERSION = + ConfigBuilder("spark.kubernetes.pyspark.pythonversion") + .doc("This sets the python version. Either 2 or 3. (Python2 or Python3)") + .stringConf + .checkValue(pv => List("2", "3").contains(pv), + "Ensure that Python Version is either Python2 or Python3") + .createWithDefault("2") + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 3884755e3aca7..7637d7b3452a7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -75,6 +75,7 @@ private[spark] object Constants { val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" val ENV_PYSPARK_FILES = "PYSPARK_FILES" val ENV_PYSPARK_ARGS = "PYSPARK_APP_ARGS" + val ENV_PYSPARK_PYTHON_VERSION = "PYSPARK_PYTHON_VERSION" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 0e28de97b0567..d97aaacefa0f7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -76,6 +76,9 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def pySparkAppArgs(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_APP_ARGS) + def pySparkPythonVersion(): String = sparkConf + .get(PYSPARK_PYTHON_VERSION) + def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) def imagePullSecrets(): Seq[LocalObjectReference] = { @@ -131,7 +134,7 @@ private[spark] object KubernetesConf { sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_APP_ARGS, appArgs.mkString(" ")) } - sparkConfWithMainAppJar.set(MEMORY_OVERHEAD_FACTOR, 0.4) + sparkConfWithMainAppJar.setIfMissing(MEMORY_OVERHEAD_FACTOR, 0.4) } val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 0e2f279c189c9..258d4947aae3c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -53,7 +53,7 @@ private[spark] class BasicDriverFeatureStep( private val driverMemoryMiB = conf.get(DRIVER_MEMORY) private val memoryOverheadMiB = conf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR) * driverMemoryMiB).toInt, + .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR).getOrElse(0.1) * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 79d467eadd30f..06d103dd94711 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -54,7 +54,8 @@ private[spark] class BasicExecutorFeatureStep( private val memoryOverheadMiB = kubernetesConf .get(EXECUTOR_MEMORY_OVERHEAD) - .getOrElse(math.max((kubernetesConf.get(MEMORY_OVERHEAD_FACTOR) * executorMemoryMiB).toInt, + .getOrElse(math.max( + (kubernetesConf.get(MEMORY_OVERHEAD_FACTOR).getOrElse(0.1) * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 295d7093c5952..02d8fbc21c151 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -20,7 +20,7 @@ import io.fabric8.kubernetes.api.model.ContainerBuilder import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants.{ENV_PYSPARK_ARGS, ENV_PYSPARK_FILES, ENV_PYSPARK_PRIMARY} +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep @@ -32,7 +32,7 @@ private[spark] class PythonDriverFeatureStep( require(mainResource.isDefined, "PySpark Main Resource must be defined") val otherPyFiles = kubernetesConf.pyFiles().map(pyFile => KubernetesUtils.resolveFileUrisAndPath(pyFile.split(",")) - .mkString(",")).getOrElse("") + .mkString(":")).getOrElse("") val withPythonPrimaryFileContainer = new ContainerBuilder(pod.container) .addNewEnv() .withName(ENV_PYSPARK_ARGS) @@ -46,6 +46,10 @@ private[spark] class PythonDriverFeatureStep( .withName(ENV_PYSPARK_FILES) .withValue(if (otherPyFiles == "") {""} else otherPyFiles) .endEnv() + .addNewEnv() + .withName(ENV_PYSPARK_PYTHON_VERSION) + .withValue(kubernetesConf.pySparkPythonVersion()) + .endEnv() .build() SparkPod(pod.pod, withPythonPrimaryFileContainer) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index a4587cbe37f32..46ae7a2853412 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -93,7 +93,7 @@ class KubernetesConfSuite extends SparkFunSuite { None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) + assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR).isEmpty) } test("Creating driver conf with a python primary file") { @@ -114,14 +114,15 @@ class KubernetesConfSuite extends SparkFunSuite { Some(inputPyFiles.mkString(","))) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) + assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === Some(0.4)) assert(kubernetesConfWithMainResource.sparkFiles === Array("local:///opt/spark/example4.py", mainResourceFile) ++ inputPyFiles) } - test("Resolve driver labels, annotations, secret mount paths, and envs.") { + test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false) + .set(MEMORY_OVERHEAD_FACTOR, 0.3) CUSTOM_LABELS.foreach { case (key, value) => sparkConf.set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$key", value) } @@ -151,6 +152,7 @@ class KubernetesConfSuite extends SparkFunSuite { assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) assert(conf.roleEnvs === CUSTOM_ENVS) + assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === Some(0.3)) } test("Basic executor translated fields.") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index f84d5e2adb35f..37c22b1033d12 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -32,7 +32,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val mainResource = "local:///main.py" val pyFiles = Seq("local:///example2.py", "local:///example3.py") val expectedPySparkFiles = - "/example2.py,/example3.py" + "/example2.py:/example3.py" val baseDriverPod = SparkPod.initialPod() val sparkConf = new SparkConf(false) .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) @@ -57,7 +57,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 3) + assert(driverContainerwithPySpark.getEnv.size === 4) val envs = driverContainerwithPySpark .getEnv .asScala @@ -66,12 +66,14 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) assert(envs(ENV_PYSPARK_ARGS) === "5 7") + assert(envs(ENV_PYSPARK_PYTHON_VERSION) === "2") } test("Python Step testing empty pyfiles") { val mainResource = "local:///main.py" val baseDriverPod = SparkPod.initialPod() val sparkConf = new SparkConf(false) .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) + .set(PYSPARK_PYTHON_VERSION, "3") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( @@ -89,7 +91,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 3) + assert(driverContainerwithPySpark.getEnv.size === 4) val envs = driverContainerwithPySpark .getEnv .asScala @@ -97,5 +99,6 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .toMap assert(envs(ENV_PYSPARK_FILES) === "") assert(envs(ENV_PYSPARK_ARGS) === "") + assert(envs(ENV_PYSPARK_PYTHON_VERSION) === "3") } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index a36dc87dd8027..0067ff99280fe 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -18,16 +18,17 @@ ARG base_img FROM $base_img WORKDIR / -COPY python /opt/spark/python +RUN mkdir ${SPARK_HOME}/python +COPY python/lib ${SPARK_HOME}/python/lib RUN apk add --no-cache python && \ + apk add --no-cache python3 && \ python -m ensurepip && \ + python3 -m ensurepip && \ rm -r /usr/lib/python*/ensurepip && \ pip install --upgrade pip setuptools && \ rm -r /root/.cache -ENV PYTHON_VERSION 2.7.13 -ENV PYSPARK_PYTHON python -ENV PYSPARK_DRIVER_PYTHON python -ENV PYTHONPATH ${SPARK_HOME}/python/:${SPARK_HOME}/python/lib/py4j-0.10.6-src.zip:${PYTHONPATH} + +ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4j-*.zip WORKDIR /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 27840bfe635e5..d32b6bd37443a 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -53,12 +53,21 @@ if [ -n "$SPARK_MOUNTED_FILES_DIR" ]; then cp -R "$SPARK_MOUNTED_FILES_DIR/." . fi -PYSPARK_SECONDARY="$PYSPARK_APP_ARGS" -if [ ! -z "$PYSPARK_FILES" ]; then - PYSPARK_SECONDARY="$PYSPARK_FILES $PYSPARK_APP_ARGS" +if [ -n "$PYSPARK_FILES" ]; then + PYTHONPATH="$PYTHONPATH:$PYSPARK_FILES" fi - +if [ "$PYSPARK_PYTHON_VERSION" == "2" ]; then + pyv="$(python -V 2>&1)" + export PYTHON_VERSION="${pyv:7}" + export PYSPARK_PYTHON="python" + export PYSPARK_DRIVER_PYTHON="python" +elif [ "$PYSPARK_PYTHON_VERSION" == "3" ]; then + pyv3="$(python3 -V 2>&1)" + export PYTHON_VERSION="${pyv3:7}" + export PYSPARK_PYTHON="python3" + export PYSPARK_DRIVER_PYTHON="python3" +fi case "$SPARK_K8S_CMD" in driver) @@ -74,7 +83,7 @@ case "$SPARK_K8S_CMD" in "$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client - "$@" $PYSPARK_PRIMARY $PYSPARK_SECONDARY + "$@" $PYSPARK_PRIMARY $PYSPARK_APP_ARGS ) ;; From 98cc0448a4fd9e85b749c54510ae860ad8148e49 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 6 May 2018 23:41:29 -0400 Subject: [PATCH 06/16] restructured step based pattern to resolve comments --- .../spark/deploy/k8s/KubernetesConf.scala | 6 +- .../k8s/features/BasicDriverFeatureStep.scala | 16 +---- .../bindings/JavaDriverFeatureStep.scala | 42 ++++++++++++ .../bindings/PythonDriverFeatureStep.scala | 65 +++++++++++-------- .../submit/KubernetesClientApplication.scala | 2 +- .../k8s/submit/KubernetesDriverBuilder.scala | 14 ++-- .../bindings/JavaDriverFeatureStepSuite.scala | 54 +++++++++++++++ .../PythonDriverFeatureStepSuite.scala | 10 ++- .../submit/KubernetesDriverBuilderSuite.scala | 17 +++-- .../src/main/dockerfiles/spark/entrypoint.sh | 8 ++- 10 files changed, 173 insertions(+), 61 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index d97aaacefa0f7..d95f1a1709cb5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -67,15 +67,14 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) + def getRoleConf: T = roleSpecificConf + def pyFiles(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_PY_FILES) def pySparkMainResource(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) - def pySparkAppArgs(): Option[String] = sparkConf - .get(KUBERNETES_PYSPARK_APP_ARGS) - def pySparkPythonVersion(): String = sparkConf .get(PYSPARK_PYTHON_VERSION) @@ -132,7 +131,6 @@ private[spark] object KubernetesConf { maybePyFiles.foreach{maybePyFiles => additionalFiles.appendAll(maybePyFiles.split(","))} sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) - sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_APP_ARGS, appArgs.mkString(" ")) } sparkConfWithMainAppJar.setIfMissing(MEMORY_OVERHEAD_FACTOR, 0.4) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 258d4947aae3c..01693f2b5ec33 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -27,7 +27,6 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ -import org.apache.spark.launcher.SparkLauncher private[spark] class BasicDriverFeatureStep( conf: KubernetesConf[KubernetesDriverSpecificConf]) @@ -77,7 +76,7 @@ private[spark] class BasicDriverFeatureStep( ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) } - val withoutArgsDriverContainer: ContainerBuilder = new ContainerBuilder(pod.container) + val driverContainer = new ContainerBuilder(pod.container) .withName(DRIVER_CONTAINER_NAME) .withImage(driverContainerImage) .withImagePullPolicy(conf.imagePullPolicy()) @@ -97,19 +96,8 @@ private[spark] class BasicDriverFeatureStep( .addToArgs(driverDockerContainer) .addToArgs("--properties-file", SPARK_CONF_PATH) .addToArgs("--class", conf.roleSpecificConf.mainClass) + .build() - val driverContainer = - if (driverDockerContainer == "driver-py") { - withoutArgsDriverContainer - .build() - } else { - // The user application jar is merged into the spark.jars list and managed through that - // property, so there is no need to reference it explicitly here. - withoutArgsDriverContainer - .addToArgs(SparkLauncher.NO_RESOURCE) - .addToArgs(conf.roleSpecificConf.appArgs: _*) - .build() - } val driverPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(driverPodName) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala new file mode 100644 index 0000000000000..fda0fff712df1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.bindings + +import io.fabric8.kubernetes.api.model.ContainerBuilder +import io.fabric8.kubernetes.api.model.HasMetadata + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf +import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep +import org.apache.spark.launcher.SparkLauncher + +private[spark] class JavaDriverFeatureStep( + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) + extends KubernetesFeatureConfigStep { + override def configurePod(pod: SparkPod): SparkPod = { + val withDriverArgs = new ContainerBuilder(pod.container) + // The user application jar is merged into the spark.jars list and managed through that + // property, so there is no need to reference it explicitly here. + .addToArgs(SparkLauncher.NO_RESOURCE) + .addToArgs(kubernetesConf.roleSpecificConf.appArgs: _*) + .build() + SparkPod(pod.pod, withDriverArgs) + } + override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 02d8fbc21c151..fb2fbb9dd423a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -16,42 +16,55 @@ */ package org.apache.spark.deploy.k8s.features.bindings +import scala.collection.JavaConverters._ + import io.fabric8.kubernetes.api.model.ContainerBuilder +import io.fabric8.kubernetes.api.model.EnvVar +import io.fabric8.kubernetes.api.model.EnvVarBuilder import io.fabric8.kubernetes.api.model.HasMetadata -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep private[spark] class PythonDriverFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) + kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val mainResource = kubernetesConf.pySparkMainResource() - require(mainResource.isDefined, "PySpark Main Resource must be defined") - val otherPyFiles = kubernetesConf.pyFiles().map(pyFile => - KubernetesUtils.resolveFileUrisAndPath(pyFile.split(",")) - .mkString(":")).getOrElse("") - val withPythonPrimaryFileContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_PYSPARK_ARGS) - .withValue(kubernetesConf.pySparkAppArgs().getOrElse("")) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(KubernetesUtils.resolveFileUri(mainResource.get)) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_FILES) - .withValue(if (otherPyFiles == "") {""} else otherPyFiles) - .endEnv() - .addNewEnv() - .withName(ENV_PYSPARK_PYTHON_VERSION) - .withValue(kubernetesConf.pySparkPythonVersion()) - .endEnv() - .build() - SparkPod(pod.pod, withPythonPrimaryFileContainer) + val roleConf = kubernetesConf.roleSpecificConf + require(roleConf.mainAppResource.isDefined, "PySpark Main Resource must be defined") + val maybePythonArgs: Option[EnvVar] = Option(roleConf.appArgs).filter(_.nonEmpty).map( + s => + new EnvVarBuilder() + .withName(ENV_PYSPARK_ARGS) + .withValue(s.mkString(",")) + .build()) + val maybePythonFiles: Option[EnvVar] = kubernetesConf.pyFiles().map( + pyFiles => + new EnvVarBuilder() + .withName(ENV_PYSPARK_FILES) + .withValue(KubernetesUtils.resolveFileUrisAndPath(pyFiles.split(",")) + .mkString(":")) + .build()) + val envSeq : Seq[EnvVar] = + Seq(new EnvVarBuilder() + .withName(ENV_PYSPARK_PRIMARY) + .withValue(KubernetesUtils.resolveFileUri(kubernetesConf.pySparkMainResource().get)) + .build(), + new EnvVarBuilder() + .withName(ENV_PYSPARK_PYTHON_VERSION) + .withValue(kubernetesConf.pySparkPythonVersion()) + .build()) + val pythonEnvs = envSeq ++ + maybePythonArgs.toSeq ++ + maybePythonFiles.toSeq + + val withPythonPrimaryContainer = new ContainerBuilder(pod.container) + .addAllToEnv(pythonEnvs.asJava).build() + + SparkPod(pod.pod, withPythonPrimaryContainer) } override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 30475108fd2c0..eaff47205dbbc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -214,7 +214,7 @@ private[spark] class KubernetesClientApplication extends SparkApplication { val kubernetesResourceNamePrefix = { s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") } - sparkConf.set("spark.kubernetes.python.pyFiles", clientArguments.maybePyFiles.getOrElse("")) + sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, appName, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 6d7b644a2ee77..54fb5f3148d25 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf} import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, MountSecretsFeatureStep} import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep -import org.apache.spark.deploy.k8s.features.bindings.PythonDriverFeatureStep +import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep} private[spark] class KubernetesDriverBuilder( provideBasicStep: (KubernetesConf[KubernetesDriverSpecificConf]) => BasicDriverFeatureStep = @@ -32,8 +32,12 @@ private[spark] class KubernetesDriverBuilder( provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountSecretsFeatureStep) = new MountSecretsFeatureStep(_), + provideJavaStep: ( + KubernetesConf[KubernetesDriverSpecificConf] + => JavaDriverFeatureStep) = + new JavaDriverFeatureStep(_), providePythonStep: ( - KubernetesConf[_ <: KubernetesRoleSpecificConf] + KubernetesConf[KubernetesDriverSpecificConf] => PythonDriverFeatureStep) = new PythonDriverFeatureStep(_)) { @@ -45,8 +49,10 @@ private[spark] class KubernetesDriverBuilder( provideServiceStep(kubernetesConf)) val maybeRoleSecretNamesStep = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { Some(provideSecretsStep(kubernetesConf)) } else None - val maybeNonJVMBindings = kubernetesConf.roleSpecificConf.mainAppResource.getOrElse(None) + val bindingsStep = kubernetesConf.roleSpecificConf.mainAppResource.getOrElse(None) match { + case JavaMainAppResource(_) => + Some(provideJavaStep(kubernetesConf)) case PythonMainAppResource(_) => Some(providePythonStep(kubernetesConf)) case _ => None @@ -54,7 +60,7 @@ private[spark] class KubernetesDriverBuilder( val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ maybeRoleSecretNamesStep.toSeq ++ - maybeNonJVMBindings.toSeq + bindingsStep.toSeq var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { val configuredPod = feature.configurePod(spec.pod) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..910c887d11f36 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features.bindings + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.submit.PythonMainAppResource + +class JavaDriverFeatureStepSuite extends SparkFunSuite { + + + test("Python Step modifies container correctly") { + val baseDriverPod = SparkPod.initialPod() + val sparkConf = new SparkConf(false) + val kubernetesConf = KubernetesConf( + sparkConf, + KubernetesDriverSpecificConf( + Some(PythonMainAppResource("local:///main.jar")), + "test-app", + "java-runner", + Seq("5 7")), + "", + "", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String]) + + val step = new JavaDriverFeatureStep(kubernetesConf) + val driverPod = step.configurePod(baseDriverPod).pod + val driverContainerwithJavaStep = step.configurePod(baseDriverPod).container + assert(driverContainerwithJavaStep.getArgs.size === 2) + val args = driverContainerwithJavaStep + .getArgs.asScala + assert(args === List("spark-internal", "5 7")) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 37c22b1033d12..892f3f161737c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -38,14 +38,14 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) .set(KUBERNETES_PYSPARK_PY_FILES, pyFiles.mkString(",")) .set("spark.files", "local:///example.py") - .set(KUBERNETES_PYSPARK_APP_ARGS, "5 7") + .set(PYSPARK_PYTHON_VERSION, "2") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( Some(PythonMainAppResource("local:///main.py")), "test-app", "python-runner", - Seq.empty[String]), + Seq("5 7")), "", "", Map.empty, @@ -57,7 +57,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 4) +// assert(driverContainerwithPySpark.getEnv.size === 4) val envs = driverContainerwithPySpark .getEnv .asScala @@ -91,14 +91,12 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 4) + assert(driverContainerwithPySpark.getEnv.size === 2) val envs = driverContainerwithPySpark .getEnv .asScala .map(env => (env.getName, env.getValue)) .toMap - assert(envs(ENV_PYSPARK_FILES) === "") - assert(envs(ENV_PYSPARK_ARGS) === "") assert(envs(ENV_PYSPARK_PYTHON_VERSION) === "3") } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index f0b1b1b82b7a3..89e08704c9f85 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf} import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, KubernetesFeaturesTestUtils, MountSecretsFeatureStep} -import org.apache.spark.deploy.k8s.features.bindings.PythonDriverFeatureStep +import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep} class KubernetesDriverBuilderSuite extends SparkFunSuite { @@ -27,6 +27,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val CREDENTIALS_STEP_TYPE = "credentials" private val SERVICE_STEP_TYPE = "service" private val SECRETS_STEP_TYPE = "mount-secrets" + private val JAVA_STEP_TYPE = "java-bindings" private val PYSPARK_STEP_TYPE = "pyspark-bindings" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( @@ -41,6 +42,9 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val secretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) + private val javaStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + JAVA_STEP_TYPE, classOf[JavaDriverFeatureStep]) + private val pythonStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( PYSPARK_STEP_TYPE, classOf[PythonDriverFeatureStep]) @@ -50,13 +54,14 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => credentialsStep, _ => serviceStep, _ => secretsStep, + _ => javaStep, _ => pythonStep) test("Apply fundamental steps all the time.") { val conf = KubernetesConf( new SparkConf(false), KubernetesDriverSpecificConf( - None, + Some(JavaMainAppResource("example.jar")), "test-app", "main", Seq.empty), @@ -71,14 +76,15 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE) + SERVICE_STEP_TYPE, + JAVA_STEP_TYPE) } test("Apply secrets step if secrets are present.") { val conf = KubernetesConf( new SparkConf(false), KubernetesDriverSpecificConf( - None, + Some(JavaMainAppResource("example.jar")), "test-app", "main", Seq.empty), @@ -94,7 +100,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, SERVICE_STEP_TYPE, - SECRETS_STEP_TYPE) + SECRETS_STEP_TYPE, + JAVA_STEP_TYPE) } test("Apply Python step if main resource is python.") { diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index d32b6bd37443a..78620d233a88d 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -57,6 +57,12 @@ if [ -n "$PYSPARK_FILES" ]; then PYTHONPATH="$PYTHONPATH:$PYSPARK_FILES" fi +PYSPARK_ARGS="" +if [ -n "$PYSPARK_APP_ARGS" ]; then + PYSPARK_ARGS="$PYSPARK_APP_ARGS" +fi + + if [ "$PYSPARK_PYTHON_VERSION" == "2" ]; then pyv="$(python -V 2>&1)" export PYTHON_VERSION="${pyv:7}" @@ -83,7 +89,7 @@ case "$SPARK_K8S_CMD" in "$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" --deploy-mode client - "$@" $PYSPARK_PRIMARY $PYSPARK_APP_ARGS + "$@" $PYSPARK_PRIMARY $PYSPARK_ARGS ) ;; From bf738dccbd5a213f3a51ef37f5c02a8915c8fbb6 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 8 May 2018 18:51:15 -0400 Subject: [PATCH 07/16] resolved comments --- .../k8s/features/BasicDriverFeatureStep.scala | 7 ---- .../bindings/JavaDriverFeatureStep.scala | 4 ++ .../bindings/PythonDriverFeatureStep.scala | 13 +++--- .../k8s/submit/KubernetesDriverBuilder.scala | 16 +++----- .../deploy/k8s/KubernetesConfSuite.scala | 14 +++---- .../BasicDriverFeatureStepSuite.scala | 2 - .../bindings/JavaDriverFeatureStepSuite.scala | 29 +++++++------ .../PythonDriverFeatureStepSuite.scala | 41 +++++++++++-------- .../submit/KubernetesDriverBuilderSuite.scala | 35 +++++++++++++--- 9 files changed, 94 insertions(+), 67 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 01693f2b5ec33..c884aff7c3a0c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -44,10 +44,6 @@ private[spark] class BasicDriverFeatureStep( private val driverCpuCores = conf.get("spark.driver.cores", "1") private val driverLimitCores = conf.get(KUBERNETES_DRIVER_LIMIT_CORES) - private val driverDockerContainer = conf.roleSpecificConf.mainAppResource.map { - case JavaMainAppResource(_) => "driver" - case PythonMainAppResource(_) => "driver-py" - }.getOrElse(throw new SparkException("Must specify a JVM or Python Resource")) // Memory settings private val driverMemoryMiB = conf.get(DRIVER_MEMORY) private val memoryOverheadMiB = conf @@ -93,9 +89,6 @@ private[spark] class BasicDriverFeatureStep( .addToRequests("memory", driverMemoryQuantity) .addToLimits("memory", driverMemoryQuantity) .endResources() - .addToArgs(driverDockerContainer) - .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", conf.roleSpecificConf.mainClass) .build() val driverPod = new PodBuilder(pod.pod) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala index fda0fff712df1..40200a3b83322 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala @@ -20,6 +20,7 @@ import io.fabric8.kubernetes.api.model.ContainerBuilder import io.fabric8.kubernetes.api.model.HasMetadata import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants.SPARK_CONF_PATH import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep import org.apache.spark.launcher.SparkLauncher @@ -29,6 +30,9 @@ private[spark] class JavaDriverFeatureStep( extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val withDriverArgs = new ContainerBuilder(pod.container) + .addToArgs("driver") + .addToArgs("--properties-file", SPARK_CONF_PATH) + .addToArgs("--class", kubernetesConf.roleSpecificConf.mainClass) // The user application jar is merged into the spark.jars list and managed through that // property, so there is no need to reference it explicitly here. .addToArgs(SparkLauncher.NO_RESOURCE) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index fb2fbb9dd423a..05775c0b8dec8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.k8s.features.bindings import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.ContainerBuilder -import io.fabric8.kubernetes.api.model.EnvVar import io.fabric8.kubernetes.api.model.EnvVarBuilder import io.fabric8.kubernetes.api.model.HasMetadata @@ -35,20 +34,20 @@ private[spark] class PythonDriverFeatureStep( override def configurePod(pod: SparkPod): SparkPod = { val roleConf = kubernetesConf.roleSpecificConf require(roleConf.mainAppResource.isDefined, "PySpark Main Resource must be defined") - val maybePythonArgs: Option[EnvVar] = Option(roleConf.appArgs).filter(_.nonEmpty).map( + val maybePythonArgs = Option(roleConf.appArgs).filter(_.nonEmpty).map( s => new EnvVarBuilder() .withName(ENV_PYSPARK_ARGS) .withValue(s.mkString(",")) .build()) - val maybePythonFiles: Option[EnvVar] = kubernetesConf.pyFiles().map( + val maybePythonFiles = kubernetesConf.pyFiles().map( pyFiles => new EnvVarBuilder() .withName(ENV_PYSPARK_FILES) .withValue(KubernetesUtils.resolveFileUrisAndPath(pyFiles.split(",")) .mkString(":")) .build()) - val envSeq : Seq[EnvVar] = + val envSeq = Seq(new EnvVarBuilder() .withName(ENV_PYSPARK_PRIMARY) .withValue(KubernetesUtils.resolveFileUri(kubernetesConf.pySparkMainResource().get)) @@ -62,7 +61,11 @@ private[spark] class PythonDriverFeatureStep( maybePythonFiles.toSeq val withPythonPrimaryContainer = new ContainerBuilder(pod.container) - .addAllToEnv(pythonEnvs.asJava).build() + .addAllToEnv(pythonEnvs.asJava) + .addToArgs("driver-py") + .addToArgs("--properties-file", SPARK_CONF_PATH) + .addToArgs("--class", roleConf.mainClass) + .build() SparkPod(pod.pod, withPythonPrimaryContainer) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 54fb5f3148d25..0edf010691197 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -35,7 +35,7 @@ private[spark] class KubernetesDriverBuilder( provideJavaStep: ( KubernetesConf[KubernetesDriverSpecificConf] => JavaDriverFeatureStep) = - new JavaDriverFeatureStep(_), + new JavaDriverFeatureStep(_), providePythonStep: ( KubernetesConf[KubernetesDriverSpecificConf] => PythonDriverFeatureStep) = @@ -49,18 +49,14 @@ private[spark] class KubernetesDriverBuilder( provideServiceStep(kubernetesConf)) val maybeRoleSecretNamesStep = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { Some(provideSecretsStep(kubernetesConf)) } else None - val bindingsStep = kubernetesConf.roleSpecificConf.mainAppResource.getOrElse(None) - match { + val bindingsStep = kubernetesConf.roleSpecificConf.mainAppResource.map { case JavaMainAppResource(_) => - Some(provideJavaStep(kubernetesConf)) + provideJavaStep(kubernetesConf) case PythonMainAppResource(_) => - Some(providePythonStep(kubernetesConf)) - case _ => None - } + providePythonStep(kubernetesConf)}.getOrElse(provideJavaStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = - baseFeatures ++ - maybeRoleSecretNamesStep.toSeq ++ - bindingsStep.toSeq + (baseFeatures :+ bindingsStep) ++ + maybeRoleSecretNamesStep.toSeq var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) for (feature <- allFeatures) { val configuredPod = feature.configurePod(spec.pod) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 46ae7a2853412..65ff963f204ca 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -53,10 +53,10 @@ class KubernetesConfSuite extends SparkFunSuite { APP_NAME, RESOURCE_NAME_PREFIX, APP_ID, - None, + mainAppResource = None, MAIN_CLASS, APP_ARGS, - None) + maybePyFiles = None) assert(conf.appId === APP_ID) assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) @@ -78,7 +78,7 @@ class KubernetesConfSuite extends SparkFunSuite { mainAppJar, MAIN_CLASS, APP_ARGS, - None) + maybePyFiles = None) assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") .split(",") === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) @@ -87,10 +87,10 @@ class KubernetesConfSuite extends SparkFunSuite { APP_NAME, RESOURCE_NAME_PREFIX, APP_ID, - None, + mainAppResource = None, MAIN_CLASS, APP_ARGS, - None) + maybePyFiles = None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR).isEmpty) @@ -141,10 +141,10 @@ class KubernetesConfSuite extends SparkFunSuite { APP_NAME, RESOURCE_NAME_PREFIX, APP_ID, - None, + mainAppResource = None, MAIN_CLASS, APP_ARGS, - None) + maybePyFiles = None) assert(conf.roleLabels === Map( SPARK_APP_ID_LABEL -> APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 23fb27cc666b2..539c36262d080 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -158,8 +158,6 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val basePod = SparkPod.initialPod() val configuredJavaPod = javaFeatureStep.configurePod(basePod) val configuredPythonPod = pythonFeatureStep.configurePod(basePod) - assert(configuredJavaPod.container.getArgs.get(0) === "driver") - assert(configuredPythonPod.container.getArgs.get(0) === "driver-py") } test("Additional system properties resolve jars and set cluster-mode confs.") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala index 910c887d11f36..e570f8b90a32b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -20,35 +20,40 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.PythonMainAppResource class JavaDriverFeatureStepSuite extends SparkFunSuite { - - test("Python Step modifies container correctly") { + test("Java Step modifies container correctly") { val baseDriverPod = SparkPod.initialPod() val sparkConf = new SparkConf(false) val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( Some(PythonMainAppResource("local:///main.jar")), - "test-app", + "test-class", "java-runner", Seq("5 7")), - "", - "", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Seq.empty[String]) + appResourceNamePrefix = "", + appId = "", + roleLabels = Map.empty, + roleAnnotations = Map.empty, + roleSecretNamesToMountPaths = Map.empty, + roleEnvs = Map.empty, + sparkFiles = Seq.empty[String]) val step = new JavaDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithJavaStep = step.configurePod(baseDriverPod).container - assert(driverContainerwithJavaStep.getArgs.size === 2) + assert(driverContainerwithJavaStep.getArgs.size === 7) val args = driverContainerwithJavaStep .getArgs.asScala - assert(args === List("spark-internal", "5 7")) + assert(args === List( + "driver", + "--properties-file", SPARK_CONF_PATH, + "--class", "test-class", + "spark-internal", "5 7")) + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 892f3f161737c..555b920f531e0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -46,18 +46,18 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { "test-app", "python-runner", Seq("5 7")), - "", - "", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Seq.empty[String]) + appResourceNamePrefix = "", + appId = "", + roleLabels = Map.empty, + roleAnnotations = Map.empty, + roleSecretNamesToMountPaths = Map.empty, + roleEnvs = Map.empty, + sparkFiles = Seq.empty[String]) val step = new PythonDriverFeatureStep(kubernetesConf) val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container -// assert(driverContainerwithPySpark.getEnv.size === 4) + assert(driverContainerwithPySpark.getEnv.size === 4) val envs = driverContainerwithPySpark .getEnv .asScala @@ -78,20 +78,25 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { sparkConf, KubernetesDriverSpecificConf( Some(PythonMainAppResource("local:///main.py")), - "test-app", + "test-class-py", "python-runner", Seq.empty[String]), - "", - "", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Seq.empty[String]) + appResourceNamePrefix = "", + appId = "", + roleLabels = Map.empty, + roleAnnotations = Map.empty, + roleSecretNamesToMountPaths = Map.empty, + roleEnvs = Map.empty, + sparkFiles = Seq.empty[String]) val step = new PythonDriverFeatureStep(kubernetesConf) - val driverPod = step.configurePod(baseDriverPod).pod val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 2) + val args = driverContainerwithPySpark + .getArgs.asScala + assert(driverContainerwithPySpark.getArgs.size === 5) + assert(args === List( + "driver-py", + "--properties-file", SPARK_CONF_PATH, + "--class", "test-class-py")) val envs = driverContainerwithPySpark .getEnv .asScala diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 89e08704c9f85..a5210733906c0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -76,8 +76,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - JAVA_STEP_TYPE) + JAVA_STEP_TYPE, + SERVICE_STEP_TYPE) } test("Apply secrets step if secrets are present.") { @@ -100,8 +100,31 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, SERVICE_STEP_TYPE, - SECRETS_STEP_TYPE, - JAVA_STEP_TYPE) + JAVA_STEP_TYPE, + SECRETS_STEP_TYPE) + } + + test("Apply Java step if main resource is none.") { + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf( + None, + "test-app", + "main", + Seq.empty), + "prefix", + "appId", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Seq.empty[String]) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + BASIC_STEP_TYPE, + CREDENTIALS_STEP_TYPE, + JAVA_STEP_TYPE, + SERVICE_STEP_TYPE) } test("Apply Python step if main resource is python.") { @@ -123,8 +146,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - PYSPARK_STEP_TYPE) + PYSPARK_STEP_TYPE, + SERVICE_STEP_TYPE) } private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) From 0344f9012ffd2860457c3f05462c757eb72419fb Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 9 May 2018 14:42:28 -0400 Subject: [PATCH 08/16] resolving style issues --- .../k8s/features/bindings/PythonDriverFeatureStepSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index 555b920f531e0..b59516e305bdf 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -20,9 +20,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.submit.PythonMainAppResource import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.PythonMainAppResource class PythonDriverFeatureStepSuite extends SparkFunSuite { From f2fc53e00815f84b54d07c0e564aeefa20ad4d58 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 13 May 2018 15:50:20 -0400 Subject: [PATCH 09/16] resolved commits --- .../main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala | 2 ++ .../deploy/k8s/features/bindings/PythonDriverFeatureStep.scala | 2 ++ .../src/main/dockerfiles/spark/bindings/python/Dockerfile | 1 + 3 files changed, 5 insertions(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index d95f1a1709cb5..2163efa863512 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -124,6 +124,8 @@ private[spark] object KubernetesConf { if (!previousJars.contains(res)) { sparkConfWithMainAppJar.setJars(previousJars ++ Seq(res)) } + // The function of this outer match is to account for multiple nonJVM + // bindings that will all have increased MEMORY_OVERHEAD case nonJVM: NonJVMResource => nonJVM match { case PythonMainAppResource(res) => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 05775c0b8dec8..b48efbb7e12ca 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -41,6 +41,8 @@ private[spark] class PythonDriverFeatureStep( .withValue(s.mkString(",")) .build()) val maybePythonFiles = kubernetesConf.pyFiles().map( + // Dilineation by ":" is to append the PySpark Files to the PYTHONPATH + // of the respective PySpark pod pyFiles => new EnvVarBuilder() .withName(ENV_PYSPARK_FILES) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index 0067ff99280fe..15e757fa61c98 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -20,6 +20,7 @@ FROM $base_img WORKDIR / RUN mkdir ${SPARK_HOME}/python COPY python/lib ${SPARK_HOME}/python/lib +# TODO: Investigate running both pip and pip3 via virtualenvs RUN apk add --no-cache python && \ apk add --no-cache python3 && \ python -m ensurepip && \ From 914ff75788e40eeac14b90d2fa9ad56fcf7b7fd3 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 11 May 2018 03:15:58 -0400 Subject: [PATCH 10/16] resolve rebase conflicts --- .../spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala index 91e184b84b86e..5f5149854c185 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala @@ -43,7 +43,8 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { Map.empty, Map.empty, Map.empty, - Map.empty) + Map.empty, + Seq.empty[String]) } test("Resolve to default local dir if neither env nor configuration are set") { From d4006075efc746ca99010b59b94582a8104498e4 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Sun, 13 May 2018 18:01:33 -0400 Subject: [PATCH 11/16] import statements refactoring --- .../k8s/features/bindings/JavaDriverFeatureStep.scala | 6 ++---- .../k8s/features/bindings/PythonDriverFeatureStep.scala | 8 ++------ 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala index 40200a3b83322..f52ec9fdc677e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala @@ -16,12 +16,10 @@ */ package org.apache.spark.deploy.k8s.features.bindings -import io.fabric8.kubernetes.api.model.ContainerBuilder -import io.fabric8.kubernetes.api.model.HasMetadata +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} import org.apache.spark.deploy.k8s.Constants.SPARK_CONF_PATH -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep import org.apache.spark.launcher.SparkLauncher diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index b48efbb7e12ca..2faec7016bb39 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -18,14 +18,10 @@ package org.apache.spark.deploy.k8s.features.bindings import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.ContainerBuilder -import io.fabric8.kubernetes.api.model.EnvVarBuilder -import io.fabric8.kubernetes.api.model.HasMetadata +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf -import org.apache.spark.deploy.k8s.KubernetesUtils import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep private[spark] class PythonDriverFeatureStep( From 7bedeb6ee154136b3462c6b8454efb8a1b56dd5c Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 31 May 2018 18:53:28 -0400 Subject: [PATCH 12/16] resolved comments --- .../main/scala/org/apache/spark/deploy/k8s/Config.scala | 4 ++-- .../scala/org/apache/spark/deploy/k8s/Constants.scala | 2 +- .../org/apache/spark/deploy/k8s/KubernetesConf.scala | 8 +++----- .../deploy/k8s/features/BasicDriverFeatureStep.scala | 2 +- .../deploy/k8s/features/BasicExecutorFeatureStep.scala | 2 +- .../k8s/features/bindings/PythonDriverFeatureStep.scala | 6 +++--- .../apache/spark/deploy/k8s/KubernetesConfSuite.scala | 6 +++--- .../k8s/features/BasicDriverFeatureStepSuite.scala | 9 ++++++--- .../features/bindings/PythonDriverFeatureStepSuite.scala | 9 ++++----- .../main/dockerfiles/spark/bindings/python/Dockerfile | 4 ++++ .../docker/src/main/dockerfiles/spark/entrypoint.sh | 4 ++-- 11 files changed, 30 insertions(+), 26 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index db6148ab036cb..f2baf6ed19519 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -183,9 +183,9 @@ private[spark] object Config extends Logging { .doubleConf .checkValue(mem_overhead => mem_overhead >= 0 && mem_overhead < 1, "Ensure that memory overhead is a double between 0 --> 1.0") - .createOptional + .createWithDefault(0.1) - val PYSPARK_PYTHON_VERSION = + val PYSPARK_MAJOR_PYTHON_VERSION = ConfigBuilder("spark.kubernetes.pyspark.pythonversion") .doc("This sets the python version. Either 2 or 3. (Python2 or Python3)") .stringConf diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 7637d7b3452a7..69bd03d1eda6f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -75,7 +75,7 @@ private[spark] object Constants { val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" val ENV_PYSPARK_FILES = "PYSPARK_FILES" val ENV_PYSPARK_ARGS = "PYSPARK_APP_ARGS" - val ENV_PYSPARK_PYTHON_VERSION = "PYSPARK_PYTHON_VERSION" + val ENV_PYSPARK_MAJOR_PYTHON_VERSION = "PYSPARK_MAJOR_PYTHON_VERSION" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 2163efa863512..ec8671fca1775 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -67,8 +67,6 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( .map(str => str.split(",").toSeq) .getOrElse(Seq.empty[String]) - def getRoleConf: T = roleSpecificConf - def pyFiles(): Option[String] = sparkConf .get(KUBERNETES_PYSPARK_PY_FILES) @@ -76,7 +74,7 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) def pySparkPythonVersion(): String = sparkConf - .get(PYSPARK_PYTHON_VERSION) + .get(PYSPARK_MAJOR_PYTHON_VERSION) def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) @@ -125,7 +123,7 @@ private[spark] object KubernetesConf { sparkConfWithMainAppJar.setJars(previousJars ++ Seq(res)) } // The function of this outer match is to account for multiple nonJVM - // bindings that will all have increased MEMORY_OVERHEAD + // bindings that will all have increased MEMORY_OVERHEAD_FACTOR to 0.4 case nonJVM: NonJVMResource => nonJVM match { case PythonMainAppResource(res) => @@ -134,7 +132,7 @@ private[spark] object KubernetesConf { additionalFiles.appendAll(maybePyFiles.split(","))} sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) } - sparkConfWithMainAppJar.setIfMissing(MEMORY_OVERHEAD_FACTOR, 0.4) + sparkConfWithMainAppJar.set(MEMORY_OVERHEAD_FACTOR, 0.4) } val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index c884aff7c3a0c..143dc8a12304e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -48,7 +48,7 @@ private[spark] class BasicDriverFeatureStep( private val driverMemoryMiB = conf.get(DRIVER_MEMORY) private val memoryOverheadMiB = conf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR).getOrElse(0.1) * driverMemoryMiB).toInt, + .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR) * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 6c5a10a45459d..91c54a9776982 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -55,7 +55,7 @@ private[spark] class BasicExecutorFeatureStep( private val memoryOverheadMiB = kubernetesConf .get(EXECUTOR_MEMORY_OVERHEAD) .getOrElse(math.max( - (kubernetesConf.get(MEMORY_OVERHEAD_FACTOR).getOrElse(0.1) * executorMemoryMiB).toInt, + (kubernetesConf.get(MEMORY_OVERHEAD_FACTOR) * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 2faec7016bb39..c20bcac1f8987 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -31,10 +31,10 @@ private[spark] class PythonDriverFeatureStep( val roleConf = kubernetesConf.roleSpecificConf require(roleConf.mainAppResource.isDefined, "PySpark Main Resource must be defined") val maybePythonArgs = Option(roleConf.appArgs).filter(_.nonEmpty).map( - s => + pyArgs => new EnvVarBuilder() .withName(ENV_PYSPARK_ARGS) - .withValue(s.mkString(",")) + .withValue(pyArgs.mkString(",")) .build()) val maybePythonFiles = kubernetesConf.pyFiles().map( // Dilineation by ":" is to append the PySpark Files to the PYTHONPATH @@ -51,7 +51,7 @@ private[spark] class PythonDriverFeatureStep( .withValue(KubernetesUtils.resolveFileUri(kubernetesConf.pySparkMainResource().get)) .build(), new EnvVarBuilder() - .withName(ENV_PYSPARK_PYTHON_VERSION) + .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) .withValue(kubernetesConf.pySparkPythonVersion()) .build()) val pythonEnvs = envSeq ++ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 65ff963f204ca..db0067f788b4e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -93,7 +93,7 @@ class KubernetesConfSuite extends SparkFunSuite { maybePyFiles = None) assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR).isEmpty) + assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) } test("Creating driver conf with a python primary file") { @@ -114,7 +114,7 @@ class KubernetesConfSuite extends SparkFunSuite { Some(inputPyFiles.mkString(","))) assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === Some(0.4)) + assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) assert(kubernetesConfWithMainResource.sparkFiles === Array("local:///opt/spark/example4.py", mainResourceFile) ++ inputPyFiles) } @@ -152,7 +152,7 @@ class KubernetesConfSuite extends SparkFunSuite { assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) assert(conf.roleEnvs === CUSTOM_ENVS) - assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === Some(0.3)) + assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } test("Basic executor translated fields.") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 539c36262d080..fadad2361c95c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -122,11 +122,14 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { } test("Check appropriate entrypoint rerouting for various bindings") { - val sparkConf = new SparkConf() + val javaSparkConf = new SparkConf() + .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") + .set(CONTAINER_IMAGE, "spark-driver:latest") + val pythonSparkConf = new SparkConf() .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") .set(CONTAINER_IMAGE, "spark-driver:latest") val javaKubernetesConf = KubernetesConf( - sparkConf, + javaSparkConf, KubernetesDriverSpecificConf( Some(JavaMainAppResource("")), APP_NAME, @@ -140,7 +143,7 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { DRIVER_ENVS, Seq.empty[String]) val pythonKubernetesConf = KubernetesConf( - sparkConf, + pythonSparkConf, KubernetesDriverSpecificConf( Some(PythonMainAppResource("")), APP_NAME, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala index b59516e305bdf..3eb56e690b6a2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.deploy.k8s.submit.PythonMainAppResource class PythonDriverFeatureStepSuite extends SparkFunSuite { - test("Python Step modifies container correctly") { val expectedMainResource = "/main.py" val mainResource = "local:///main.py" @@ -38,7 +37,7 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) .set(KUBERNETES_PYSPARK_PY_FILES, pyFiles.mkString(",")) .set("spark.files", "local:///example.py") - .set(PYSPARK_PYTHON_VERSION, "2") + .set(PYSPARK_MAJOR_PYTHON_VERSION, "2") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( @@ -66,14 +65,14 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) assert(envs(ENV_PYSPARK_ARGS) === "5 7") - assert(envs(ENV_PYSPARK_PYTHON_VERSION) === "2") + assert(envs(ENV_PYSPARK_MAJOR_PYTHON_VERSION) === "2") } test("Python Step testing empty pyfiles") { val mainResource = "local:///main.py" val baseDriverPod = SparkPod.initialPod() val sparkConf = new SparkConf(false) .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) - .set(PYSPARK_PYTHON_VERSION, "3") + .set(PYSPARK_MAJOR_PYTHON_VERSION, "3") val kubernetesConf = KubernetesConf( sparkConf, KubernetesDriverSpecificConf( @@ -102,6 +101,6 @@ class PythonDriverFeatureStepSuite extends SparkFunSuite { .asScala .map(env => (env.getName, env.getValue)) .toMap - assert(envs(ENV_PYSPARK_PYTHON_VERSION) === "3") + assert(envs(ENV_PYSPARK_MAJOR_PYTHON_VERSION) === "3") } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index 15e757fa61c98..72bb9620b45de 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -25,8 +25,12 @@ RUN apk add --no-cache python && \ apk add --no-cache python3 && \ python -m ensurepip && \ python3 -m ensurepip && \ + # We remove ensurepip since it adds no functionality since pip is + # installed on the image and it just takes up 1.6MB on the image rm -r /usr/lib/python*/ensurepip && \ pip install --upgrade pip setuptools && \ + # You may install with python3 packages by using pip3.6 + # Removed the .cache to save space rm -r /root/.cache ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4j-*.zip diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh index 78620d233a88d..acdb4b1f09e0a 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -63,12 +63,12 @@ if [ -n "$PYSPARK_APP_ARGS" ]; then fi -if [ "$PYSPARK_PYTHON_VERSION" == "2" ]; then +if [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "2" ]; then pyv="$(python -V 2>&1)" export PYTHON_VERSION="${pyv:7}" export PYSPARK_PYTHON="python" export PYSPARK_DRIVER_PYTHON="python" -elif [ "$PYSPARK_PYTHON_VERSION" == "3" ]; then +elif [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "3" ]; then pyv3="$(python3 -V 2>&1)" export PYTHON_VERSION="${pyv3:7}" export PYSPARK_PYTHON="python3" From 24a704e74f2c5816e5ea60dbf607be00c090ae8b Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 1 Jun 2018 10:44:33 -0400 Subject: [PATCH 13/16] setIfMissing --- .../main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index e3329b9e66dc6..b0ccaa36b01ed 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -133,7 +133,7 @@ private[spark] object KubernetesConf { additionalFiles.appendAll(maybePyFiles.split(","))} sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) } - sparkConfWithMainAppJar.set(MEMORY_OVERHEAD_FACTOR, 0.4) + sparkConfWithMainAppJar.setIfMissing(MEMORY_OVERHEAD_FACTOR, 0.4) } val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( From 6a6d69d01be0739716947320a4ad4d80e8eaa115 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 7 Jun 2018 13:06:09 -0400 Subject: [PATCH 14/16] added e2e tests on --py-files and inclusion of docs on config values --- docs/running-on-kubernetes.md | 16 ++++++++ .../src/main/python/py_container_checks.py | 31 +++++++++++++++ examples/src/main/python/pyfiles.py | 38 +++++++++++++++++++ .../org/apache/spark/deploy/k8s/Config.scala | 4 +- 4 files changed, 87 insertions(+), 2 deletions(-) create mode 100644 examples/src/main/python/py_container_checks.py create mode 100644 examples/src/main/python/pyfiles.py diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index a4b2b98b0b649..65677b339f47c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -624,4 +624,20 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.secrets.ENV_VAR=spark-secret:key. + + spark.kubernetes.memoryOverheadFactor + 0.1 + + This sets the Memory Overhead Factor that will allocate memory to non-JVM jobs which in the case of JVM tasks will default to 0.10 and 0.40 for non-JVM jobs. + This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with + a higher default. + + + + spark.kubernetes.pyspark.pythonversion + "2" + + This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. + + diff --git a/examples/src/main/python/py_container_checks.py b/examples/src/main/python/py_container_checks.py new file mode 100644 index 0000000000000..a8fae8add303e --- /dev/null +++ b/examples/src/main/python/py_container_checks.py @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import os +import sys + +""" + These are various tests to test the Python container image. + This file will be distributed via --py-files in the e2e tests. +""" +def version_check(python_env, major_python_version): + env_version = os.environ.get('PYSPARK_PYTHON') + print("Python runtime version check is: " + \ + str(sys.version_info[0] == major_python_version)) + + print("Python environment version check is: " + \ + str(env_version == python_env)) diff --git a/examples/src/main/python/pyfiles.py b/examples/src/main/python/pyfiles.py new file mode 100644 index 0000000000000..fc61b9d492599 --- /dev/null +++ b/examples/src/main/python/pyfiles.py @@ -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 __future__ import print_function + +import sys + +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + """ + Usage: pyfiles [major_python_version] + """ + spark = SparkSession \ + .builder \ + .appName("PyFilesTest") \ + .getOrCreate() + + from py_container_checks import version_check + # Begin of Python container checks + version_check(sys.argv[1], 2 if sys.argv[1]=="python" else 3) + + spark.stop() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 712bf140ba0e5..590deaa72e7ee 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -187,10 +187,10 @@ private[spark] object Config extends Logging { val PYSPARK_MAJOR_PYTHON_VERSION = ConfigBuilder("spark.kubernetes.pyspark.pythonversion") - .doc("This sets the python version. Either 2 or 3. (Python2 or Python3)") + .doc("This sets the major Python version. Either 2 or 3. (Python2 or Python3)") .stringConf .checkValue(pv => List("2", "3").contains(pv), - "Ensure that Python Version is either Python2 or Python3") + "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") From ab92913f1d0c303a5ff6b2937019d5e47c25611d Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Thu, 7 Jun 2018 18:03:02 -0400 Subject: [PATCH 15/16] style issues --- examples/src/main/python/py_container_checks.py | 13 +++++++------ examples/src/main/python/pyfiles.py | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/examples/src/main/python/py_container_checks.py b/examples/src/main/python/py_container_checks.py index a8fae8add303e..f6b3be2806c82 100644 --- a/examples/src/main/python/py_container_checks.py +++ b/examples/src/main/python/py_container_checks.py @@ -18,14 +18,15 @@ import os import sys -""" - These are various tests to test the Python container image. - This file will be distributed via --py-files in the e2e tests. -""" + def version_check(python_env, major_python_version): + """ + These are various tests to test the Python container image. + This file will be distributed via --py-files in the e2e tests. + """ env_version = os.environ.get('PYSPARK_PYTHON') - print("Python runtime version check is: " + \ + print("Python runtime version check is: " + str(sys.version_info[0] == major_python_version)) - print("Python environment version check is: " + \ + print("Python environment version check is: " + str(env_version == python_env)) diff --git a/examples/src/main/python/pyfiles.py b/examples/src/main/python/pyfiles.py index fc61b9d492599..4193654b49a12 100644 --- a/examples/src/main/python/pyfiles.py +++ b/examples/src/main/python/pyfiles.py @@ -33,6 +33,6 @@ from py_container_checks import version_check # Begin of Python container checks - version_check(sys.argv[1], 2 if sys.argv[1]=="python" else 3) + version_check(sys.argv[1], 2 if sys.argv[1] == "python" else 3) spark.stop() From a61d8973a8961fa69e50757f049b04bda292a088 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Fri, 8 Jun 2018 12:47:08 -0400 Subject: [PATCH 16/16] resolve comments on docs and addition of unit test --- docs/running-on-kubernetes.md | 6 ++---- .../spark/deploy/k8s/KubernetesConfSuite.scala | 17 +++++++++++++++++ 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 65677b339f47c..23ef934a5fb30 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -270,7 +270,6 @@ future versions of the spark-kubernetes integration. Some of these include: -* PySpark * R * Dynamic Executor Scaling * Local File Dependency Management @@ -628,9 +627,8 @@ specific to Spark on Kubernetes. spark.kubernetes.memoryOverheadFactor 0.1 - This sets the Memory Overhead Factor that will allocate memory to non-JVM jobs which in the case of JVM tasks will default to 0.10 and 0.40 for non-JVM jobs. - This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with - a higher default. + This sets the Memory Overhead Factor that will allocate memory to non-JVM memory, which includes off-heap memory allocations, non-JVM tasks, and various systems processes. For JVM-based jobs this value will default to 0.10 and 0.40 for non-JVM jobs. + This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with a higher default. diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala index 7f28918ed720e..661f942435921 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesConfSuite.scala @@ -122,6 +122,23 @@ class KubernetesConfSuite extends SparkFunSuite { === Array("local:///opt/spark/example4.py", mainResourceFile) ++ inputPyFiles) } + test("Testing explicit setting of memory overhead on non-JVM tasks") { + val sparkConf = new SparkConf(false) + .set(MEMORY_OVERHEAD_FACTOR, 0.3) + + val mainResourceFile = "local:///opt/spark/main.py" + val mainAppResource = Some(PythonMainAppResource(mainResourceFile)) + val conf = KubernetesConf.createDriverConf( + sparkConf, + APP_NAME, + RESOURCE_NAME_PREFIX, + APP_ID, + mainAppResource, + MAIN_CLASS, + APP_ARGS, + None) + assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) + } test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false)