From b61f4957bb55e77d75d65fcf3235bd521ff3440f Mon Sep 17 00:00:00 2001 From: mccheah Date: Wed, 6 Sep 2017 16:01:37 -0700 Subject: [PATCH 01/25] Move executor pod construction to a separate class. (#452) * Move executor pod construction to a separate class. This is the first of several measures to make KubernetesClusterSchedulerBackend feasible to test. * Revert change to README * Address comments. * Resolve merge conflicts. Move MiB change to ExecutorPodFactory. --- .../kubernetes/ExecutorPodFactory.scala | 294 ++++++++++++++++++ .../kubernetes/KubernetesClusterManager.scala | 10 +- .../KubernetesClusterSchedulerBackend.scala | 288 +---------------- .../NodeAffinityExecutorPodModifier.scala | 85 +++++ .../kubernetes/ShuffleServiceConfig.scala | 22 ++ 5 files changed, 420 insertions(+), 279 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala new file mode 100644 index 0000000000000..6355afa0a5041 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala @@ -0,0 +1,294 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.kubernetes + +import java.util.concurrent.atomic.AtomicLong + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} +import org.apache.commons.io.FilenameUtils +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} +import org.apache.spark.util.Utils + +// Configures executor pods. Construct one of these with a SparkConf to set up properties that are +// common across all executors. Then, pass in dynamic parameters into createExecutorPod. +private[spark] trait ExecutorPodFactory { + def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + shuffleServiceConfig: Option[ShuffleServiceConfig], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod +} + +private[spark] class ExecutorPodFactoryImpl( + sparkConf: SparkConf, + nodeAffinityExecutorPodModifier: NodeAffinityExecutorPodModifier, + mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], + executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], + executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) + extends ExecutorPodFactory { + + import ExecutorPodFactoryImpl._ + + private val executorExtraClasspath = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) + private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) + + private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + sparkConf, + KUBERNETES_EXECUTOR_LABEL_PREFIX, + KUBERNETES_EXECUTOR_LABELS, + "executor label") + require( + !executorLabels.contains(SPARK_APP_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") + require( + !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), + s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + + s" Spark.") + + private val executorAnnotations = + ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + sparkConf, + KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, + KUBERNETES_EXECUTOR_ANNOTATIONS, + "executor annotation") + private val nodeSelector = + ConfigurationUtils.parsePrefixedKeyValuePairs( + sparkConf, + KUBERNETES_NODE_SELECTOR_PREFIX, + "node selector") + + private val executorDockerImage = sparkConf.get(EXECUTOR_DOCKER_IMAGE) + private val dockerImagePullPolicy = sparkConf.get(DOCKER_IMAGE_PULL_POLICY) + private val executorPort = sparkConf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) + private val blockmanagerPort = sparkConf + .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) + private val kubernetesDriverPodName = sparkConf + .get(KUBERNETES_DRIVER_POD_NAME) + .getOrElse(throw new SparkException("Must specify the driver pod name")) + + private val executorPodNamePrefix = sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) + + private val executorMemoryMiB = sparkConf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) + private val executorMemoryString = sparkConf.get( + org.apache.spark.internal.config.EXECUTOR_MEMORY.key, + org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) + + private val memoryOverheadMiB = sparkConf + .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) + .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, + MEMORY_OVERHEAD_MIN_MIB)) + private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB + + private val executorCores = sparkConf.getDouble("spark.executor.cores", 1d) + private val executorLimitCores = sparkConf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) + + override def createExecutorPod( + executorId: String, + applicationId: String, + driverUrl: String, + executorEnvs: Seq[(String, String)], + shuffleServiceConfig: Option[ShuffleServiceConfig], + driverPod: Pod, + nodeToLocalTaskCount: Map[String, Int]): Pod = { + val name = s"$executorPodNamePrefix-exec-$executorId" + + // hostname must be no longer than 63 characters, so take the last 63 characters of the pod + // name as the hostname. This preserves uniqueness since the end of name contains + // executorId and applicationId + val hostname = name.substring(Math.max(0, name.length - 63)) + val resolvedExecutorLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> applicationId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ + executorLabels + val executorMemoryQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryMiB}Mi") + .build() + val executorMemoryLimitQuantity = new QuantityBuilder(false) + .withAmount(s"${executorMemoryWithOverhead}Mi") + .build() + val executorCpuQuantity = new QuantityBuilder(false) + .withAmount(executorCores.toString) + .build() + val executorExtraClasspathEnv = executorExtraClasspath.map { cp => + new EnvVarBuilder() + .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) + .withValue(cp) + .build() + } + val executorExtraJavaOptionsEnv = sparkConf + .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) + .map { opts => + val delimitedOpts = Utils.splitCommandString(opts) + delimitedOpts.zipWithIndex.map { + case (opt, index) => + new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + } + }.getOrElse(Seq.empty[EnvVar]) + val executorEnv = (Seq( + (ENV_EXECUTOR_PORT, executorPort.toString), + (ENV_DRIVER_URL, driverUrl), + // Executor backend expects integral value for executor cores, so round it up to an int. + (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, applicationId), + (ENV_EXECUTOR_ID, executorId), + (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ executorEnvs) + .map(env => new EnvVarBuilder() + .withName(env._1) + .withValue(env._2) + .build() + ) ++ Seq( + new EnvVarBuilder() + .withName(ENV_EXECUTOR_POD_IP) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) + .build() + ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq + val requiredPorts = Seq( + (EXECUTOR_PORT_NAME, executorPort), + (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) + .map(port => { + new ContainerPortBuilder() + .withName(port._1) + .withContainerPort(port._2) + .build() + }) + + val executorContainer = new ContainerBuilder() + .withName(s"executor") + .withImage(executorDockerImage) + .withImagePullPolicy(dockerImagePullPolicy) + .withNewResources() + .addToRequests("memory", executorMemoryQuantity) + .addToLimits("memory", executorMemoryLimitQuantity) + .addToRequests("cpu", executorCpuQuantity) + .endResources() + .addAllToEnv(executorEnv.asJava) + .withPorts(requiredPorts.asJava) + .build() + + val executorPod = new PodBuilder() + .withNewMetadata() + .withName(name) + .withLabels(resolvedExecutorLabels.asJava) + .withAnnotations(executorAnnotations.asJava) + .withOwnerReferences() + .addNewOwnerReference() + .withController(true) + .withApiVersion(driverPod.getApiVersion) + .withKind(driverPod.getKind) + .withName(driverPod.getMetadata.getName) + .withUid(driverPod.getMetadata.getUid) + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withHostname(hostname) + .withRestartPolicy("Never") + .withNodeSelector(nodeSelector.asJava) + .endSpec() + .build() + + val containerWithExecutorLimitCores = executorLimitCores.map { + limitCores => + val executorCpuLimitQuantity = new QuantityBuilder(false) + .withAmount(limitCores) + .build() + new ContainerBuilder(executorContainer) + .editResources() + .addToLimits("cpu", executorCpuLimitQuantity) + .endResources() + .build() + }.getOrElse(executorContainer) + + val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => + config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => + new ContainerBuilder(container) + .addNewVolumeMount() + .withName(FilenameUtils.getBaseName(dir)) + .withMountPath(dir) + .endVolumeMount() + .build() + } + }.getOrElse(containerWithExecutorLimitCores) + val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => + config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => + new PodBuilder(builder) + .editSpec() + .addNewVolume() + .withName(FilenameUtils.getBaseName(dir)) + .withNewHostPath() + .withPath(dir) + .endHostPath() + .endVolume() + .endSpec() + .build() + } + }.getOrElse(executorPod) + val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = + mountSmallFilesBootstrap.map { bootstrap => + bootstrap.mountSmallFilesSecret( + withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) + }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) + val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = + executorInitContainerBootstrap.map { bootstrap => + val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( + PodWithDetachedInitContainer( + withMaybeSmallFilesMountedPod, + new ContainerBuilder().build(), + withMaybeSmallFilesMountedContainer)) + + val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => + plugin.mountResourceStagingServerSecretIntoInitContainer( + podWithDetachedInitContainer.initContainer) + }.getOrElse(podWithDetachedInitContainer.initContainer) + + val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( + podWithDetachedInitContainer.pod, resolvedInitContainer) + + val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => + plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) + }.getOrElse(podWithAttachedInitContainer) + + (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) + }.getOrElse((withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer)) + + val executorPodWithNodeAffinity = + nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( + executorPodWithInitContainer, nodeToLocalTaskCount) + new PodBuilder(executorPodWithNodeAffinity) + .editSpec() + .addToContainers(initBootstrappedExecutorContainer) + .endSpec() + .build() + } +} + +private object ExecutorPodFactoryImpl { + private val DEFAULT_STATIC_PORT = 10000 +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index b89e81bcb0be9..f63d0aeabad3b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -96,12 +96,16 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf, Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + val executorPodFactory = new ExecutorPodFactoryImpl( + sparkConf, + NodeAffinityExecutorPodModifierImpl, + mountSmallFilesBootstrap, + executorInitContainerbootStrap, + executorInitContainerSecretVolumePlugin) new KubernetesClusterSchedulerBackend( sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, - executorInitContainerbootStrap, - executorInitContainerSecretVolumePlugin, - mountSmallFilesBootstrap, + executorPodFactory, kubernetesClient) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 49cac4d81a0b1..20fe23fbcd92f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -33,11 +33,10 @@ import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} -import org.apache.spark.internal.config import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} @@ -49,9 +48,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, - executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], - executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], - mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], + executorPodFactory: ExecutorPodFactory, kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -71,42 +68,8 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorsToRemove = Collections.newSetFromMap[String]( new ConcurrentHashMap[String, java.lang.Boolean]()).asScala - private val executorExtraClasspath = conf.get( - org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) - private val executorJarsDownloadDir = conf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - - private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - conf, - KUBERNETES_EXECUTOR_LABEL_PREFIX, - KUBERNETES_EXECUTOR_LABELS, - "executor label") - require( - !executorLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is" + - s" reserved for Spark.") - require( - !executorLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - s" Spark.") - - private val executorAnnotations = - ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( - conf, - KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, - KUBERNETES_EXECUTOR_ANNOTATIONS, - "executor annotation") - private val nodeSelector = - ConfigurationUtils.parsePrefixedKeyValuePairs( - conf, - KUBERNETES_NODE_SELECTOR_PREFIX, - "node-selector") private var shufflePodCache: Option[ShufflePodCache] = None - private val executorDockerImage = conf.get(EXECUTOR_DOCKER_IMAGE) - private val dockerImagePullPolicy = conf.get(DOCKER_IMAGE_PULL_POLICY) private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) - private val executorPort = conf.getInt("spark.executor.port", DEFAULT_STATIC_PORT) - private val blockmanagerPort = conf - .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) @@ -114,22 +77,8 @@ private[spark] class KubernetesClusterSchedulerBackend( throw new SparkException("Must specify the driver pod name")) private val executorPodNamePrefix = conf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - private val executorMemoryMiB = conf.get(org.apache.spark.internal.config.EXECUTOR_MEMORY) - private val executorMemoryString = conf.get( - org.apache.spark.internal.config.EXECUTOR_MEMORY.key, - org.apache.spark.internal.config.EXECUTOR_MEMORY.defaultValueString) - - private val memoryOverheadMiB = conf - .get(KUBERNETES_EXECUTOR_MEMORY_OVERHEAD) - .getOrElse(math.max((MEMORY_OVERHEAD_FACTOR * executorMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) - private val executorMemoryWithOverheadMiB = executorMemoryMiB + memoryOverheadMiB - - private val executorCores = conf.getDouble("spark.executor.cores", 1d) - private val executorLimitCores = conf.getOption(KUBERNETES_EXECUTOR_LIMIT_CORES.key) - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) + ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). @@ -181,7 +130,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorWatchResource = new AtomicReference[Closeable] protected var totalExpectedExecutors = new AtomicInteger(0) - private val driverUrl = RpcEndpointAddress( sc.getConf.get("spark.driver.host"), sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), @@ -274,8 +222,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private val objectMapper = new ObjectMapper().registerModule(DefaultScalaModule) - private def getShuffleClient(): KubernetesExternalShuffleClient = { new KubernetesExternalShuffleClient( SparkTransportConf.fromSparkConf(conf, "shuffle"), @@ -390,37 +336,6 @@ private[spark] class KubernetesClusterSchedulerBackend( nodeToLocalTaskCount.toMap[String, Int] } - private def addNodeAffinityAnnotationIfUseful( - baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { - def scaleToRange(value: Int, baseMin: Double, baseMax: Double, - rangeMin: Double, rangeMax: Double): Int = - (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt - - if (nodeToTaskCount.nonEmpty) { - val taskTotal = nodeToTaskCount.foldLeft(0)(_ + _._2) - // Normalize to node affinity weights in 1 to 100 range. - val nodeToWeight = nodeToTaskCount.map{ - case (node, taskCount) => - (node, scaleToRange(taskCount, 1, taskTotal, rangeMin = 1, rangeMax = 100))} - val weightToNodes = nodeToWeight.groupBy(_._2).mapValues(_.keys) - // see https://kubernetes.io/docs/concepts/configuration/assign-pod-node - val nodeAffinityJson = objectMapper.writeValueAsString(SchedulerAffinity(NodeAffinity( - preferredDuringSchedulingIgnoredDuringExecution = - for ((weight, nodes) <- weightToNodes) yield - WeightedPreference(weight, - Preference(Array(MatchExpression("kubernetes.io/hostname", "In", nodes)))) - ))) - // TODO: Use non-annotation syntax when we switch to K8s version 1.6. - logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") - new PodBuilder(baseExecutorPod).editMetadata() - .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) - .endMetadata() - .build() - } else { - baseExecutorPod - } - } - /** * Allocates a new executor pod * @@ -431,179 +346,16 @@ private[spark] class KubernetesClusterSchedulerBackend( */ private def allocateNewExecutorPod(nodeToLocalTaskCount: Map[String, Int]): (String, Pod) = { val executorId = EXECUTOR_ID_COUNTER.incrementAndGet().toString - val name = s"$executorPodNamePrefix-exec-$executorId" - - // hostname must be no longer than 63 characters, so take the last 63 characters of the pod - // name as the hostname. This preserves uniqueness since the end of name contains - // executorId and applicationId - val hostname = name.substring(Math.max(0, name.length - 63)) - val resolvedExecutorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> applicationId(), - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorLabels - val executorMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryMiB}Mi") - .build() - val executorMemoryLimitQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryWithOverheadMiB}Mi") - .build() - val executorCpuQuantity = new QuantityBuilder(false) - .withAmount(executorCores.toString) - .build() - val executorExtraClasspathEnv = executorExtraClasspath.map { cp => - new EnvVarBuilder() - .withName(ENV_EXECUTOR_EXTRA_CLASSPATH) - .withValue(cp) - .build() - } - val executorExtraJavaOptionsEnv = conf - .get(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS) - .map { opts => - val delimitedOpts = Utils.splitCommandString(opts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() - } - }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = (Seq( - (ENV_EXECUTOR_PORT, executorPort.toString), - (ENV_DRIVER_URL, driverUrl), - // Executor backend expects integral value for executor cores, so round it up to an int. - (ENV_EXECUTOR_CORES, math.ceil(executorCores).toInt.toString), - (ENV_EXECUTOR_MEMORY, executorMemoryString), - (ENV_APPLICATION_ID, applicationId()), - (ENV_EXECUTOR_ID, executorId), - (ENV_MOUNTED_CLASSPATH, s"$executorJarsDownloadDir/*")) ++ sc.executorEnvs.toSeq) - .map(env => new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() - ) ++ Seq( - new EnvVarBuilder() - .withName(ENV_EXECUTOR_POD_IP) - .withValueFrom(new EnvVarSourceBuilder() - .withNewFieldRef("v1", "status.podIP") - .build()) - .build() - ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq - val requiredPorts = Seq( - (EXECUTOR_PORT_NAME, executorPort), - (BLOCK_MANAGER_PORT_NAME, blockmanagerPort)) - .map(port => { - new ContainerPortBuilder() - .withName(port._1) - .withContainerPort(port._2) - .build() - }) - - val executorContainer = new ContainerBuilder() - .withName(s"executor") - .withImage(executorDockerImage) - .withImagePullPolicy(dockerImagePullPolicy) - .withNewResources() - .addToRequests("memory", executorMemoryQuantity) - .addToLimits("memory", executorMemoryLimitQuantity) - .addToRequests("cpu", executorCpuQuantity) - .endResources() - .addAllToEnv(executorEnv.asJava) - .withPorts(requiredPorts.asJava) - .build() - - val executorPod = new PodBuilder() - .withNewMetadata() - .withName(name) - .withLabels(resolvedExecutorLabels.asJava) - .withAnnotations(executorAnnotations.asJava) - .withOwnerReferences() - .addNewOwnerReference() - .withController(true) - .withApiVersion(driverPod.getApiVersion) - .withKind(driverPod.getKind) - .withName(driverPod.getMetadata.getName) - .withUid(driverPod.getMetadata.getUid) - .endOwnerReference() - .endMetadata() - .withNewSpec() - .withHostname(hostname) - .withRestartPolicy("Never") - .withNodeSelector(nodeSelector.asJava) - .endSpec() - .build() - - val containerWithExecutorLimitCores = executorLimitCores.map { - limitCores => - val executorCpuLimitQuantity = new QuantityBuilder(false) - .withAmount(limitCores) - .build() - new ContainerBuilder(executorContainer) - .editResources() - .addToLimits("cpu", executorCpuLimitQuantity) - .endResources() - .build() - }.getOrElse(executorContainer) - - val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => - new ContainerBuilder(container) - .addNewVolumeMount() - .withName(FilenameUtils.getBaseName(dir)) - .withMountPath(dir) - .endVolumeMount() - .build() - } - }.getOrElse(containerWithExecutorLimitCores) - val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => - new PodBuilder(builder) - .editSpec() - .addNewVolume() - .withName(FilenameUtils.getBaseName(dir)) - .withNewHostPath() - .withPath(dir) - .endHostPath() - .endVolume() - .endSpec() - .build() - } - }.getOrElse(executorPod) - val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = - mountSmallFilesBootstrap.map { bootstrap => - bootstrap.mountSmallFilesSecret( - withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) - }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) - val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = - executorInitContainerBootstrap.map { bootstrap => - val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( - PodWithDetachedInitContainer( - withMaybeSmallFilesMountedPod, - new ContainerBuilder().build(), - withMaybeSmallFilesMountedContainer)) - - val resolvedInitContainer = executorMountInitContainerSecretPlugin.map { plugin => - plugin.mountResourceStagingServerSecretIntoInitContainer( - podWithDetachedInitContainer.initContainer) - }.getOrElse(podWithDetachedInitContainer.initContainer) - - val podWithAttachedInitContainer = InitContainerUtil.appendInitContainer( - podWithDetachedInitContainer.pod, resolvedInitContainer) - - val resolvedPodWithMountedSecret = executorMountInitContainerSecretPlugin.map { plugin => - plugin.addResourceStagingServerSecretVolumeToPod(podWithAttachedInitContainer) - }.getOrElse(podWithAttachedInitContainer) - - (resolvedPodWithMountedSecret, podWithDetachedInitContainer.mainContainer) - }.getOrElse((withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer)) - - val executorPodWithNodeAffinity = addNodeAffinityAnnotationIfUseful( - executorPodWithInitContainer, nodeToLocalTaskCount) - val resolvedExecutorPod = new PodBuilder(executorPodWithNodeAffinity) - .editSpec() - .addToContainers(initBootstrappedExecutorContainer) - .endSpec() - .build() + val executorPod = executorPodFactory.createExecutorPod( + executorId, + applicationId(), + driverUrl, + sc.conf.getExecutorEnv, + shuffleServiceConfig, + driverPod, + nodeToLocalTaskCount) try { - (executorId, kubernetesClient.pods.create(resolvedExecutorPod)) + (executorId, kubernetesClient.pods.create(executorPod)) } catch { case throwable: Throwable => logError("Failed to allocate executor pod.", throwable) @@ -786,10 +538,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } } } -case class ShuffleServiceConfig( - shuffleNamespace: String, - shuffleLabels: Map[String, String], - shuffleDirs: Seq[String]) private object KubernetesClusterSchedulerBackend { private val DEFAULT_STATIC_PORT = 10000 @@ -804,15 +552,3 @@ private object KubernetesClusterSchedulerBackend { } } -/** - * These case classes model K8s node affinity syntax for - * preferredDuringSchedulingIgnoredDuringExecution. - * - * see https://kubernetes.io/docs/concepts/configuration/assign-pod-node - */ -case class SchedulerAffinity(nodeAffinity: NodeAffinity) -case class NodeAffinity(preferredDuringSchedulingIgnoredDuringExecution: - Iterable[WeightedPreference]) -case class WeightedPreference(weight: Int, preference: Preference) -case class Preference(matchExpressions: Array[MatchExpression]) -case class MatchExpression(key: String, operator: String, values: Iterable[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala new file mode 100644 index 0000000000000..d73bc6cf93aa2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.kubernetes + +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.module.scala.DefaultScalaModule +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} + +import org.apache.spark.deploy.kubernetes.constants.ANNOTATION_EXECUTOR_NODE_AFFINITY +import org.apache.spark.internal.Logging + +// Applies a node affinity annotation to executor pods so that pods can be placed optimally for +// locality. +private[spark] trait NodeAffinityExecutorPodModifier { + def addNodeAffinityAnnotationIfUseful( + baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod +} + +private[spark] object NodeAffinityExecutorPodModifierImpl + extends NodeAffinityExecutorPodModifier with Logging { + + private val OBJECT_MAPPER = new ObjectMapper().registerModule(DefaultScalaModule) + + private def scaleToRange( + value: Int, + baseMin: Double, + baseMax: Double, + rangeMin: Double, + rangeMax: Double): Int = { + (((rangeMax - rangeMin) * (value - baseMin) / (baseMax - baseMin)) + rangeMin).toInt + } + override def addNodeAffinityAnnotationIfUseful( + baseExecutorPod: Pod, nodeToTaskCount: Map[String, Int]): Pod = { + if (nodeToTaskCount.nonEmpty) { + val taskTotal = nodeToTaskCount.foldLeft(0)(_ + _._2) + // Normalize to node affinity weights in 1 to 100 range. + val nodeToWeight = nodeToTaskCount.map { + case (node, taskCount) => + (node, scaleToRange(taskCount, 1, taskTotal, rangeMin = 1, rangeMax = 100)) + } + val weightToNodes = nodeToWeight.groupBy(_._2).mapValues(_.keys) + // @see https://kubernetes.io/docs/concepts/configuration/assign-pod-node + val nodeAffinityJson = OBJECT_MAPPER.writeValueAsString(SchedulerAffinity(NodeAffinity( + preferredDuringSchedulingIgnoredDuringExecution = + for ((weight, nodes) <- weightToNodes) yield { + WeightedPreference( + weight, + Preference(Array(MatchExpression("kubernetes.io/hostname", "In", nodes)))) + }))) + // TODO: Use non-annotation syntax when we switch to K8s version 1.6. + logDebug(s"Adding nodeAffinity as annotation $nodeAffinityJson") + new PodBuilder(baseExecutorPod) + .editMetadata() + .addToAnnotations(ANNOTATION_EXECUTOR_NODE_AFFINITY, nodeAffinityJson) + .endMetadata() + .build() + } else { + baseExecutorPod + } + } +} + +// These case classes model K8s node affinity syntax fo +// preferredDuringSchedulingIgnoredDuringExecution. +// see https://kubernetes.io/docs/concepts/configuration/assign-pod-node +private case class SchedulerAffinity(nodeAffinity: NodeAffinity) +private case class NodeAffinity( + preferredDuringSchedulingIgnoredDuringExecution: Iterable[WeightedPreference]) +private case class WeightedPreference(weight: Int, preference: Preference) +private case class Preference(matchExpressions: Array[MatchExpression]) +private case class MatchExpression(key: String, operator: String, values: Iterable[String]) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala new file mode 100644 index 0000000000000..ca1bbbe17076f --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.kubernetes + +private[spark] case class ShuffleServiceConfig( + shuffleNamespace: String, + shuffleLabels: Map[String, String], + shuffleDirs: Seq[String]) From f28cb17b4047bb9c531926843f304eab4d46db8c Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Thu, 7 Sep 2017 11:43:58 -0700 Subject: [PATCH 02/25] Added configuration properties to inject arbitrary secrets into the driver/executors (#479) * Added configuration properties to inject arbitrary secrets into the driver/executors * Addressed comments --- docs/running-on-kubernetes.md | 16 +++++ .../spark/deploy/kubernetes/config.scala | 3 + ...DriverConfigurationStepsOrchestrator.scala | 21 +++++- .../submit/MountSecretsBootstrap.scala | 67 +++++++++++++++++++ .../submit/submitsteps/MountSecretsStep.scala | 37 ++++++++++ .../kubernetes/ExecutorPodFactory.scala | 17 +++-- .../kubernetes/KubernetesClusterManager.scala | 22 ++++-- .../KubernetesClusterSchedulerBackend.scala | 10 ++- ...rConfigurationStepsOrchestratorSuite.scala | 28 +++++++- .../submit/MountSecretsBootstrapSuite.scala | 46 +++++++++++++ .../kubernetes/submit/SecretVolumeUtils.scala | 36 ++++++++++ .../submitsteps/MountSecretsStepSuite.scala | 46 +++++++++++++ 12 files changed, 331 insertions(+), 18 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index cbaab84379db0..65c8763b55d17 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -800,6 +800,22 @@ from the other deployment modes. See the [configuration page](configuration.html the Driver process. The user can specify multiple of these to set multiple environment variables. + + spark.kubernetes.driver.secrets.[SecretName] + (none) + + Mounts the Kubernetes secret named SecretName onto the path specified by the value + in the driver Pod. The user can specify multiple instances of this for multiple secrets. + + + + spark.kubernetes.executor.secrets.[SecretName] + (none) + + Mounts the Kubernetes secret named SecretName onto the path specified by the value + in the executor Pods. The user can specify multiple instances of this for multiple secrets. + + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala index 53a184cba7a4d..9dfd13e1817f8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala @@ -152,6 +152,9 @@ package object config extends Logging { .stringConf .createOptional + private[spark] val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." + private[spark] val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." + private[spark] val KUBERNETES_DRIVER_POD_NAME = ConfigBuilder("spark.kubernetes.driver.pod.name") .doc("Name of the driver pod.") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index b66da0b154698..1bb336fa616d0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -83,6 +83,11 @@ private[spark] class DriverConfigurationStepsOrchestrator( val allDriverLabels = driverCustomLabels ++ Map( SPARK_APP_ID_LABEL -> kubernetesAppId, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val driverSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs( + submissionSparkConf, + KUBERNETES_DRIVER_SECRETS_PREFIX, + "driver secrets") + val initialSubmissionStep = new BaseDriverConfigurationStep( kubernetesAppId, kubernetesResourceNamePrefix, @@ -92,8 +97,10 @@ private[spark] class DriverConfigurationStepsOrchestrator( mainClass, appArgs, submissionSparkConf) + val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -153,17 +160,27 @@ private[spark] class DriverConfigurationStepsOrchestrator( } else { (filesDownloadPath, Seq.empty[DriverConfigurationStep]) } + val dependencyResolutionStep = new DependencyResolutionStep( sparkJars, sparkFiles, jarsDownloadPath, localFilesDownloadPath) + + val mountSecretsStep = if (driverSecretNamesToMountPaths.nonEmpty) { + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(driverSecretNamesToMountPaths) + Some(new MountSecretsStep(mountSecretsBootstrap)) + } else { + None + } + Seq( initialSubmissionStep, kubernetesCredentialsStep, dependencyResolutionStep) ++ submittedDependenciesBootstrapSteps ++ - pythonStep.toSeq + pythonStep.toSeq ++ + mountSecretsStep.toSeq } private def areAnyFilesNonContainerLocal(files: Seq[String]): Boolean = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala new file mode 100644 index 0000000000000..ae10c9390b221 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} + +/** + * Bootstraps a driver or executor pod with needed secrets mounted. + */ +private[spark] trait MountSecretsBootstrap { + + /** + * Mounts Kubernetes secrets as secret volumes into the given container in the given pod. + * + * @param pod the pod into which the secret volumes are being added. + * @param container the container into which the secret volumes are being mounted. + * @return the updated pod and container with the secrets mounted. + */ + def mountSecrets(pod: Pod, container: Container): (Pod, Container) +} + +private[spark] class MountSecretsBootstrapImpl( + secretNamesToMountPaths: Map[String, String]) extends MountSecretsBootstrap { + + override def mountSecrets(pod: Pod, container: Container): (Pod, Container) = { + var podBuilder = new PodBuilder(pod) + secretNamesToMountPaths.keys.foreach(name => + podBuilder = podBuilder + .editOrNewSpec() + .addNewVolume() + .withName(secretVolumeName(name)) + .withNewSecret() + .withSecretName(name) + .endSecret() + .endVolume() + .endSpec()) + + var containerBuilder = new ContainerBuilder(container) + secretNamesToMountPaths.foreach(namePath => + containerBuilder = containerBuilder + .addNewVolumeMount() + .withName(secretVolumeName(namePath._1)) + .withMountPath(namePath._2) + .endVolumeMount() + ) + + (podBuilder.build(), containerBuilder.build()) + } + + private def secretVolumeName(secretName: String): String = { + secretName + "-volume" + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala new file mode 100644 index 0000000000000..d20865daba3e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import org.apache.spark.deploy.kubernetes.submit.MountSecretsBootstrap + +/** + * A driver configuration step for mounting user-specified secrets onto user-specified paths. + * + * @param mountSecretsBootstrap a utility actually handling mounting of the secrets. + */ +private[spark] class MountSecretsStep( + mountSecretsBootstrap: MountSecretsBootstrap) extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets(driverSpec.driverPod, driverSpec.driverContainer) + driverSpec.copy( + driverPod = driverPodWithSecretsMounted, + driverContainer = driverContainerWithSecretsMounted + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala index 6355afa0a5041..caf1f45521da5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala @@ -16,17 +16,16 @@ */ package org.apache.spark.scheduler.cluster.kubernetes -import java.util.concurrent.atomic.AtomicLong +import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} import org.apache.commons.io.FilenameUtils -import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} +import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} import org.apache.spark.util.Utils // Configures executor pods. Construct one of these with a SparkConf to set up properties that are @@ -45,6 +44,7 @@ private[spark] trait ExecutorPodFactory { private[spark] class ExecutorPodFactoryImpl( sparkConf: SparkConf, nodeAffinityExecutorPodModifier: NodeAffinityExecutorPodModifier, + mountSecretsBootstrap: Option[MountSecretsBootstrap], mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) @@ -250,11 +250,18 @@ private[spark] class ExecutorPodFactoryImpl( .build() } }.getOrElse(executorPod) + + val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = + mountSecretsBootstrap.map {bootstrap => + bootstrap.mountSecrets(withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) + }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) + val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = mountSmallFilesBootstrap.map { bootstrap => bootstrap.mountSmallFilesSecret( - withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) - }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) + withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) + }.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)) + val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = executorInitContainerBootstrap.map { bootstrap => val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index f63d0aeabad3b..999824d1bc1d0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -21,10 +21,10 @@ import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrapImpl +import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} @@ -51,6 +51,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET) val maybeExecutorInitContainerSecretMountPath = sparkConf.get(EXECUTOR_INIT_CONTAINER_SECRET_MOUNT_DIR) + val executorInitContainerSecretVolumePlugin = for { initContainerSecretName <- maybeExecutorInitContainerSecretName initContainerSecretMountPath <- maybeExecutorInitContainerSecretMountPath @@ -59,10 +60,11 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit initContainerSecretName, initContainerSecretMountPath) } + // Only set up the bootstrap if they've provided both the config map key and the config map // name. The config map might not be provided if init-containers aren't being used to // bootstrap dependencies. - val executorInitContainerbootStrap = for { + val executorInitContainerBootstrap = for { configMap <- maybeInitContainerConfigMap configMapKey <- maybeInitContainerConfigMapKey } yield { @@ -75,12 +77,22 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit configMap, configMapKey) } + val mountSmallFilesBootstrap = for { secretName <- maybeSubmittedFilesSecret secretMountPath <- maybeSubmittedFilesSecretMountPath } yield { new MountSmallFilesBootstrapImpl(secretName, secretMountPath) } + + val executorSecretNamesToMountPaths = ConfigurationUtils.parsePrefixedKeyValuePairs(sparkConf, + KUBERNETES_EXECUTOR_SECRETS_PREFIX, "executor secrets") + val mountSecretBootstrap = if (executorSecretNamesToMountPaths.nonEmpty) { + Some(new MountSecretsBootstrapImpl(executorSecretNamesToMountPaths)) + } else { + None + } + if (maybeInitContainerConfigMap.isEmpty) { logWarning("The executor's init-container config map was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") @@ -89,6 +101,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit logWarning("The executor's init-container config map key was not specified. Executors will" + " therefore not attempt to fetch remote or submitted dependencies.") } + val kubernetesClient = SparkKubernetesClientFactory.createKubernetesClient( KUBERNETES_MASTER_INTERNAL_URL, Some(sparkConf.get(KUBERNETES_NAMESPACE)), @@ -99,8 +112,9 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val executorPodFactory = new ExecutorPodFactoryImpl( sparkConf, NodeAffinityExecutorPodModifierImpl, + mountSecretBootstrap, mountSmallFilesBootstrap, - executorInitContainerbootStrap, + executorInitContainerBootstrap, executorInitContainerSecretVolumePlugin) new KubernetesClusterSchedulerBackend( sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 20fe23fbcd92f..a0a7e71a66485 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -22,16 +22,14 @@ import java.util.Collections import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} -import com.fasterxml.jackson.databind.ObjectMapper -import com.fasterxml.jackson.module.scala.DefaultScalaModule -import io.fabric8.kubernetes.api.model._ -import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} -import io.fabric8.kubernetes.client.Watcher.Action -import org.apache.commons.io.FilenameUtils import scala.collection.{concurrent, mutable} import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action + import org.apache.spark.{SparkContext, SparkEnv, SparkException} import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala index c168e7b5407ba..6bad594629f76 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.kubernetes.submit.submitsteps._ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -29,6 +29,9 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" private val APP_ARGS = Array("arg1", "arg2") private val ADDITIONAL_PYTHON_FILES = Seq("local:///var/apps/python/py1.py") + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" test("Base submission steps without an init-container or python files.") { val sparkConf = new SparkConf(false) @@ -116,6 +119,29 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[MountSmallLocalFilesStep]) } + test("Submission steps with driver secrets to mount") { + val sparkConf = new SparkConf(false) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_FOO", SECRET_MOUNT_PATH) + .set(s"$KUBERNETES_DRIVER_SECRETS_PREFIX$SECRET_BAR", SECRET_MOUNT_PATH) + val mainAppResource = JavaMainAppResource("local:///var/apps/jars/main.jar") + val orchestrator = new DriverConfigurationStepsOrchestrator( + NAMESPACE, + APP_ID, + LAUNCH_TIME, + mainAppResource, + APP_NAME, + MAIN_CLASS, + APP_ARGS, + ADDITIONAL_PYTHON_FILES, + sparkConf) + validateStepTypes( + orchestrator, + classOf[BaseDriverConfigurationStep], + classOf[DriverKubernetesCredentialsStep], + classOf[DependencyResolutionStep], + classOf[MountSecretsStep]) + } + private def validateStepTypes( orchestrator: DriverConfigurationStepsOrchestrator, types: Class[_ <: DriverConfigurationStep]*): Unit = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala new file mode 100644 index 0000000000000..a23ee667004a2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkFunSuite + +private[spark] class MountSecretsBootstrapSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("Mounts all given secrets") { + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val driverContainer = new ContainerBuilder().build() + val driverPod = new PodBuilder().build() + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val (driverPodWithSecretsMounted, driverContainerWithSecretsMounted) = + mountSecretsBootstrap.mountSecrets(driverPod, driverContainer) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala new file mode 100644 index 0000000000000..860bc6e0438aa --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Container, Pod} + +private[spark] object SecretVolumeUtils { + + def podHasVolume(driverPod: Pod, volumeName: String): Boolean = { + driverPod.getSpec.getVolumes.asScala.exists(volume => volume.getName == volumeName) + } + + def containerHasVolume( + driverContainer: Container, + volumeName: String, + mountPath: String): Boolean = { + driverContainer.getVolumeMounts.asScala.exists(volumeMount => + volumeMount.getName == volumeName && volumeMount.getMountPath == mountPath) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala new file mode 100644 index 0000000000000..b94e7345cd6e1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, SecretVolumeUtils} + +private[spark] class MountSecretsStepSuite extends SparkFunSuite { + + private val SECRET_FOO = "foo" + private val SECRET_BAR = "bar" + private val SECRET_MOUNT_PATH = "/etc/secrets/driver" + + test("Mounts all given secrets") { + val baseDriverSpec = KubernetesDriverSpec.initialSpec(new SparkConf(false)) + val secretNamesToMountPaths = Map( + SECRET_FOO -> SECRET_MOUNT_PATH, + SECRET_BAR -> SECRET_MOUNT_PATH) + + val mountSecretsBootstrap = new MountSecretsBootstrapImpl(secretNamesToMountPaths) + val mountSecretsStep = new MountSecretsStep(mountSecretsBootstrap) + val configuredDriverSpec = mountSecretsStep.configureDriver(baseDriverSpec) + val driverPodWithSecretsMounted = configuredDriverSpec.driverPod + val driverContainerWithSecretsMounted = configuredDriverSpec.driverContainer + + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.podHasVolume(driverPodWithSecretsMounted, volumeName))) + Seq(s"$SECRET_FOO-volume", s"$SECRET_BAR-volume").foreach(volumeName => + assert(SecretVolumeUtils.containerHasVolume( + driverContainerWithSecretsMounted, volumeName, SECRET_MOUNT_PATH))) + } +} From c0e3a5e3ea987579d7221a4fa52111003dcf0bcb Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 7 Sep 2017 15:17:46 -0700 Subject: [PATCH 03/25] Extract more of the shuffle management to a different class. (#454) * Extract more of the shuffle management to a different class. More efforts to reduce the complexity of the KubernetesClusterSchedulerBackend. The scheduler backend should not be concerned about anything other than the coordination of the executor lifecycle. * Fix scalastyle * Add override annotation * Fix Java style * Remove unused imports. * Move volume index to the beginning to satisfy index * Address PR comments. --- .../KubernetesExternalShuffleClient.java | 61 +------ .../KubernetesExternalShuffleClientImpl.java | 77 +++++++++ .../kubernetes/ExecutorPodFactory.scala | 41 +---- .../kubernetes/KubernetesClusterManager.scala | 19 ++- .../KubernetesClusterSchedulerBackend.scala | 95 ++--------- .../KubernetesExternalShuffleManager.scala | 158 ++++++++++++++++++ .../cluster/kubernetes/ShufflePodCache.scala | 93 ----------- ...ubernetesExternalShuffleServiceSuite.scala | 4 +- 8 files changed, 286 insertions(+), 262 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java index c034e6c04a5f2..e9e94c1855a53 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClient.java @@ -17,64 +17,13 @@ package org.apache.spark.network.shuffle.kubernetes; -import org.apache.spark.network.client.RpcResponseCallback; -import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.sasl.SecretKeyHolder; -import org.apache.spark.network.shuffle.ExternalShuffleClient; -import org.apache.spark.network.shuffle.protocol.RegisterDriver; -import org.apache.spark.network.util.TransportConf; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - +import java.io.Closeable; import java.io.IOException; -import java.nio.ByteBuffer; - -/** - * A client for talking to the external shuffle service in Kubernetes cluster mode. - * - * This is used by the each Spark executor to register with a corresponding external - * shuffle service on the cluster. The purpose is for cleaning up shuffle files - * reliably if the application exits unexpectedly. - */ -public class KubernetesExternalShuffleClient extends ExternalShuffleClient { - private static final Logger logger = LoggerFactory - .getLogger(KubernetesExternalShuffleClient.class); - - /** - * Creates an Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}. - * Please refer to docs on {@link ExternalShuffleClient} for more information. - */ - public KubernetesExternalShuffleClient( - TransportConf conf, - SecretKeyHolder secretKeyHolder, - boolean authEnabled, - long registrationTimeoutMs) { - super(conf, secretKeyHolder, authEnabled, registrationTimeoutMs); - } - - public void registerDriverWithShuffleService(String host, int port) - throws IOException, InterruptedException { - checkInit(); - ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer(); - TransportClient client = clientFactory.createClient(host, port); - client.sendRpc(registerDriver, new RegisterDriverCallback()); - } - private class RegisterDriverCallback implements RpcResponseCallback { - @Override - public void onSuccess(ByteBuffer response) { - logger.info("Successfully registered app " + appId + " with external shuffle service."); - } +public interface KubernetesExternalShuffleClient extends Closeable { - @Override - public void onFailure(Throwable e) { - logger.warn("Unable to register app " + appId + " with external shuffle service. " + - "Please manually remove shuffle data after driver exit. Error: " + e); - } - } + void init(String appId); - @Override - public void close() { - super.close(); - } + void registerDriverWithShuffleService(String host, int port) + throws IOException, InterruptedException; } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java new file mode 100644 index 0000000000000..4302eff0c31f2 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java @@ -0,0 +1,77 @@ +/* + * 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.network.shuffle.kubernetes; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.shuffle.ExternalShuffleClient; +import org.apache.spark.network.shuffle.protocol.RegisterDriver; +import org.apache.spark.network.util.TransportConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * A client for talking to the external shuffle service in Kubernetes cluster mode. + * + * This is used by the each Spark executor to register with a corresponding external + * shuffle service on the cluster. The purpose is for cleaning up shuffle files + * reliably if the application exits unexpectedly. + */ +public class KubernetesExternalShuffleClientImpl + extends ExternalShuffleClient implements KubernetesExternalShuffleClient { + + private static final Logger logger = LoggerFactory + .getLogger(KubernetesExternalShuffleClientImpl.class); + + /** + * Creates a Kubernetes external shuffle client that wraps the {@link ExternalShuffleClient}. + * Please refer to docs on {@link ExternalShuffleClient} for more information. + */ + public KubernetesExternalShuffleClientImpl( + TransportConf conf, + SecretKeyHolder secretKeyHolder, + boolean saslEnabled) { + super(conf, secretKeyHolder, saslEnabled); + } + + @Override + public void registerDriverWithShuffleService(String host, int port) + throws IOException, InterruptedException { + checkInit(); + ByteBuffer registerDriver = new RegisterDriver(appId, 0).toByteBuffer(); + TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(registerDriver, new RegisterDriverCallback()); + } + + private class RegisterDriverCallback implements RpcResponseCallback { + @Override + public void onSuccess(ByteBuffer response) { + logger.info("Successfully registered app " + appId + " with external shuffle service."); + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Unable to register app " + appId + " with external shuffle service. " + + "Please manually remove shuffle data after driver exit. Error: " + e); + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala index caf1f45521da5..2339612658537 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala @@ -36,7 +36,6 @@ private[spark] trait ExecutorPodFactory { applicationId: String, driverUrl: String, executorEnvs: Seq[(String, String)], - shuffleServiceConfig: Option[ShuffleServiceConfig], driverPod: Pod, nodeToLocalTaskCount: Map[String, Int]): Pod } @@ -47,7 +46,8 @@ private[spark] class ExecutorPodFactoryImpl( mountSecretsBootstrap: Option[MountSecretsBootstrap], mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], - executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin]) + executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], + shuffleManager: Option[KubernetesExternalShuffleManager]) extends ExecutorPodFactory { import ExecutorPodFactoryImpl._ @@ -111,7 +111,6 @@ private[spark] class ExecutorPodFactoryImpl( applicationId: String, driverUrl: String, executorEnvs: Seq[(String, String)], - shuffleServiceConfig: Option[ShuffleServiceConfig], driverPod: Pod, nodeToLocalTaskCount: Map[String, Int]): Pod = { val name = s"$executorPodNamePrefix-exec-$executorId" @@ -179,6 +178,9 @@ private[spark] class ExecutorPodFactoryImpl( .withContainerPort(port._2) .build() }) + val shuffleVolumesWithMounts = + shuffleManager.map(_.getExecutorShuffleDirVolumesWithMounts) + .getOrElse(Seq.empty) val executorContainer = new ContainerBuilder() .withName(s"executor") @@ -191,6 +193,7 @@ private[spark] class ExecutorPodFactoryImpl( .endResources() .addAllToEnv(executorEnv.asJava) .withPorts(requiredPorts.asJava) + .addAllToVolumeMounts(shuffleVolumesWithMounts.map(_._2).asJava) .build() val executorPod = new PodBuilder() @@ -211,6 +214,7 @@ private[spark] class ExecutorPodFactoryImpl( .withHostname(hostname) .withRestartPolicy("Never") .withNodeSelector(nodeSelector.asJava) + .addAllToVolumes(shuffleVolumesWithMounts.map(_._1).asJava) .endSpec() .build() @@ -226,42 +230,15 @@ private[spark] class ExecutorPodFactoryImpl( .build() }.getOrElse(executorContainer) - val withMaybeShuffleConfigExecutorContainer = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(containerWithExecutorLimitCores) { (container, dir) => - new ContainerBuilder(container) - .addNewVolumeMount() - .withName(FilenameUtils.getBaseName(dir)) - .withMountPath(dir) - .endVolumeMount() - .build() - } - }.getOrElse(containerWithExecutorLimitCores) - val withMaybeShuffleConfigPod = shuffleServiceConfig.map { config => - config.shuffleDirs.foldLeft(executorPod) { (builder, dir) => - new PodBuilder(builder) - .editSpec() - .addNewVolume() - .withName(FilenameUtils.getBaseName(dir)) - .withNewHostPath() - .withPath(dir) - .endHostPath() - .endVolume() - .endSpec() - .build() - } - }.getOrElse(executorPod) - val (withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) = mountSecretsBootstrap.map {bootstrap => - bootstrap.mountSecrets(withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer) - }.getOrElse((withMaybeShuffleConfigPod, withMaybeShuffleConfigExecutorContainer)) - + bootstrap.mountSecrets(executorPod, containerWithExecutorLimitCores) + }.getOrElse((executorPod, containerWithExecutorLimitCores)) val (withMaybeSmallFilesMountedPod, withMaybeSmallFilesMountedContainer) = mountSmallFilesBootstrap.map { bootstrap => bootstrap.mountSmallFilesSecret( withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer) }.getOrElse((withMaybeSecretsMountedPod, withMaybeSecretsMountedContainer)) - val (executorPodWithInitContainer, initBootstrappedExecutorContainer) = executorInitContainerBootstrap.map { bootstrap => val podWithDetachedInitContainer = bootstrap.bootstrapInitContainerAndVolumes( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index 999824d1bc1d0..df2d94ec85216 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -26,7 +26,10 @@ import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} import org.apache.spark.internal.Logging +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.util.Utils private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { @@ -109,17 +112,31 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit sparkConf, Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) + + val kubernetesShuffleManager = if (Utils.isDynamicAllocationEnabled(sparkConf)) { + val kubernetesExternalShuffleClient = new KubernetesExternalShuffleClientImpl( + SparkTransportConf.fromSparkConf(sparkConf, "shuffle"), + sc.env.securityManager, + sc.env.securityManager.isAuthenticationEnabled()) + Some(new KubernetesExternalShuffleManagerImpl( + sparkConf, + kubernetesClient, + kubernetesExternalShuffleClient)) + } else None + val executorPodFactory = new ExecutorPodFactoryImpl( sparkConf, NodeAffinityExecutorPodModifierImpl, mountSecretBootstrap, mountSmallFilesBootstrap, executorInitContainerBootstrap, - executorInitContainerSecretVolumePlugin) + executorInitContainerSecretVolumePlugin, + kubernetesShuffleManager) new KubernetesClusterSchedulerBackend( sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], sc, executorPodFactory, + kubernetesShuffleManager, kubernetesClient) } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index a0a7e71a66485..1593a9a842e98 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -22,21 +22,16 @@ import java.util.Collections import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} -import scala.collection.{concurrent, mutable} -import scala.collection.JavaConverters._ -import scala.concurrent.{ExecutionContext, Future} - import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action +import scala.collection.{concurrent, mutable} +import scala.collection.JavaConverters._ +import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkContext, SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSmallFilesBootstrap} -import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig} @@ -47,6 +42,7 @@ private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, val sc: SparkContext, executorPodFactory: ExecutorPodFactory, + shuffleManager: Option[KubernetesExternalShuffleManager], kubernetesClient: KubernetesClient) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { @@ -66,7 +62,6 @@ private[spark] class KubernetesClusterSchedulerBackend( private val executorsToRemove = Collections.newSetFromMap[String]( new ConcurrentHashMap[String, java.lang.Boolean]()).asScala - private var shufflePodCache: Option[ShufflePodCache] = None private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) private val kubernetesDriverPodName = conf @@ -87,37 +82,6 @@ private[spark] class KubernetesClusterSchedulerBackend( throw new SparkException(s"Executor cannot find driver pod", throwable) } - private val shuffleServiceConfig: Option[ShuffleServiceConfig] = - if (Utils.isDynamicAllocationEnabled(sc.conf)) { - val shuffleNamespace = conf.get(KUBERNETES_SHUFFLE_NAMESPACE) - val parsedShuffleLabels = ConfigurationUtils.parseKeyValuePairs( - conf.get(KUBERNETES_SHUFFLE_LABELS), KUBERNETES_SHUFFLE_LABELS.key, - "shuffle-labels") - if (parsedShuffleLabels.isEmpty) { - throw new SparkException(s"Dynamic allocation enabled " + - s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified") - } - - val shuffleDirs = conf.get(KUBERNETES_SHUFFLE_DIR).map { - _.split(",") - }.getOrElse(Utils.getConfiguredLocalDirs(conf)) - Some( - ShuffleServiceConfig(shuffleNamespace, - parsedShuffleLabels, - shuffleDirs)) - } else { - None - } - - // A client for talking to the external shuffle service - private val kubernetesExternalShuffleClient: Option[KubernetesExternalShuffleClient] = { - if (Utils.isDynamicAllocationEnabled(sc.conf)) { - Some(getShuffleClient()) - } else { - None - } - } - override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { 0.8 @@ -220,14 +184,6 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - private def getShuffleClient(): KubernetesExternalShuffleClient = { - new KubernetesExternalShuffleClient( - SparkTransportConf.fromSparkConf(conf, "shuffle"), - sc.env.securityManager, - sc.env.securityManager.isAuthenticationEnabled(), - conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) - } - private def getInitialTargetExecutorNumber(defaultNumExecutors: Int = 1): Int = { if (Utils.isDynamicAllocationEnabled(conf)) { val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", 0) @@ -260,23 +216,17 @@ private[spark] class KubernetesClusterSchedulerBackend( allocator.scheduleWithFixedDelay( allocatorRunnable, 0, podAllocationInterval, TimeUnit.SECONDS) + shuffleManager.foreach(_.start(applicationId())) if (!Utils.isDynamicAllocationEnabled(sc.conf)) { doRequestTotalExecutors(initialExecutors) - } else { - shufflePodCache = shuffleServiceConfig - .map { config => new ShufflePodCache( - kubernetesClient, config.shuffleNamespace, config.shuffleLabels) } - shufflePodCache.foreach(_.start()) - kubernetesExternalShuffleClient.foreach(_.init(applicationId())) } } override def stop(): Unit = { // stop allocation of new resources and caches. allocator.shutdown() - shufflePodCache.foreach(_.stop()) - kubernetesExternalShuffleClient.foreach(_.close()) + shuffleManager.foreach(_.stop()) // send stop message to executors so they shut down cleanly super.stop() @@ -349,7 +299,6 @@ private[spark] class KubernetesClusterSchedulerBackend( applicationId(), driverUrl, sc.conf.getExecutorEnv, - shuffleServiceConfig, driverPod, nodeToLocalTaskCount) try { @@ -499,37 +448,27 @@ private[spark] class KubernetesClusterSchedulerBackend( new PartialFunction[Any, Unit]() { override def isDefinedAt(msg: Any): Boolean = { msg match { - case RetrieveSparkAppConfig(_) => - Utils.isDynamicAllocationEnabled(sc.conf) + case RetrieveSparkAppConfig(executorId) => + shuffleManager.isDefined case _ => false } } override def apply(msg: Any): Unit = { msg match { - case RetrieveSparkAppConfig(executorId) => - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - var resolvedProperties = sparkProperties - val runningExecutorPod = kubernetesClient + case RetrieveSparkAppConfig(executorId) if shuffleManager.isDefined => + val runningExecutorPod = RUNNING_EXECUTOR_PODS_LOCK.synchronized { + kubernetesClient .pods() .withName(runningExecutorsToPods(executorId).getMetadata.getName) .get() - val nodeName = runningExecutorPod.getSpec.getNodeName - val shufflePodIp = shufflePodCache.get.getShufflePodForExecutor(nodeName) - - // Inform the shuffle pod about this application so it can watch. - kubernetesExternalShuffleClient.foreach( - _.registerDriverWithShuffleService(shufflePodIp, externalShufflePort)) - - resolvedProperties = resolvedProperties ++ Seq( - (SPARK_SHUFFLE_SERVICE_HOST.key, shufflePodIp)) - - val reply = SparkAppConfig( - resolvedProperties, - SparkEnv.get.securityManager.getIOEncryptionKey(), - hadoopDelegationCreds) - context.reply(reply) } + val shuffleSpecificProperties = shuffleManager.get + .getShuffleServiceConfigurationForExecutor(runningExecutorPod) + val reply = SparkAppConfig( + sparkProperties ++ shuffleSpecificProperties, + SparkEnv.get.securityManager.getIOEncryptionKey()) + context.reply(reply) } } }.orElse(super.receiveAndReply(context)) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala new file mode 100644 index 0000000000000..d00783b84a948 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.kubernetes + +import io.fabric8.kubernetes.api.model.{Pod, Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} +import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.internal.readiness.Readiness +import org.apache.commons.io.FilenameUtils +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.kubernetes.ConfigurationUtils +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.internal.Logging +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient +import org.apache.spark.util.Utils + +private[spark] trait KubernetesExternalShuffleManager { + + def start(appId: String): Unit + + def stop(): Unit + + /** + * Returns the properties that should be applied for this executor pod, given that + * this executor will need to communicate with an external shuffle service. + * + * In practice, this seq will always have a size of 1, but since this method's semantics are that + * the returned values are key-value pairs to apply as properties, it is clearer to express + * this as a collection. + */ + def getShuffleServiceConfigurationForExecutor(executorPod: Pod): Seq[(String, String)] + + def getExecutorShuffleDirVolumesWithMounts: Seq[(Volume, VolumeMount)] + +} + +private[spark] class KubernetesExternalShuffleManagerImpl( + sparkConf: SparkConf, + client: KubernetesClient, + shuffleClient: KubernetesExternalShuffleClient) + extends KubernetesExternalShuffleManager with Logging { + + private val shuffleNamespace = sparkConf.get(KUBERNETES_SHUFFLE_NAMESPACE) + private val shufflePodLabels = ConfigurationUtils.parseKeyValuePairs( + sparkConf.get(KUBERNETES_SHUFFLE_LABELS), + KUBERNETES_SHUFFLE_LABELS.key, + "shuffle-labels") + if (shufflePodLabels.isEmpty) { + throw new SparkException(s"Dynamic allocation enabled " + + s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified") + } + private val externalShufflePort = sparkConf.getInt("spark.shuffle.service.port", 7337) + private val shuffleDirs = sparkConf.get(KUBERNETES_SHUFFLE_DIR).map { + _.split(",") + }.getOrElse(Utils.getConfiguredLocalDirs(sparkConf)) + private var shufflePodCache = scala.collection.mutable.Map[String, String]() + private var watcher: Watch = _ + + override def start(appId: String): Unit = { + // seed the initial cache. + val pods = client.pods() + .inNamespace(shuffleNamespace) + .withLabels(shufflePodLabels.asJava) + .list() + pods.getItems.asScala.foreach { + pod => + if (Readiness.isReady(pod)) { + addShufflePodToCache(pod) + } else { + logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + + s"on node ${pod.getSpec.getNodeName}") + } + } + + watcher = client + .pods() + .inNamespace(shuffleNamespace) + .withLabels(shufflePodLabels.asJava) + .watch(new Watcher[Pod] { + override def eventReceived(action: Watcher.Action, p: Pod): Unit = { + action match { + case Action.DELETED | Action.ERROR => + shufflePodCache.remove(p.getSpec.getNodeName) + case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => + addShufflePodToCache(p) + } + } + override def onClose(e: KubernetesClientException): Unit = {} + }) + shuffleClient.init(appId) + } + + private def addShufflePodToCache(pod: Pod): Unit = shufflePodCache.synchronized { + if (shufflePodCache.contains(pod.getSpec.getNodeName)) { + val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get + logError(s"Ambiguous specification of shuffle service pod. " + + s"Found multiple matching pods: ${pod.getMetadata.getName}, " + + s"${registeredPodName} on ${pod.getSpec.getNodeName}") + + throw new SparkException(s"Ambiguous specification of shuffle service pod. " + + s"Found multiple matching pods: ${pod.getMetadata.getName}, " + + s"${registeredPodName} on ${pod.getSpec.getNodeName}") + } else { + shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP + } + } + + override def stop(): Unit = { + watcher.close() + shuffleClient.close() + } + + override def getShuffleServiceConfigurationForExecutor(executorPod: Pod) + : Seq[(String, String)] = { + val nodeName = executorPod.getSpec.getNodeName + val shufflePodIp = shufflePodCache.synchronized { + shufflePodCache.get(nodeName).getOrElse( + throw new SparkException(s"Unable to find shuffle pod on node $nodeName")) + } + // Inform the shuffle pod about this application so it can watch. + shuffleClient.registerDriverWithShuffleService(shufflePodIp, externalShufflePort) + Seq((SPARK_SHUFFLE_SERVICE_HOST.key, shufflePodIp)) + } + + override def getExecutorShuffleDirVolumesWithMounts(): Seq[(Volume, VolumeMount)] = { + shuffleDirs.zipWithIndex.map { + case (shuffleDir, shuffleDirIndex) => + val volumeName = s"$shuffleDirIndex-${FilenameUtils.getBaseName(shuffleDir)}" + val volume = new VolumeBuilder() + .withName(volumeName) + .withNewHostPath(shuffleDir) + .build() + val volumeMount = new VolumeMountBuilder() + .withName(volumeName) + .withMountPath(shuffleDir) + .build() + (volume, volumeMount) + } + } +} + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala deleted file mode 100644 index 15e02664589eb..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShufflePodCache.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.kubernetes - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.Pod -import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} -import io.fabric8.kubernetes.client.Watcher.Action -import io.fabric8.kubernetes.client.internal.readiness.Readiness - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging - -private[spark] class ShufflePodCache ( - client: KubernetesClient, - dsNamespace: String, - dsLabels: Map[String, String]) extends Logging { - - private var shufflePodCache = scala.collection.mutable.Map[String, String]() - private var watcher: Watch = _ - - def start(): Unit = { - // seed the initial cache. - val pods = client.pods() - .inNamespace(dsNamespace).withLabels(dsLabels.asJava).list() - pods.getItems.asScala.foreach { - pod => - if (Readiness.isReady(pod)) { - addShufflePodToCache(pod) - } else { - logWarning(s"Found unready shuffle pod ${pod.getMetadata.getName} " + - s"on node ${pod.getSpec.getNodeName}") - } - } - - watcher = client - .pods() - .inNamespace(dsNamespace) - .withLabels(dsLabels.asJava) - .watch(new Watcher[Pod] { - override def eventReceived(action: Watcher.Action, p: Pod): Unit = { - action match { - case Action.DELETED | Action.ERROR => - shufflePodCache.remove(p.getSpec.getNodeName) - case Action.ADDED | Action.MODIFIED if Readiness.isReady(p) => - addShufflePodToCache(p) - } - } - override def onClose(e: KubernetesClientException): Unit = {} - }) - } - - private def addShufflePodToCache(pod: Pod): Unit = { - if (shufflePodCache.contains(pod.getSpec.getNodeName)) { - val registeredPodName = shufflePodCache.get(pod.getSpec.getNodeName).get - logError(s"Ambiguous specification of shuffle service pod. " + - s"Found multiple matching pods: ${pod.getMetadata.getName}, " + - s"${registeredPodName} on ${pod.getSpec.getNodeName}") - - throw new SparkException(s"Ambiguous specification of shuffle service pod. " + - s"Found multiple matching pods: ${pod.getMetadata.getName}, " + - s"${registeredPodName} on ${pod.getSpec.getNodeName}") - } else { - shufflePodCache(pod.getSpec.getNodeName) = pod.getStatus.getPodIP - } - } - - def stop(): Unit = { - watcher.close() - } - - def getShufflePodForExecutor(executorNode: String): String = { - shufflePodCache.get(executorNode) - .getOrElse(throw new SparkException(s"Unable to find shuffle pod on node $executorNode")) - } -} - diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala index aea7c6798b5c4..213a62bb833e5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.internal.config import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient +import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite { @@ -33,7 +33,7 @@ private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite SPARK_CONF, new SecurityManager(SPARK_CONF)) - val shuffleClient = new KubernetesExternalShuffleClient( + val shuffleClient = new KubernetesExternalShuffleClientImpl( SparkTransportConf.fromSparkConf(SPARK_CONF, "shuffle"), new SecurityManager(SPARK_CONF), false, From 701fe97298f980a18181b8a8786ec6f0a84cbae7 Mon Sep 17 00:00:00 2001 From: mccheah Date: Thu, 7 Sep 2017 20:52:13 -0700 Subject: [PATCH 04/25] Unit Tests for KubernetesClusterSchedulerBackend (#459) * Start unit tests for the scheduler backend. * More tests for the scheduler backend. * Unit tests and possible preemptive corrections to failover logic. * Address PR comments. * Resolve merge conflicts. Move MiB change to ExecutorPodFactory. * Revert accidental thread pool name change --- .../kubernetes/KubernetesClusterManager.scala | 14 +- .../KubernetesClusterSchedulerBackend.scala | 170 ++++---- ...bernetesClusterSchedulerBackendSuite.scala | 383 ++++++++++++++++++ 3 files changed, 483 insertions(+), 84 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala index df2d94ec85216..764e351e70286 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterManager extends ExternalClusterManager with Logging { @@ -132,12 +132,18 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit executorInitContainerBootstrap, executorInitContainerSecretVolumePlugin, kubernetesShuffleManager) + val allocatorExecutor = ThreadUtils + .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") + val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( + "kubernetes-executor-requests") new KubernetesClusterSchedulerBackend( - sc.taskScheduler.asInstanceOf[TaskSchedulerImpl], - sc, + scheduler.asInstanceOf[TaskSchedulerImpl], + sc.env.rpcEnv, executorPodFactory, kubernetesShuffleManager, - kubernetesClient) + kubernetesClient, + allocatorExecutor, + requestExecutorsService) } override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 1593a9a842e98..54612c80cdf28 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster.kubernetes import java.io.Closeable import java.net.InetAddress import java.util.Collections -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.{ConcurrentHashMap, ExecutorService, ScheduledExecutorService, ThreadPoolExecutor, TimeUnit} import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import io.fabric8.kubernetes.api.model._ @@ -29,25 +29,28 @@ import scala.collection.{concurrent, mutable} import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} -import org.apache.spark.{SparkContext, SparkEnv, SparkException} +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.Utils private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, - val sc: SparkContext, + rpcEnv: RpcEnv, executorPodFactory: ExecutorPodFactory, shuffleManager: Option[KubernetesExternalShuffleManager], - kubernetesClient: KubernetesClient) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + kubernetesClient: KubernetesClient, + allocatorExecutor: ScheduledExecutorService, + requestExecutorsService: ExecutorService) + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { import KubernetesClusterSchedulerBackend._ + private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) private val RUNNING_EXECUTOR_PODS_LOCK = new Object // Indexed by executor IDs and guarded by RUNNING_EXECUTOR_PODS_LOCK. private val runningExecutorsToPods = new mutable.HashMap[String, Pod] @@ -57,10 +60,10 @@ private[spark] class KubernetesClusterSchedulerBackend( private val EXECUTOR_PODS_BY_IPS_LOCK = new Object // Indexed by executor IP addrs and guarded by EXECUTOR_PODS_BY_IPS_LOCK private val executorPodsByIPs = new mutable.HashMap[String, Pod] - private val failedPods: concurrent.Map[String, ExecutorExited] = new - ConcurrentHashMap[String, ExecutorExited]().asScala - private val executorsToRemove = Collections.newSetFromMap[String]( - new ConcurrentHashMap[String, java.lang.Boolean]()).asScala + private val podsWithKnownExitReasons: concurrent.Map[String, ExecutorExited] = + new ConcurrentHashMap[String, ExecutorExited]().asScala + private val disconnectedPodsByExecutorIdPendingRemoval = + new ConcurrentHashMap[String, Pod]().asScala private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) @@ -68,10 +71,8 @@ private[spark] class KubernetesClusterSchedulerBackend( .get(KUBERNETES_DRIVER_POD_NAME) .getOrElse( throw new SparkException("Must specify the driver pod name")) - private val executorPodNamePrefix = conf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-executor-requests")) + requestExecutorsService) private val driverPod = try { kubernetesClient.pods().inNamespace(kubernetesNamespace). @@ -93,9 +94,9 @@ private[spark] class KubernetesClusterSchedulerBackend( protected var totalExpectedExecutors = new AtomicInteger(0) private val driverUrl = RpcEndpointAddress( - sc.getConf.get("spark.driver.host"), - sc.getConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), - CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + conf.get("spark.driver.host"), + conf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val initialExecutors = getInitialTargetExecutorNumber() @@ -109,21 +110,14 @@ private[spark] class KubernetesClusterSchedulerBackend( s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + s"is ${podAllocationSize}, should be a positive integer") - private val allocator = ThreadUtils - .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") + private val allocatorRunnable = new Runnable { - private val allocatorRunnable: Runnable = new Runnable { - - // Number of times we are allowed check for the loss reason for an executor before we give up - // and assume the executor failed for good, and attribute it to a framework fault. - private val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 - private val executorsToRecover = new mutable.HashSet[String] // Maintains a map of executor id to count of checks performed to learn the loss reason // for an executor. - private val executorReasonChecks = new mutable.HashMap[String, Int] + private val executorReasonCheckAttemptCounts = new mutable.HashMap[String, Int] override def run(): Unit = { - removeFailedExecutors() + handleDisconnectedExecutors() RUNNING_EXECUTOR_PODS_LOCK.synchronized { if (totalRegisteredExecutors.get() < runningExecutorsToPods.size) { logDebug("Waiting for pending executors before scaling") @@ -132,7 +126,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } else { val nodeToLocalTaskCount = getNodesWithLocalTaskCounts for (i <- 0 until math.min( - totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { + totalExpectedExecutors.get - runningExecutorsToPods.size, podAllocationSize)) { val (executorId, pod) = allocateNewExecutorPod(nodeToLocalTaskCount) runningExecutorsToPods.put(executorId, pod) runningPodsToExecutors.put(pod.getMetadata.getName, executorId) @@ -143,43 +137,47 @@ private[spark] class KubernetesClusterSchedulerBackend( } } - def removeFailedExecutors(): Unit = { - val localRunningExecutorsToPods = RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.toMap - } - executorsToRemove.foreach { case (executorId) => - localRunningExecutorsToPods.get(executorId).map { pod: Pod => - failedPods.get(pod.getMetadata.getName).map { executorExited: ExecutorExited => - logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) - removeExecutor(executorId, executorExited) - if (!executorExited.exitCausedByApp) { - executorsToRecover.add(executorId) - } - }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId)) - }.getOrElse(removeExecutorOrIncrementLossReasonCheckCount(executorId)) - - executorsToRecover.foreach(executorId => { - executorsToRemove -= executorId - executorReasonChecks -= executorId - RUNNING_EXECUTOR_PODS_LOCK.synchronized { - runningExecutorsToPods.remove(executorId).map { pod: Pod => - kubernetesClient.pods().delete(pod) - runningPodsToExecutors.remove(pod.getMetadata.getName) - }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) + def handleDisconnectedExecutors(): Unit = { + // For each disconnected executor, synchronize with the loss reasons that may have been found + // by the executor pod watcher. If the loss reason was discovered by the watcher, + // inform the parent class with removeExecutor. + val disconnectedPodsByExecutorIdPendingRemovalCopy = + Map.empty ++ disconnectedPodsByExecutorIdPendingRemoval + disconnectedPodsByExecutorIdPendingRemovalCopy.foreach { case (executorId, executorPod) => + val knownExitReason = podsWithKnownExitReasons.remove(executorPod.getMetadata.getName) + knownExitReason.fold { + removeExecutorOrIncrementLossReasonCheckCount(executorId) + } { executorExited => + logDebug(s"Removing executor $executorId with loss reason " + executorExited.message) + removeExecutor(executorId, executorExited) + // We keep around executors that have exit conditions caused by the application. This + // allows them to be debugged later on. Otherwise, mark them as to be deleted from the + // the API server. + if (!executorExited.exitCausedByApp) { + deleteExecutorFromClusterAndDataStructures(executorId) } - }) - executorsToRecover.clear() + } } } def removeExecutorOrIncrementLossReasonCheckCount(executorId: String): Unit = { - val reasonCheckCount = executorReasonChecks.getOrElse(executorId, 0) - if (reasonCheckCount > MAX_EXECUTOR_LOST_REASON_CHECKS) { - removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons")) - executorsToRecover.add(executorId) - executorReasonChecks -= executorId + val reasonCheckCount = executorReasonCheckAttemptCounts.getOrElse(executorId, 0) + if (reasonCheckCount >= MAX_EXECUTOR_LOST_REASON_CHECKS) { + removeExecutor(executorId, SlaveLost("Executor lost for unknown reasons.")) + deleteExecutorFromClusterAndDataStructures(executorId) } else { - executorReasonChecks.put(executorId, reasonCheckCount + 1) + executorReasonCheckAttemptCounts.put(executorId, reasonCheckCount + 1) + } + } + + def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { + disconnectedPodsByExecutorIdPendingRemoval -= executorId + executorReasonCheckAttemptCounts -= executorId + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.remove(executorId).map { pod => + kubernetesClient.pods().delete(pod) + runningPodsToExecutors.remove(pod.getMetadata.getName) + }.getOrElse(logWarning(s"Unable to remove pod for unknown executor $executorId")) } } } @@ -214,18 +212,18 @@ private[spark] class KubernetesClusterSchedulerBackend( .withLabel(SPARK_APP_ID_LABEL, applicationId()) .watch(new ExecutorPodsWatcher())) - allocator.scheduleWithFixedDelay( - allocatorRunnable, 0, podAllocationInterval, TimeUnit.SECONDS) + allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable, 0L, podAllocationInterval, TimeUnit.SECONDS) shuffleManager.foreach(_.start(applicationId())) - if (!Utils.isDynamicAllocationEnabled(sc.conf)) { + if (!Utils.isDynamicAllocationEnabled(conf)) { doRequestTotalExecutors(initialExecutors) } } override def stop(): Unit = { // stop allocation of new resources and caches. - allocator.shutdown() + allocatorExecutor.shutdown() shuffleManager.foreach(_.stop()) // send stop message to executors so they shut down cleanly @@ -298,7 +296,7 @@ private[spark] class KubernetesClusterSchedulerBackend( executorId, applicationId(), driverUrl, - sc.conf.getExecutorEnv, + conf.getExecutorEnv, driverPod, nodeToLocalTaskCount) try { @@ -318,11 +316,14 @@ private[spark] class KubernetesClusterSchedulerBackend( override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { RUNNING_EXECUTOR_PODS_LOCK.synchronized { for (executor <- executorIds) { - runningExecutorsToPods.remove(executor) match { - case Some(pod) => - kubernetesClient.pods().delete(pod) - runningPodsToExecutors.remove(pod.getMetadata.getName) - case None => logWarning(s"Unable to remove pod for unknown executor $executor") + val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) + maybeRemovedExecutor.foreach { executorPod => + kubernetesClient.pods().delete(executorPod) + disconnectedPodsByExecutorIdPendingRemoval(executor) = executorPod + runningPodsToExecutors.remove(executorPod.getMetadata.getName) + } + if (maybeRemovedExecutor.isEmpty) { + logWarning(s"Unable to remove pod for unknown executor $executor") } } } @@ -396,10 +397,9 @@ private[spark] class KubernetesClusterSchedulerBackend( } def handleErroredPod(pod: Pod): Unit = { - val alreadyReleased = isPodAlreadyReleased(pod) val containerExitStatus = getExecutorExitStatus(pod) // container was probably actively killed by the driver. - val exitReason = if (alreadyReleased) { + val exitReason = if (isPodAlreadyReleased(pod)) { ExecutorExited(containerExitStatus, exitCausedByApp = false, s"Container in pod " + pod.getMetadata.getName + " exited from explicit termination request.") @@ -411,17 +411,23 @@ private[spark] class KubernetesClusterSchedulerBackend( // Here we can't be sure that that exit was caused by the application but this seems // to be the right default since we know the pod was not explicitly deleted by // the user. - "Pod exited with following container exit status code " + containerExitStatus + s"Pod ${pod.getMetadata.getName}'s executor container exited with exit status" + + s" code $containerExitStatus." } ExecutorExited(containerExitStatus, exitCausedByApp = true, containerExitReason) } - failedPods.put(pod.getMetadata.getName, exitReason) + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) } def handleDeletedPod(pod: Pod): Unit = { - val exitReason = ExecutorExited(getExecutorExitStatus(pod), exitCausedByApp = false, - "Pod " + pod.getMetadata.getName + " deleted or lost.") - failedPods.put(pod.getMetadata.getName, exitReason) + val exitMessage = if (isPodAlreadyReleased(pod)) { + s"Container in pod ${pod.getMetadata.getName} exited from explicit termination request." + } else { + s"Pod ${pod.getMetadata.getName} deleted or lost." + } + val exitReason = ExecutorExited( + getExecutorExitStatus(pod), exitCausedByApp = false, exitMessage) + podsWithKnownExitReasons.put(pod.getMetadata.getName, exitReason) } } @@ -433,12 +439,15 @@ private[spark] class KubernetesClusterSchedulerBackend( rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends DriverEndpoint(rpcEnv, sparkProperties) { - private val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337) override def onDisconnected(rpcAddress: RpcAddress): Unit = { addressToExecutorId.get(rpcAddress).foreach { executorId => if (disableExecutor(executorId)) { - executorsToRemove.add(executorId) + RUNNING_EXECUTOR_PODS_LOCK.synchronized { + runningExecutorsToPods.get(executorId).foreach { pod => + disconnectedPodsByExecutorIdPendingRemoval(executorId) = pod + } + } } } } @@ -448,7 +457,7 @@ private[spark] class KubernetesClusterSchedulerBackend( new PartialFunction[Any, Unit]() { override def isDefinedAt(msg: Any): Boolean = { msg match { - case RetrieveSparkAppConfig(executorId) => + case RetrieveSparkAppConfig(_) => shuffleManager.isDefined case _ => false } @@ -477,11 +486,12 @@ private[spark] class KubernetesClusterSchedulerBackend( } private object KubernetesClusterSchedulerBackend { - private val DEFAULT_STATIC_PORT = 10000 - private val EXECUTOR_ID_COUNTER = new AtomicLong(0L) private val VMEM_EXCEEDED_EXIT_CODE = -103 private val PMEM_EXCEEDED_EXIT_CODE = -104 private val UNKNOWN_EXIT_CODE = -111 + // Number of times we are allowed check for the loss reason for an executor before we give up + // and assume the executor failed for good, and attribute it to a framework fault. + val MAX_EXECUTOR_LOST_REASON_CHECKS = 10 def memLimitExceededLogMessage(diagnostics: String): String = { s"Pod/Container killed for exceeding memory limits. $diagnostics" + diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..b30d1c2543bea --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala @@ -0,0 +1,383 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.kubernetes + +import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} + +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder, PodList} +import io.fabric8.kubernetes.client.{KubernetesClient, Watch, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action +import io.fabric8.kubernetes.client.dsl.{FilterWatchListDeletable, MixedOperation, NonNamespaceOperation, PodResource} +import org.mockito.{AdditionalAnswers, ArgumentCaptor, Mock, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito.{doNothing, never, times, verify, when} +import org.scalatest.BeforeAndAfter +import org.scalatest.mock.MockitoSugar._ +import scala.collection.JavaConverters._ +import scala.concurrent.Future + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpoint, RpcEndpointAddress, RpcEndpointRef, RpcEnv, RpcTimeout} +import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend + +private[spark] class KubernetesClusterSchedulerBackendSuite + extends SparkFunSuite with BeforeAndAfter { + + private val APP_ID = "test-spark-app" + private val DRIVER_POD_NAME = "spark-driver-pod" + private val NAMESPACE = "test-namespace" + private val SPARK_DRIVER_HOST = "localhost" + private val SPARK_DRIVER_PORT = 7077 + private val POD_ALLOCATION_INTERVAL = 60L + private val DRIVER_URL = RpcEndpointAddress( + SPARK_DRIVER_HOST, SPARK_DRIVER_PORT, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + private val FIRST_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod1") + .endMetadata() + .withNewSpec() + .withNodeName("node1") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private val SECOND_EXECUTOR_POD = new PodBuilder() + .withNewMetadata() + .withName("pod2") + .endMetadata() + .withNewSpec() + .withNodeName("node2") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.101") + .endStatus() + .build() + + private type PODS = MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + private type LABELLED_PODS = FilterWatchListDeletable[ + Pod, PodList, java.lang.Boolean, Watch, Watcher[Pod]] + private type IN_NAMESPACE_PODS = NonNamespaceOperation[ + Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + + @Mock + private var sparkContext: SparkContext = _ + + @Mock + private var listenerBus: LiveListenerBus = _ + + @Mock + private var taskSchedulerImpl: TaskSchedulerImpl = _ + + @Mock + private var allocatorExecutor: ScheduledExecutorService = _ + + @Mock + private var requestExecutorsService: ExecutorService = _ + + @Mock + private var executorPodFactory: ExecutorPodFactory = _ + + @Mock + private var shuffleManager: KubernetesExternalShuffleManager = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var podOperations: PODS = _ + + @Mock + private var podsWithLabelOperations: LABELLED_PODS = _ + + @Mock + private var podsInNamespace: IN_NAMESPACE_PODS = _ + + @Mock + private var podsWithDriverName: PodResource[Pod, DoneablePod] = _ + + @Mock + private var rpcEnv: RpcEnv = _ + + @Mock + private var driverEndpointRef: RpcEndpointRef = _ + + @Mock + private var executorPodsWatch: Watch = _ + + private var sparkConf: SparkConf = _ + private var executorPodsWatcherArgument: ArgumentCaptor[Watcher[Pod]] = _ + private var allocatorRunnable: ArgumentCaptor[Runnable] = _ + private var requestExecutorRunnable: ArgumentCaptor[Runnable] = _ + private var driverEndpoint: ArgumentCaptor[RpcEndpoint] = _ + + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(DRIVER_POD_NAME) + .addToLabels(SPARK_APP_ID_LABEL, APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .endMetadata() + .build() + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf() + .set("spark.app.id", APP_ID) + .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) + .set(KUBERNETES_NAMESPACE, NAMESPACE) + .set("spark.driver.host", SPARK_DRIVER_HOST) + .set("spark.driver.port", SPARK_DRIVER_PORT.toString) + .set(KUBERNETES_ALLOCATION_BATCH_DELAY, POD_ALLOCATION_INTERVAL) + executorPodsWatcherArgument = ArgumentCaptor.forClass(classOf[Watcher[Pod]]) + allocatorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + requestExecutorRunnable = ArgumentCaptor.forClass(classOf[Runnable]) + driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) + when(sparkContext.conf).thenReturn(sparkConf) + when(sparkContext.listenerBus).thenReturn(listenerBus) + when(taskSchedulerImpl.sc).thenReturn(sparkContext) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.withLabel(SPARK_APP_ID_LABEL, APP_ID)).thenReturn(podsWithLabelOperations) + when(podsWithLabelOperations.watch(executorPodsWatcherArgument.capture())) + .thenReturn(executorPodsWatch) + when(podOperations.inNamespace(NAMESPACE)).thenReturn(podsInNamespace) + when(podsInNamespace.withName(DRIVER_POD_NAME)).thenReturn(podsWithDriverName) + when(podsWithDriverName.get()).thenReturn(driverPod) + when(allocatorExecutor.scheduleWithFixedDelay( + allocatorRunnable.capture(), + mockitoEq(0L), + mockitoEq(POD_ALLOCATION_INTERVAL), + mockitoEq(TimeUnit.SECONDS))).thenReturn(null) + // Creating Futures in Scala backed by a Java executor service resolves to running + // ExecutorService#execute (as opposed to submit) + doNothing().when(requestExecutorsService).execute(requestExecutorRunnable.capture()) + when(rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + .thenReturn(driverEndpointRef) + when(driverEndpointRef.ask[Boolean] + (any(classOf[Any])) + (any())).thenReturn(mock[Future[Boolean]]) + } + + test("Basic lifecycle expectations when starting and stopping the scheduler.") { + val scheduler = newSchedulerBackend(true) + scheduler.start() + verify(shuffleManager).start(APP_ID) + assert(executorPodsWatcherArgument.getValue != null) + assert(allocatorRunnable.getValue != null) + scheduler.stop() + verify(shuffleManager).stop() + verify(executorPodsWatch).close() + } + + test("Static allocation should request executors upon first allocator run.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Killing executors deletes the executor pods") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 2) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + allocatorRunnable.getValue.run() + scheduler.doKillExecutors(Seq("2")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations).delete(SECOND_EXECUTOR_POD) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should be requested in batches.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 2) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).create(FIRST_EXECUTOR_POD) + verify(podOperations, never()).create(SECOND_EXECUTOR_POD) + val registerFirstExecutorMessage = RegisterExecutor( + "1", mock[RpcEndpointRef], "localhost", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + allocatorRunnable.getValue.run() + verify(podOperations).create(SECOND_EXECUTOR_POD) + } + + test("Deleting executors and then running an allocator pass after finding the loss reason" + + " should only delete the pod once.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + requestExecutorRunnable.getValue.run() + when(podOperations.create(any(classOf[Pod]))) + .thenAnswer(AdditionalAnswers.returnsFirstArg()) + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + scheduler.doRequestTotalExecutors(0) + requestExecutorRunnable.getAllValues.asScala.last.run() + scheduler.doKillExecutors(Seq("1")) + requestExecutorRunnable.getAllValues.asScala.last.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + + val exitedPod = exitPod(FIRST_EXECUTOR_POD, 0) + executorPodsWatcherArgument.getValue.eventReceived(Action.DELETED, exitedPod) + allocatorRunnable.getValue.run() + verify(podOperations, times(1)).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef, times(1)).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 0, + exitCausedByApp = false, + s"Container in pod ${exitedPod.getMetadata.getName} exited from" + + s" explicit termination request."))) + } + + test("Executors that disconnect from application errors are noted as exits caused by app.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + executorPodsWatcherArgument.getValue.eventReceived( + Action.ERROR, exitPod(FIRST_EXECUTOR_POD, 1)) + + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + scheduler.doRequestTotalExecutors(1) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getAllValues.asScala.last.run() + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", ExecutorExited( + 1, + exitCausedByApp = true, + s"Pod ${FIRST_EXECUTOR_POD.getMetadata.getName}'s executor container exited with" + + " exit status code 1."))) + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + + test("Executors should only try to get the loss reason a number of times before giving up and" + + " removing the executor.") { + sparkConf + .set(KUBERNETES_ALLOCATION_BATCH_SIZE, 1) + .set(org.apache.spark.internal.config.EXECUTOR_INSTANCES, 1) + val scheduler = newSchedulerBackend(true) + scheduler.start() + expectPodCreationWithId(1, FIRST_EXECUTOR_POD) + when(podOperations.create(any(classOf[Pod]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + requestExecutorRunnable.getValue.run() + allocatorRunnable.getValue.run() + val executorEndpointRef = mock[RpcEndpointRef] + when(executorEndpointRef.address).thenReturn(RpcAddress("pod.example.com", 9000)) + val registerFirstExecutorMessage = RegisterExecutor( + "1", executorEndpointRef, "localhost:9000", 1, Map.empty[String, String]) + when(taskSchedulerImpl.resourceOffers(any())).thenReturn(Seq.empty) + driverEndpoint.getValue.receiveAndReply(mock[RpcCallContext]) + .apply(registerFirstExecutorMessage) + driverEndpoint.getValue.onDisconnected(executorEndpointRef.address) + 1 to KubernetesClusterSchedulerBackend.MAX_EXECUTOR_LOST_REASON_CHECKS foreach { _ => + allocatorRunnable.getValue.run() + verify(podOperations, never()).delete(FIRST_EXECUTOR_POD) + } + expectPodCreationWithId(2, SECOND_EXECUTOR_POD) + allocatorRunnable.getValue.run() + verify(podOperations).delete(FIRST_EXECUTOR_POD) + verify(driverEndpointRef).ask[Boolean]( + RemoveExecutor("1", SlaveLost("Executor lost for unknown reasons."))) + } + + private def newSchedulerBackend(externalShuffle: Boolean): KubernetesClusterSchedulerBackend = { + new KubernetesClusterSchedulerBackend( + taskSchedulerImpl, + rpcEnv, + executorPodFactory, + if (externalShuffle) Some(shuffleManager) else None, + kubernetesClient, + allocatorExecutor, + requestExecutorsService) + } + + private def exitPod(basePod: Pod, exitCode: Int): Pod = { + new PodBuilder(FIRST_EXECUTOR_POD) + .editStatus() + .addNewContainerStatus() + .withNewState() + .withNewTerminated() + .withExitCode(exitCode) + .endTerminated() + .endState() + .endContainerStatus() + .endStatus() + .build() + } + + private def expectPodCreationWithId(executorId: Int, expectedPod: Pod): Unit = { + when(executorPodFactory.createExecutorPod( + executorId.toString, + APP_ID, + DRIVER_URL, + sparkConf.getExecutorEnv, + driverPod, + Map.empty)).thenReturn(expectedPod) + } + +} From c0df462e34fe4b98a0cb456aa0db78b8488a485b Mon Sep 17 00:00:00 2001 From: mccheah Date: Fri, 8 Sep 2017 12:25:45 -0700 Subject: [PATCH 05/25] Use a headless service to give a hostname to the driver. (#483) * Use a headless service to give a hostname to the driver. Required since SPARK-21642 was added upstream. * Fix scalastyle. * Add back import * Fix conflict properly. * Fix orchestrator test. --- docs/running-on-kubernetes.md | 2 + .../spark/deploy/kubernetes/constants.scala | 2 +- ...DriverConfigurationStepsOrchestrator.scala | 11 +- .../DriverAddressConfigurationStep.scala | 102 ++++++++++ ...rConfigurationStepsOrchestratorSuite.scala | 7 +- .../DriverAddressConfigurationStepSuite.scala | 181 ++++++++++++++++++ 6 files changed, 300 insertions(+), 5 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 65c8763b55d17..9ddc4ee4319fb 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -24,6 +24,8 @@ should give you a list of pods and configmaps (if any) respectively. * You must have a spark distribution with Kubernetes support. This may be obtained from the [release tarball](https://github.com/apache-spark-on-k8s/spark/releases) or by [building Spark with Kubernetes support](../resource-managers/kubernetes/README.md#building-spark-with-kubernetes-support). +* You must have [Kubernetes DNS](https://kubernetes.io/docs/concepts/services-networking/dns-pod-service/) configured in +your cluster. ## Driver & Executor Images diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala index 2c2ccf31b9dd9..0a2bc46249f3a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala @@ -44,11 +44,11 @@ package object constants { private[spark] val DRIVER_CREDENTIALS_SECRET_VOLUME_NAME = "kubernetes-credentials" // Default and fixed ports - private[spark] val SUBMISSION_SERVER_PORT = 7077 private[spark] val DEFAULT_DRIVER_PORT = 7078 private[spark] val DEFAULT_BLOCKMANAGER_PORT = 7079 private[spark] val DEFAULT_UI_PORT = 4040 private[spark] val BLOCK_MANAGER_PORT_NAME = "blockmanager" + private[spark] val DRIVER_PORT_NAME = "driver-rpc-port" private[spark] val EXECUTOR_PORT_NAME = "executor" // Environment Variables diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala index 1bb336fa616d0..afc73e8f07601 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala @@ -20,10 +20,10 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.ConfigurationUtils import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher -import org.apache.spark.util.Utils +import org.apache.spark.util.{SystemClock, Utils} /** * Constructs the complete list of driver configuration steps to run to deploy the Spark driver. @@ -97,7 +97,11 @@ private[spark] class DriverConfigurationStepsOrchestrator( mainClass, appArgs, submissionSparkConf) - + val driverAddressStep = new DriverAddressConfigurationStep( + kubernetesResourceNamePrefix, + allDriverLabels, + submissionSparkConf, + new SystemClock) val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) @@ -176,6 +180,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( Seq( initialSubmissionStep, + driverAddressStep, kubernetesCredentialsStep, dependencyResolutionStep) ++ submittedDependenciesBootstrapSteps ++ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala new file mode 100644 index 0000000000000..85412ad19742b --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import io.fabric8.kubernetes.api.model.ServiceBuilder +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.internal.Logging +import org.apache.spark.util.Clock + +/** + * Allows the driver to be reachable by executor pods through a headless service. The service's + * ports should correspond to the ports that the executor will reach the pod at for RPC. + */ +private[spark] class DriverAddressConfigurationStep( + kubernetesResourceNamePrefix: String, + driverLabels: Map[String, String], + submissionSparkConf: SparkConf, + clock: Clock) extends DriverConfigurationStep with Logging { + import DriverAddressConfigurationStep._ + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + require(submissionSparkConf.getOption(DRIVER_BIND_ADDRESS_KEY).isEmpty, + s"$DRIVER_BIND_ADDRESS_KEY is not supported in Kubernetes mode, as the driver's hostname" + + s" will be managed via a Kubernetes service.") + require(submissionSparkConf.getOption(DRIVER_HOST_KEY).isEmpty, + s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be" + + s" managed via a Kubernetes service.") + + val preferredServiceName = s"$kubernetesResourceNamePrefix$DRIVER_SVC_POSTFIX" + val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { + preferredServiceName + } else { + val randomServiceId = clock.getTimeMillis() + val shorterServiceName = s"spark-$randomServiceId$DRIVER_SVC_POSTFIX" + logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is" + + s" too long (must be <= 63 characters). Falling back to use $shorterServiceName" + + s" as the driver service's name.") + shorterServiceName + } + + val driverPort = submissionSparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverBlockManagerPort = submissionSparkConf.getInt( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + val driverService = new ServiceBuilder() + .withNewMetadata() + .withName(resolvedServiceName) + .endMetadata() + .withNewSpec() + .withClusterIP("None") + .withSelector(driverLabels.asJava) + .addNewPort() + .withName(DRIVER_PORT_NAME) + .withPort(driverPort) + .withNewTargetPort(driverPort) + .endPort() + .addNewPort() + .withName(BLOCK_MANAGER_PORT_NAME) + .withPort(driverBlockManagerPort) + .withNewTargetPort(driverBlockManagerPort) + .endPort() + .endSpec() + .build() + + val namespace = submissionSparkConf.get(KUBERNETES_NAMESPACE) + val driverHostname = s"${driverService.getMetadata.getName}.$namespace.svc.cluster.local" + val resolvedSparkConf = driverSpec.driverSparkConf.clone() + .set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, driverHostname) + .set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, driverHostname) + .set("spark.driver.port", driverPort.toString) + .set( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, driverBlockManagerPort) + + driverSpec.copy( + driverSparkConf = resolvedSparkConf, + otherKubernetesResources = driverSpec.otherKubernetesResources ++ Seq(driverService)) + } +} + +private[spark] object DriverAddressConfigurationStep { + val DRIVER_BIND_ADDRESS_KEY = org.apache.spark.internal.config.DRIVER_BIND_ADDRESS.key + val DRIVER_HOST_KEY = org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key + val DRIVER_SVC_POSTFIX = "-driver-svc" + val MAX_SERVICE_NAME_LENGTH = 63 +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala index 6bad594629f76..fc9159d08fe23 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.kubernetes.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps._ +import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -50,6 +50,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep]) } @@ -72,6 +73,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], classOf[InitContainerBootstrapStep]) @@ -93,6 +95,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], classOf[PythonStep]) @@ -114,6 +117,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], classOf[MountSmallLocalFilesStep]) @@ -137,6 +141,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS validateStepTypes( orchestrator, classOf[BaseDriverConfigurationStep], + classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], classOf[MountSecretsStep]) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala new file mode 100644 index 0000000000000..08c939987603d --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala @@ -0,0 +1,181 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import io.fabric8.kubernetes.api.model.Service +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfter +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.util.Clock + +private[spark] class DriverAddressConfigurationStepSuite + extends SparkFunSuite with BeforeAndAfter { + + private val SHORT_RESOURCE_NAME_PREFIX = + "a" * (DriverAddressConfigurationStep.MAX_SERVICE_NAME_LENGTH - + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX.length) + + private val LONG_RESOURCE_NAME_PREFIX = + "a" * (DriverAddressConfigurationStep.MAX_SERVICE_NAME_LENGTH - + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX.length + 1) + private val DRIVER_LABELS = Map( + "label1key" -> "label1value", + "label2key" -> "label2value") + + @Mock + private var clock: Clock = _ + + private var sparkConf: SparkConf = _ + + before { + MockitoAnnotations.initMocks(this) + sparkConf = new SparkConf(false) + } + + test("Headless service has a port for the driver RPC and the block manager.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + assert(resolvedDriverSpec.otherKubernetesResources.size === 1) + assert(resolvedDriverSpec.otherKubernetesResources.head.isInstanceOf[Service]) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + verifyService( + 9000, + 8080, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}", + driverService) + } + + test("Hostname and ports are set according to the service name.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf + .set("spark.driver.port", "9000") + .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + + DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Ports should resolve to defaults in SparkConf and in the service.") { + val configurationStep = new DriverAddressConfigurationStep( + SHORT_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf, + clock) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + verifyService( + DEFAULT_DRIVER_PORT, + DEFAULT_BLOCKMANAGER_PORT, + s"$SHORT_RESOURCE_NAME_PREFIX${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}", + resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service]) + assert(resolvedDriverSpec.driverSparkConf.get("spark.driver.port") === + DEFAULT_DRIVER_PORT.toString) + assert(resolvedDriverSpec.driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT) === DEFAULT_BLOCKMANAGER_PORT) + } + + test("Long prefixes should switch to using a generated name.") { + val configurationStep = new DriverAddressConfigurationStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), + clock) + when(clock.getTimeMillis()).thenReturn(10000) + val baseDriverSpec = KubernetesDriverSpec.initialSpec(sparkConf.clone()) + val resolvedDriverSpec = configurationStep.configureDriver(baseDriverSpec) + val driverService = resolvedDriverSpec.otherKubernetesResources.head.asInstanceOf[Service] + val expectedServiceName = s"spark-10000${DriverAddressConfigurationStep.DRIVER_SVC_POSTFIX}" + assert(driverService.getMetadata.getName === expectedServiceName) + val expectedHostName = s"$expectedServiceName.my-namespace.svc.cluster.local" + verifySparkConfHostNames(resolvedDriverSpec.driverSparkConf, expectedHostName) + } + + test("Disallow bind address and driver host to be set explicitly.") { + val configurationStep = new DriverAddressConfigurationStep( + LONG_RESOURCE_NAME_PREFIX, + DRIVER_LABELS, + sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), + clock) + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver bind address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverAddressConfigurationStep.DRIVER_BIND_ADDRESS_KEY} is" + + s" not supported in Kubernetes mode, as the driver's hostname will be managed via" + + s" a Kubernetes service.") + } + sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) + sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The driver host address should not be allowed.") + } catch { + case e: Throwable => + assert(e.getMessage === + s"requirement failed: ${DriverAddressConfigurationStep.DRIVER_HOST_KEY} is" + + s" not supported in Kubernetes mode, as the driver's hostname will be managed via" + + s" a Kubernetes service.") + } + } + + private def verifyService( + driverPort: Int, + blockManagerPort: Int, + expectedServiceName: String, + service: Service): Unit = { + assert(service.getMetadata.getName === expectedServiceName) + assert(service.getSpec.getClusterIP === "None") + assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) + assert(service.getSpec.getPorts.size() === 2) + val driverServicePorts = service.getSpec.getPorts.asScala + assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) + assert(driverServicePorts.head.getPort.intValue() === driverPort) + assert(driverServicePorts.head.getTargetPort.getIntVal === driverPort) + assert(driverServicePorts(1).getName === BLOCK_MANAGER_PORT_NAME) + assert(driverServicePorts(1).getPort.intValue() === blockManagerPort) + assert(driverServicePorts(1).getTargetPort.getIntVal === blockManagerPort) + } + + private def verifySparkConfHostNames( + driverSparkConf: SparkConf, expectedHostName: String): Unit = { + assert(driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_HOST_ADDRESS) === expectedHostName) + assert(driverSparkConf.get( + org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) === expectedHostName) + } +} From 8fb4b5d62ceb1dde0ccfc04b946393e219dfa31c Mon Sep 17 00:00:00 2001 From: Varun Katta Date: Thu, 14 Sep 2017 21:46:47 -0700 Subject: [PATCH 06/25] Code enhancement: Replaced explicit synchronized access to a hashmap with a concurrent map. (#392) * Replaced explicit synchronized access to hashmap with a concurrent map * Removed usages of scala.collection.concurrent.Map --- .../KubernetesClusterSchedulerBackend.scala | 60 ++++++++----------- 1 file changed, 24 insertions(+), 36 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala index 54612c80cdf28..1834cddfa7304 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala @@ -25,7 +25,7 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action -import scala.collection.{concurrent, mutable} +import scala.collection.mutable import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} @@ -56,14 +56,9 @@ private[spark] class KubernetesClusterSchedulerBackend( private val runningExecutorsToPods = new mutable.HashMap[String, Pod] // Indexed by executor pod names and guarded by RUNNING_EXECUTOR_PODS_LOCK. private val runningPodsToExecutors = new mutable.HashMap[String, String] - // TODO(varun): Get rid of this lock object by my making the underlying map a concurrent hash map. - private val EXECUTOR_PODS_BY_IPS_LOCK = new Object - // Indexed by executor IP addrs and guarded by EXECUTOR_PODS_BY_IPS_LOCK - private val executorPodsByIPs = new mutable.HashMap[String, Pod] - private val podsWithKnownExitReasons: concurrent.Map[String, ExecutorExited] = - new ConcurrentHashMap[String, ExecutorExited]().asScala - private val disconnectedPodsByExecutorIdPendingRemoval = - new ConcurrentHashMap[String, Pod]().asScala + private val executorPodsByIPs = new ConcurrentHashMap[String, Pod]() + private val podsWithKnownExitReasons = new ConcurrentHashMap[String, ExecutorExited]() + private val disconnectedPodsByExecutorIdPendingRemoval = new ConcurrentHashMap[String, Pod]() private val kubernetesNamespace = conf.get(KUBERNETES_NAMESPACE) @@ -102,13 +97,13 @@ private[spark] class KubernetesClusterSchedulerBackend( private val podAllocationInterval = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) require(podAllocationInterval > 0, s"Allocation batch delay " + - s"${KUBERNETES_ALLOCATION_BATCH_DELAY} " + - s"is ${podAllocationInterval}, should be a positive integer") + s"$KUBERNETES_ALLOCATION_BATCH_DELAY " + + s"is $podAllocationInterval, should be a positive integer") private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) require(podAllocationSize > 0, s"Allocation batch size " + - s"${KUBERNETES_ALLOCATION_BATCH_SIZE} " + - s"is ${podAllocationSize}, should be a positive integer") + s"$KUBERNETES_ALLOCATION_BATCH_SIZE " + + s"is $podAllocationSize, should be a positive integer") private val allocatorRunnable = new Runnable { @@ -141,10 +136,10 @@ private[spark] class KubernetesClusterSchedulerBackend( // For each disconnected executor, synchronize with the loss reasons that may have been found // by the executor pod watcher. If the loss reason was discovered by the watcher, // inform the parent class with removeExecutor. - val disconnectedPodsByExecutorIdPendingRemovalCopy = - Map.empty ++ disconnectedPodsByExecutorIdPendingRemoval - disconnectedPodsByExecutorIdPendingRemovalCopy.foreach { case (executorId, executorPod) => - val knownExitReason = podsWithKnownExitReasons.remove(executorPod.getMetadata.getName) + disconnectedPodsByExecutorIdPendingRemoval.keys().asScala.foreach { case (executorId) => + val executorPod = disconnectedPodsByExecutorIdPendingRemoval.get(executorId) + val knownExitReason = Option(podsWithKnownExitReasons.remove( + executorPod.getMetadata.getName)) knownExitReason.fold { removeExecutorOrIncrementLossReasonCheckCount(executorId) } { executorExited => @@ -171,7 +166,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } def deleteExecutorFromClusterAndDataStructures(executorId: String): Unit = { - disconnectedPodsByExecutorIdPendingRemoval -= executorId + disconnectedPodsByExecutorIdPendingRemoval.remove(executorId) executorReasonCheckAttemptCounts -= executorId RUNNING_EXECUTOR_PODS_LOCK.synchronized { runningExecutorsToPods.remove(executorId).map { pod => @@ -239,9 +234,7 @@ private[spark] class KubernetesClusterSchedulerBackend( runningExecutorsToPods.clear() runningPodsToExecutors.clear() } - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.clear() - } + executorPodsByIPs.clear() val resource = executorWatchResource.getAndSet(null) if (resource != null) { resource.close() @@ -262,14 +255,11 @@ private[spark] class KubernetesClusterSchedulerBackend( * locality if an executor launches on the cluster node. */ private def getNodesWithLocalTaskCounts() : Map[String, Int] = { - val executorPodsWithIPs = EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.values.toList // toList makes a defensive copy. - } val nodeToLocalTaskCount = mutable.Map[String, Int]() ++ KubernetesClusterSchedulerBackend.this.synchronized { hostToLocalTaskCount } - for (pod <- executorPodsWithIPs) { + for (pod <- executorPodsByIPs.values().asScala) { // Remove cluster nodes that are running our executors already. // TODO: This prefers spreading out executors across nodes. In case users want // consolidating executors on fewer nodes, introduce a flag. See the spark.deploy.spreadOut @@ -319,7 +309,7 @@ private[spark] class KubernetesClusterSchedulerBackend( val maybeRemovedExecutor = runningExecutorsToPods.remove(executor) maybeRemovedExecutor.foreach { executorPod => kubernetesClient.pods().delete(executorPod) - disconnectedPodsByExecutorIdPendingRemoval(executor) = executorPod + disconnectedPodsByExecutorIdPendingRemoval.put(executor, executorPod) runningPodsToExecutors.remove(executorPod.getMetadata.getName) } if (maybeRemovedExecutor.isEmpty) { @@ -331,9 +321,11 @@ private[spark] class KubernetesClusterSchedulerBackend( } def getExecutorPodByIP(podIP: String): Option[Pod] = { - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs.get(podIP) - } + // Note: Per https://github.com/databricks/scala-style-guide#concurrency, we don't + // want to be switching to scala.collection.concurrent.Map on + // executorPodsByIPs. + val pod = executorPodsByIPs.get(podIP) + Option(pod) } private class ExecutorPodsWatcher extends Watcher[Pod] { @@ -346,18 +338,14 @@ private[spark] class KubernetesClusterSchedulerBackend( val podIP = pod.getStatus.getPodIP val clusterNodeName = pod.getSpec.getNodeName logDebug(s"Executor pod $pod ready, launched at $clusterNodeName as IP $podIP.") - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs += ((podIP, pod)) - } + executorPodsByIPs.put(podIP, pod) } else if ((action == Action.MODIFIED && pod.getMetadata.getDeletionTimestamp != null) || action == Action.DELETED || action == Action.ERROR) { val podName = pod.getMetadata.getName val podIP = pod.getStatus.getPodIP logDebug(s"Executor pod $podName at IP $podIP was at $action.") if (podIP != null) { - EXECUTOR_PODS_BY_IPS_LOCK.synchronized { - executorPodsByIPs -= podIP - } + executorPodsByIPs.remove(podIP) } if (action == Action.ERROR) { logInfo(s"Received pod $podName exited event. Reason: " + pod.getStatus.getReason) @@ -445,7 +433,7 @@ private[spark] class KubernetesClusterSchedulerBackend( if (disableExecutor(executorId)) { RUNNING_EXECUTOR_PODS_LOCK.synchronized { runningExecutorsToPods.get(executorId).foreach { pod => - disconnectedPodsByExecutorIdPendingRemoval(executorId) = pod + disconnectedPodsByExecutorIdPendingRemoval.put(executorId, pod) } } } From a771a0271b319334ba058449da679e91327d0a5e Mon Sep 17 00:00:00 2001 From: Steve Larkin Date: Fri, 15 Sep 2017 18:43:12 +0200 Subject: [PATCH 07/25] docs: Fix path to spark-base Dockerfile (#493) --- docs/running-on-kubernetes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 9ddc4ee4319fb..6bf5140af6a3c 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -61,7 +61,7 @@ to the registry. For example, if the registry host is `registry-host` and the registry is listening on port 5000: cd $SPARK_HOME - docker build -t registry-host:5000/spark-base:latest -f dockerfiles/driver/spark-base . + docker build -t registry-host:5000/spark-base:latest -f dockerfiles/spark-base/Dockerfile . docker build -t registry-host:5000/spark-driver:latest -f dockerfiles/driver/Dockerfile . docker build -t registry-host:5000/spark-executor:latest -f dockerfiles/executor/Dockerfile . docker build -t registry-host:5000/spark-init:latest -f dockerfiles/init-container/Dockerfile . From be394c6dd2ff42a0652b08e271d18cd73c02167f Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Sat, 16 Sep 2017 10:57:14 -0700 Subject: [PATCH 08/25] Improve the image building workflow (#488) --- sbin/build-push-docker-images.sh | 72 ++++++++++++++++++++++++++++++++ 1 file changed, 72 insertions(+) create mode 100755 sbin/build-push-docker-images.sh diff --git a/sbin/build-push-docker-images.sh b/sbin/build-push-docker-images.sh new file mode 100755 index 0000000000000..efd6f96516d90 --- /dev/null +++ b/sbin/build-push-docker-images.sh @@ -0,0 +1,72 @@ +#!/usr/bin/env bash + +# 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. +# + +# This script builds and pushes docker images when run from a release of Spark +# with Kubernetes support. + +declare -A path=( [spark-driver]=dockerfiles/driver/Dockerfile \ + [spark-executor]=dockerfiles/executor/Dockerfile \ + [spark-driver-py]=dockerfiles/driver-py/Dockerfile \ + [spark-executor-py]=dockerfiles/executor-py/Dockerfile \ + [spark-init]=dockerfiles/init-container/Dockerfile \ + [spark-shuffle]=dockerfiles/shuffle-service/Dockerfile \ + [spark-resource-staging-server]=dockerfiles/resource-staging-server/Dockerfile ) + +function build { + docker build -t spark-base -f dockerfiles/spark-base/Dockerfile . + for image in "${!path[@]}"; do + docker build -t ${REPO}/$image:${TAG} -f ${path[$image]} . + done +} + + +function push { + for image in "${!path[@]}"; do + docker push ${REPO}/$image:${TAG} + done +} + +function usage { + echo "Usage: ./sbin/build-push-docker-images.sh -r -t build" + echo " ./sbin/build-push-docker-images.sh -r -t push" + echo "for example: ./sbin/build-push-docker-images.sh -r docker.io/kubespark -t v2.2.0 push" +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage + exit 0 +fi + +while getopts r:t: option +do + case "${option}" + in + r) REPO=${OPTARG};; + t) TAG=${OPTARG};; + esac +done + +if [ -z "$REPO" ] || [ -z "$TAG" ]; then + usage +else + case "${@: -1}" in + build) build;; + push) push;; + *) usage;; + esac +fi \ No newline at end of file From b84056e25cb10869ce5fbdf0a56484595eb03b7f Mon Sep 17 00:00:00 2001 From: Sahil Prasad Date: Sat, 16 Sep 2017 18:26:35 -0400 Subject: [PATCH 09/25] =?UTF-8?q?Fail=20submission=20if=20submitter-local?= =?UTF-8?q?=20files=20are=20provided=20without=20resourc=E2=80=A6=20(#447)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Fail submission if submitter-local files are provided without resource staging server URI * Modified logic to validate only submitted jars; added orchestrator tests * Incorporated feedback * Fix failing test case --- ...tainerConfigurationStepsOrchestrator.scala | 8 +- ...rConfigurationStepsOrchestratorSuite.scala | 76 ++++++++++++++++++- 2 files changed, 82 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index e4ea5235af18f..b99d4e86bf5bf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -20,7 +20,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} import org.apache.spark.deploy.kubernetes.config._ import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploaderImpl +import org.apache.spark.deploy.kubernetes.submit.{KubernetesFileUtils, SubmittedDependencyUploaderImpl} import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} import org.apache.spark.util.Utils @@ -62,6 +62,12 @@ private[spark] class InitContainerConfigurationStepsOrchestrator( submissionSparkConf.get(RESOURCE_STAGING_SERVER_INTERNAL_SSL_ENABLED) .orElse(submissionSparkConf.get(RESOURCE_STAGING_SERVER_SSL_ENABLED)) .getOrElse(false) + + OptionRequirements.requireSecondIfFirstIsDefined( + KubernetesFileUtils.getOnlySubmitterLocalFiles(sparkJars).headOption, + resourceStagingServerUri, + "Local JARs were provided, however no resource staging server URI was found.") + OptionRequirements.requireNandDefined( maybeResourceStagingServerInternalClientCert, maybeResourceStagingServerInternalTrustStore, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala index 1cc8007803457..ea4d60795787c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -44,6 +44,80 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val INIT_CONTAINER_CONFIG_MAP_KEY = "spark-init-config-map-key" private val STAGING_SERVER_URI = "http://localhost:8000" + test ("error thrown if local jars provided without resource staging server") { + val sparkConf = new SparkConf(true) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + assert(sparkConf.get(RESOURCE_STAGING_SERVER_URI).isEmpty) + + val thrown = intercept[IllegalArgumentException] { + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS, + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + } + + assert(thrown.getMessage contains "Local JARs were provided, however no resource staging" + + " server URI was found.") + } + + test ("error not thrown with non-local jars and resource staging server provided") { + val sparkConf = new SparkConf(true) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 2) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + assert(initSteps(1).isInstanceOf[SubmittedResourcesInitContainerConfigurationStep]) + } + + test ("error not thrown with non-local jars and no resource staging server provided") { + val sparkConf = new SparkConf(true) + .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") + .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) + + val orchestrator = new InitContainerConfigurationStepsOrchestrator( + NAMESPACE, + APP_RESOURCE_PREFIX, + SPARK_JARS.take(1), + SPARK_FILES, + JARS_DOWNLOAD_PATH, + FILES_DOWNLOAD_PATH, + DOCKER_IMAGE_PULL_POLICY, + DRIVER_LABELS, + INIT_CONTAINER_CONFIG_MAP_NAME, + INIT_CONTAINER_CONFIG_MAP_KEY, + sparkConf) + val initSteps : Seq[InitContainerConfigurationStep] = + orchestrator.getAllConfigurationSteps() + assert(initSteps.length == 1) + assert(initSteps.head.isInstanceOf[BaseInitContainerConfigurationStep]) + } + test ("including step to contact resource staging server") { val sparkConf = new SparkConf(true) .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") @@ -77,7 +151,7 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { val orchestrator = new InitContainerConfigurationStepsOrchestrator( NAMESPACE, APP_RESOURCE_PREFIX, - SPARK_JARS, + SPARK_JARS.take(1), SPARK_FILES, JARS_DOWNLOAD_PATH, FILES_DOWNLOAD_PATH, From 8b17246c3ed2962664f344c58e24dbac3f7428e9 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Thu, 21 Sep 2017 14:57:06 -0700 Subject: [PATCH 10/25] Rename package to k8s (#497) * Rename package to k8s * Rename string constants --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- ...che.spark.scheduler.ExternalClusterManager | 2 +- .../CompressionUtils.scala | 2 +- .../ConfigurationUtils.scala | 2 +- ...nerResourceStagingServerSecretPlugin.scala | 4 ++-- .../KubernetesCredentials.scala | 2 +- .../KubernetesExternalShuffleService.scala | 6 ++--- .../OptionRequirements.scala | 2 +- .../PodWithDetachedInitContainer.scala | 2 +- .../SparkKubernetesClientFactory.scala | 4 ++-- .../SparkPodInitContainerBootstrap.scala | 4 ++-- .../deploy/{kubernetes => k8s}/config.scala | 4 ++-- .../{kubernetes => k8s}/constants.scala | 2 +- .../{kubernetes => k8s}/submit/Client.scala | 10 ++++----- .../ContainerNameEqualityPredicate.scala | 2 +- ...DriverConfigurationStepsOrchestrator.scala | 12 +++++----- .../submit/InitContainerUtil.scala | 4 ++-- .../submit/KubernetesFileUtils.scala | 2 +- .../submit/LoggingPodStatusWatcher.scala | 6 ++--- .../submit/MainAppResource.scala | 2 +- .../submit/MountSecretsBootstrap.scala | 2 +- .../submit/MountSmallFilesBootstrap.scala | 4 ++-- ...opertiesConfigMapFromScalaMapBuilder.scala | 2 +- .../SubmittedDependencyUploaderImpl.scala | 6 ++--- .../submit/SubmittedResources.scala | 2 +- .../BaseDriverConfigurationStep.scala | 8 +++---- .../DependencyResolutionStep.scala | 8 +++---- .../DriverAddressConfigurationStep.scala | 6 ++--- .../submitsteps/DriverConfigurationStep.scala | 2 +- .../DriverKubernetesCredentialsStep.scala | 6 ++--- .../InitContainerBootstrapStep.scala | 8 +++---- .../submitsteps/KubernetesDriverSpec.scala | 2 +- .../submit/submitsteps/MountSecretsStep.scala | 4 ++-- .../MountSmallLocalFilesStep.scala | 6 ++--- .../submit/submitsteps/PythonStep.scala | 6 ++--- .../BaseInitContainerConfigurationStep.scala | 8 +++---- .../InitContainerConfigurationStep.scala | 2 +- ...tainerConfigurationStepsOrchestrator.scala | 12 +++++----- .../initcontainer/InitContainerSpec.scala | 2 +- ...ourcesInitContainerConfigurationStep.scala | 10 ++++----- .../{kubernetes => k8s}/FileFetcher.scala | 2 +- ...SparkDependencyDownloadInitContainer.scala | 6 ++--- .../PemsToKeyStoreConverter.scala | 2 +- .../ResourceStagingServer.scala | 6 ++--- ...ourceStagingServerSslOptionsProvider.scala | 6 ++--- .../ResourceStagingService.scala | 6 ++--- .../ResourceStagingServiceImpl.scala | 6 ++--- .../ResourceStagingServiceRetrofit.scala | 4 ++-- .../RetrofitClientFactory.scala | 2 +- .../SparkConfPropertiesParser.scala | 2 +- .../{kubernetes => k8s}/StagedResources.scala | 2 +- .../StagedResourcesCleaner.scala | 2 +- .../StagedResourcesOwner.scala | 2 +- .../StagedResourcesStore.scala | 2 +- .../ExecutorPodFactory.scala | 10 ++++----- .../{kubernetes => k8s}/InetAddressUtil.scala | 6 ++--- .../KubernetesClusterManager.scala | 10 ++++----- .../KubernetesClusterSchedulerBackend.scala | 6 ++--- .../KubernetesExternalShuffleManager.scala | 6 ++--- .../KubernetesTaskSchedulerImpl.scala | 5 ++--- .../KubernetesTaskSetManager.scala | 4 ++-- .../NodeAffinityExecutorPodModifier.scala | 4 ++-- .../RackResolverUtil.scala | 6 ++--- .../ShuffleServiceConfig.scala | 2 +- ...sourceStagingServerSecretPluginSuite.scala | 3 ++- .../deploy/{kubernetes => k8s}/SSLUtils.scala | 4 ++-- .../SparkPodInitContainerBootstrapSuite.scala | 3 ++- .../submit/ClientSuite.scala | 6 ++--- ...rConfigurationStepsOrchestratorSuite.scala | 6 ++--- ...ubernetesExternalShuffleServiceSuite.scala | 7 +++--- .../submit/MountSecretsBootstrapSuite.scala | 2 +- .../submit/SSLFilePairs.scala | 2 +- .../submit/SecretVolumeUtils.scala | 2 +- .../SubmittedDependencyUploaderSuite.scala | 6 ++--- .../BaseDriverConfigurationStepSuite.scala | 6 ++--- .../DependencyResolutionStepSuite.scala | 4 ++-- .../DriverAddressConfigurationStepSuite.scala | 6 ++--- ...DriverKubernetesCredentialsStepSuite.scala | 6 ++--- .../submitsteps/MountSecretsStepSuite.scala | 4 ++-- .../MountSmallLocalFilesStepSuite.scala | 8 +++---- .../submit/submitsteps/PythonStepSuite.scala | 2 +- .../initContainerBootstrapStepSuite.scala | 8 +++---- ...eInitContainerConfigurationStepSuite.scala | 6 ++--- ...rConfigurationStepsOrchestratorSuite.scala | 6 ++--- ...ittedResourcesInitContainerStepSuite.scala | 10 ++++----- ...DependencyDownloadInitContainerSuite.scala | 6 ++--- ...StagingServerSslOptionsProviderSuite.scala | 4 ++-- .../ResourceStagingServerSuite.scala | 4 ++-- .../StagedResourcesCleanerSuite.scala | 2 +- .../StagedResourcesStoreSuite.scala | 2 +- ...bernetesClusterSchedulerBackendSuite.scala | 6 ++--- .../KubernetesTaskSchedulerImplSuite.scala | 4 ++-- .../KubernetesTaskSetManagerSuite.scala | 4 ++-- .../RackResolverUtilImplSuite.scala | 2 +- .../src/main/docker/init-container/Dockerfile | 2 +- .../docker/resource-staging-server/Dockerfile | 2 +- .../main/docker/shuffle-service/Dockerfile | 2 +- .../integrationtest/PiHelper.java | 2 +- .../jobs/FileExistenceTest.scala | 2 +- .../integrationtest/jobs/GroupByTest.scala | 2 +- .../jobs/JavaOptionsTest.scala | 2 +- .../jobs/SparkPiWithInfiniteWait.scala | 4 ++-- .../integrationtest/KubernetesSuite.scala | 22 +++++++++---------- .../KubernetesTestComponents.scala | 4 ++-- .../integrationtest/ProcessUtils.scala | 2 +- .../ResourceStagingServerLauncher.scala | 6 ++--- .../SparkReadinessWatcher.scala | 2 +- .../StaticAssetServerLauncher.scala | 2 +- .../backend/GCE/GCETestBackend.scala | 8 +++---- .../backend/IntegrationTestBackend.scala | 8 +++---- .../backend/minikube/Minikube.scala | 4 ++-- .../minikube/MinikubeTestBackend.scala | 8 +++---- .../integrationtest/constants.scala | 2 +- .../docker/SparkDockerImageBuilder.scala | 2 +- .../restapis/SparkRestApiV1.scala | 2 +- 115 files changed, 261 insertions(+), 261 deletions(-) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/CompressionUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/ConfigurationUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/InitContainerResourceStagingServerSecretPlugin.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/KubernetesCredentials.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/KubernetesExternalShuffleService.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/OptionRequirements.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/PodWithDetachedInitContainer.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkKubernetesClientFactory.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkPodInitContainerBootstrap.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/config.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/constants.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/Client.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/ContainerNameEqualityPredicate.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/DriverConfigurationStepsOrchestrator.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/InitContainerUtil.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/KubernetesFileUtils.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/LoggingPodStatusWatcher.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MainAppResource.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MountSecretsBootstrap.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MountSmallFilesBootstrap.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/PropertiesConfigMapFromScalaMapBuilder.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedDependencyUploaderImpl.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedResources.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/BaseDriverConfigurationStep.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DependencyResolutionStep.scala (90%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverAddressConfigurationStep.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverConfigurationStep.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverKubernetesCredentialsStep.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/InitContainerBootstrapStep.scala (89%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/KubernetesDriverSpec.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSecretsStep.scala (91%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSmallLocalFilesStep.scala (94%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/PythonStep.scala (90%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala (89%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala (91%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerSpec.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/FileFetcher.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/KubernetesSparkDependencyDownloadInitContainer.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/PemsToKeyStoreConverter.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServer.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSslOptionsProvider.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingService.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServiceImpl.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServiceRetrofit.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/RetrofitClientFactory.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/SparkConfPropertiesParser.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResources.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesCleaner.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesOwner.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesStore.scala (98%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/ExecutorPodFactory.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/InetAddressUtil.scala (87%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesClusterManager.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesClusterSchedulerBackend.scala (99%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesExternalShuffleManager.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSchedulerImpl.scala (95%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSetManager.scala (97%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/NodeAffinityExecutorPodModifier.scala (96%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/RackResolverUtil.scala (93%) rename resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/ShuffleServiceConfig.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/InitContainerResourceStagingServerSecretPluginSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/SSLUtils.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/SparkPodInitContainerBootstrapSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/ClientSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/DriverConfigurationStepsOrchestratorSuite.scala (93%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/KubernetesExternalShuffleServiceSuite.scala (89%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/MountSecretsBootstrapSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SSLFilePairs.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SecretVolumeUtils.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/SubmittedDependencyUploaderSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/BaseDriverConfigurationStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DependencyResolutionStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverAddressConfigurationStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSecretsStepSuite.scala (92%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/MountSmallLocalFilesStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/PythonStepSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initContainerBootstrapStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala (94%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala (97%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala (96%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/KubernetesSparkDependencyDownloadInitContainerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSslOptionsProviderSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/ResourceStagingServerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesCleanerSuite.scala (99%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/{kubernetes => k8s}/StagedResourcesStoreSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesClusterSchedulerBackendSuite.scala (99%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSchedulerImplSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/KubernetesTaskSetManagerSuite.scala (98%) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/{kubernetes => k8s}/RackResolverUtilImplSuite.scala (97%) rename resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/PiHelper.java (94%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/FileExistenceTest.scala (97%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/GroupByTest.scala (96%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/JavaOptionsTest.scala (98%) rename resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/jobs/SparkPiWithInfiniteWait.scala (92%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KubernetesSuite.scala (95%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/KubernetesTestComponents.scala (95%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/ProcessUtils.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/ResourceStagingServerLauncher.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/SparkReadinessWatcher.scala (96%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/StaticAssetServerLauncher.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/GCE/GCETestBackend.scala (81%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/IntegrationTestBackend.scala (78%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/minikube/Minikube.scala (97%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/backend/minikube/MinikubeTestBackend.scala (81%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/constants.scala (93%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/docker/SparkDockerImageBuilder.scala (98%) rename resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/{kubernetes => k8s}/integrationtest/restapis/SparkRestApiV1.scala (96%) 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 ba94be6133d56..0935c0a545878 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -710,7 +710,7 @@ object SparkSubmit extends CommandLineUtils with Logging { } if (isKubernetesCluster) { - childMainClass = "org.apache.spark.deploy.kubernetes.submit.Client" + childMainClass = "org.apache.spark.deploy.k8s.submit.Client" if (args.isPython) { childArgs ++= Array("--primary-py-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") diff --git a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index 55e7e38b28a08..81d14766ffb8d 100644 --- a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1 +1 @@ -org.apache.spark.scheduler.cluster.kubernetes.KubernetesClusterManager +org.apache.spark.scheduler.cluster.k8s.KubernetesClusterManager diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala index a6f0ca502f6f0..364882d542fa6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/CompressionUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/CompressionUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStream, OutputStream} import java.util.zip.{GZIPInputStream, GZIPOutputStream} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 1a008c236d00f..7645e54e46280 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala index 265b8f197a102..99947498d6a89 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPlugin.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPlugin.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder, Secret} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait InitContainerResourceStagingServerSecretPlugin { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala index aba94e6969529..833af9b3b5638 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesCredentials.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesCredentials.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s case class KubernetesCredentials( oauthTokenBase64: Option[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala index c61f4f1d44acf..661c3a40a15a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/KubernetesExternalShuffleService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesExternalShuffleService.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File import java.nio.ByteBuffer @@ -29,8 +29,8 @@ import scala.collection.mutable import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.ExternalShuffleService -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala index eda43de0a9a5b..89053de5b9558 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/OptionRequirements.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/OptionRequirements.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s private[spark] object OptionRequirements { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala index 36b1b07dc6bc4..cdec3a155a597 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/PodWithDetachedInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/PodWithDetachedInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index d2729a2db2fa0..c90303b6ba506 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.File @@ -25,7 +25,7 @@ import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.ThreadUtils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala index 2df7ac7a204c4..35246506ecb33 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, PodBuilder, VolumeMount, VolumeMountBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ /** * This is separated out from the init-container steps API because this component can be reused to diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala index 9dfd13e1817f8..ebabed956365d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/config.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.util.concurrent.TimeUnit import org.apache.spark.{SPARK_VERSION => sparkVersion} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala index 0a2bc46249f3a..d8aec7d3c5bd7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/constants.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s package object constants { // Labels diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index 9573bc222505f..ba23caea49498 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.{Collections, UUID} @@ -24,10 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala index 434919208ba2e..94adcc5857d2a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/ContainerNameEqualityPredicate.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/ContainerNameEqualityPredicate.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.lang.Boolean diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index afc73e8f07601..17e4a18a07d47 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.{SystemClock, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala index 837ec0e8c867e..55ed06d7ca415 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/InitContainerUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/InitContainerUtil.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import io.fabric8.kubernetes.api.model.{Container, Pod, PodBuilder} import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] object InitContainerUtil { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala index ec591923f1472..00ff72ec0efb1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesFileUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesFileUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 4a8a7308b9fe4..c645b008d736d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.util.concurrent.{CountDownLatch, TimeUnit} @@ -27,7 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils -private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { +private[k8s] trait LoggingPodStatusWatcher extends Watcher[Pod] { def awaitCompletion(): Unit } @@ -39,7 +39,7 @@ private[kubernetes] trait LoggingPodStatusWatcher extends Watcher[Pod] { * @param maybeLoggingInterval ms between each state request. If provided, must be a positive * number. */ -private[kubernetes] class LoggingPodStatusWatcherImpl( +private[k8s] class LoggingPodStatusWatcherImpl( appId: String, maybeLoggingInterval: Option[Long]) extends LoggingPodStatusWatcher with Logging { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala index 436d531a850ff..f806e65974fcc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit private[spark] sealed trait MainAppResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala index ae10c9390b221..e83dcdcdc0637 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrap.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala index 79919b511ec37..2ace625b4d6a6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/MountSmallFilesBootstrap.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MountSmallFilesBootstrap.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] trait MountSmallFilesBootstrap { def mountSmallFilesSecret(pod: Pod, container: Container): (Pod, Container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala index 8103272c27518..d1d20711abffe 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/PropertiesConfigMapFromScalaMapBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/PropertiesConfigMapFromScalaMapBuilder.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.StringWriter import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala index 83d7a28f5ca10..7b829351a28ba 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{File, FileOutputStream} import javax.ws.rs.core.MediaType @@ -27,8 +27,8 @@ import okhttp3.RequestBody import retrofit2.Call import org.apache.spark.{SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner, StagedResourcesOwnerType} import org.apache.spark.util.Utils private[spark] trait SubmittedDependencyUploader { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala index 225972c1057f2..b02cb21372678 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/SubmittedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit case class SubmittedResourceIdAndSecret(resourceId: String, resourceSecret: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala index b76c77a656d2d..0eb9ee2563045 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala @@ -14,15 +14,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, PodBuilder, QuantityBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Represents the initial setup required for the driver. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala index 090240420119e..c2c76c2ba1960 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStep.scala @@ -14,16 +14,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.InitContainerConfigurationStep +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStep import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala index 85412ad19742b..615261e80f302 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStep.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.ServiceBuilder import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.internal.Logging import org.apache.spark.util.Clock diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala index 8070e32371f94..706b02db57672 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps /** * Represents a step in preparing the Kubernetes driver. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala index 74df2de480bf4..da5686099fbbb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File import java.nio.charset.StandardCharsets @@ -25,8 +25,8 @@ import scala.collection.JavaConverters._ import scala.language.implicitConversions import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ /** * Mounts Kubernetes credentials into the driver pod. The driver will use such mounted credentials diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala index b4248338cc8de..f8d1c909ac794 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/InitContainerBootstrapStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStep.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, PropertiesConfigMapFromScalaMapBuilder} +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} /** * Configures the init-container that bootstraps dependencies into the driver pod. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala index 3ec4b6c4df10f..f76c0a8c9c6bd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/KubernetesDriverSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/KubernetesDriverSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, HasMetadata, Pod, PodBuilder} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala similarity index 91% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala index d20865daba3e1..21c7cafd987cd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStep.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps -import org.apache.spark.deploy.kubernetes.submit.MountSecretsBootstrap +import org.apache.spark.deploy.k8s.submit.MountSecretsBootstrap /** * A driver configuration step for mounting user-specified secrets onto user-specified paths. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala index 6f9796a70eeb6..52503b1b3910e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,8 +22,8 @@ import com.google.common.io.{BaseEncoding, Files} import io.fabric8.kubernetes.api.model.SecretBuilder import scala.collection.JavaConverters._ -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{KubernetesFileUtils, MountSmallFilesBootstrap} import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStep( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala similarity index 90% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala index 024d643ddf9fd..f0003924e9c6d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStep.scala @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.ContainerBuilder -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class PythonStep( primaryPyFile: String, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala similarity index 89% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala index 60bf27beacaaf..dc335ac8f076e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStep.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.KubernetesFileUtils +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.KubernetesFileUtils private[spark] class BaseInitContainerConfigurationStep( sparkJars: Seq[String], diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala index 7b7622c3d4f8b..a3bdaa8e93344 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer /** * Represents a step in preparing the init-container for the driver and executors. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala similarity index 91% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala index b99d4e86bf5bf..07a45dc1f1613 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestrator.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{KubernetesFileUtils, SubmittedDependencyUploaderImpl} -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} +import org.apache.spark.deploy.k8s.{InitContainerResourceStagingServerSecretPluginImpl, OptionRequirements, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{KubernetesFileUtils, SubmittedDependencyUploaderImpl} +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServerSslOptionsProviderImpl, RetrofitClientFactoryImpl} import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala index 5b5ac3c1f17c2..5fcb0495f0b19 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerSpec.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model.{Container, HasMetadata, Pod} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala index 7aa27a1de6811..b803b67978397 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerConfigurationStep.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File @@ -24,10 +24,10 @@ import io.fabric8.kubernetes.api.model.{Secret, SecretBuilder} import scala.collection.JavaConverters._ import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.SubmittedDependencyUploader +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.SubmittedDependencyUploader import org.apache.spark.internal.config.OptionalConfigEntry import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala index d050e0a41a15a..a40ced5a55be7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/FileFetcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/FileFetcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala index 0e274678ad6f0..1c78b307c2de5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.util.concurrent.TimeUnit @@ -29,8 +29,8 @@ import scala.concurrent.duration.Duration import org.apache.spark.{SecurityManager => SparkSecurityManager, SparkConf, SSLOptions} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.kubernetes.{CompressionUtils, KubernetesCredentials} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{CompressionUtils, KubernetesCredentials} +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala index 17f90118e150d..777ed70d82c22 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/PemsToKeyStoreConverter.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/PemsToKeyStoreConverter.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, FileOutputStream, InputStreamReader} import java.security.{KeyStore, PrivateKey} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala index 0b97317eba8b1..9a63830966dbc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServer.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServer.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File @@ -31,8 +31,8 @@ import org.glassfish.jersey.server.ResourceConfig import org.glassfish.jersey.servlet.ServletContainer import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.SparkKubernetesClientFactory -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{SystemClock, ThreadUtils, Utils} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala index cb1e65421c013..30c5d3917d74a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProvider.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProvider.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File import java.security.SecureRandom @@ -24,8 +24,8 @@ import com.google.common.io.Files import org.apache.commons.lang3.RandomStringUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException, SSLOptions} -import org.apache.spark.deploy.kubernetes.OptionRequirements -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.OptionRequirements +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging private[spark] trait ResourceStagingServerSslOptionsProvider { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala index b9d283a99ade9..b48b2ca4859d6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingService.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingService.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.InputStream import javax.ws.rs.{Consumes, GET, HeaderParam, Path, PathParam, POST, Produces} @@ -22,8 +22,8 @@ import javax.ws.rs.core.{MediaType, StreamingOutput} import org.glassfish.jersey.media.multipart.FormDataParam -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Service that receives application data that can be retrieved later on. This is primarily used diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala index 7bc21c21619e1..ebf37fd277cd6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceImpl.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, OutputStream} import java.security.SecureRandom @@ -26,8 +26,8 @@ import com.google.common.io.{BaseEncoding, ByteStreams, Files} import scala.collection.concurrent.TrieMap import org.apache.spark.SparkException -import org.apache.spark.deploy.kubernetes.KubernetesCredentials -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.KubernetesCredentials +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala index 5fbf0f9c43970..53803f46163e6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServiceRetrofit.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServiceRetrofit.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import okhttp3.{RequestBody, ResponseBody} import retrofit2.Call import retrofit2.http.{Multipart, Path, Streaming} -import org.apache.spark.deploy.kubernetes.submit.SubmittedResourceIdAndSecret +import org.apache.spark.deploy.k8s.submit.SubmittedResourceIdAndSecret /** * Retrofit-compatible variant of {@link ResourceStagingService}. For documentation on diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala index 5046cb479054c..f5d41a5012bc1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/RetrofitClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/RetrofitClientFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.FileInputStream import java.net.{InetSocketAddress, URI} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala index 9e2b8a780df29..4376de42bd751 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/SparkConfPropertiesParser.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/SparkConfPropertiesParser.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala index 81f394800f803..f4038ae05fd99 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResources.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResources.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.File diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala index 5d9db728483fa..38636ac7d992d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleaner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleaner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala index 4061bc36764d7..9a9691371b3f5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesOwner.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesOwner.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import com.fasterxml.jackson.core.`type`.TypeReference import com.fasterxml.jackson.module.scala.JsonScalaEnumeration diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala similarity index 98% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala index 0c0d428e035dc..7cb819dc90804 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStore.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStore.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileOutputStream, InputStream, IOException} import java.security.SecureRandom diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 2339612658537..f39ee1b6710d4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ @@ -22,10 +22,10 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, import org.apache.commons.io.FilenameUtils import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{InitContainerUtil, MountSecretsBootstrap, MountSmallFilesBootstrap} import org.apache.spark.util.Utils // Configures executor pods. Construct one of these with a SparkConf to set up properties that are diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala similarity index 87% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala index e04ab9e541963..8d6d92ac7b412 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/InetAddressUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/InetAddressUtil.scala @@ -14,19 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.net.InetAddress /** * Gets full host names of given IP addresses from DNS. */ -private[kubernetes] trait InetAddressUtil { +private[k8s] trait InetAddressUtil { def getFullHostName(ipAddress: String): String } -private[kubernetes] object InetAddressUtilImpl extends InetAddressUtil { +private[k8s] object InetAddressUtilImpl extends InetAddressUtil { // NOTE: This does issue a network call to DNS. Caching is done internally by the InetAddress // class for both hits and misses. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 764e351e70286..cd92df439a7e6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -14,17 +14,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.io.File import io.fabric8.kubernetes.client.Config import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.{ConfigurationUtils, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} +import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPluginImpl, SparkKubernetesClientFactory, SparkPodInitContainerBootstrapImpl} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrapImpl} import org.apache.spark.internal.Logging import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala similarity index 99% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 1834cddfa7304..d30c88fcc74bf 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.io.Closeable import java.net.InetAddress @@ -30,8 +30,8 @@ import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkEnv, SparkException} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpointAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorExited, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RetrieveSparkAppConfig, SparkAppConfig} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala index d00783b84a948..181f22dfe3dcc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesExternalShuffleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.{Pod, Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watch, Watcher} @@ -25,8 +25,8 @@ import org.apache.commons.io.FilenameUtils import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.kubernetes.ConfigurationUtils -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.ConfigurationUtils +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.internal.Logging import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClient import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala similarity index 95% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala index 20136679e9b84..e4e36c6a942ae 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala @@ -14,10 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s -import org.apache.spark.SparkContext -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, TaskSet, TaskSetManager} import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala similarity index 97% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala index 44d01a5d76b01..39c521978c5b1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManager.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskSet, TaskSetManager} private[spark] class KubernetesTaskSetManager( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala similarity index 96% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala index d73bc6cf93aa2..a3573854835b4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/NodeAffinityExecutorPodModifier.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/NodeAffinityExecutorPodModifier.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule import io.fabric8.kubernetes.api.model.{Pod, PodBuilder} -import org.apache.spark.deploy.kubernetes.constants.ANNOTATION_EXECUTOR_NODE_AFFINITY +import org.apache.spark.deploy.k8s.constants.ANNOTATION_EXECUTOR_NODE_AFFINITY import org.apache.spark.internal.Logging // Applies a node affinity annotation to executor pods so that pods can be placed optimally for diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala similarity index 93% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala index 29a7dc982a5af..0b45aa0c117cc 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtil.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtil.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic @@ -25,14 +25,14 @@ import org.apache.log4j.{Level, Logger} /** * Finds rack names that cluster nodes belong to in order to support HDFS rack locality. */ -private[kubernetes] trait RackResolverUtil { +private[k8s] trait RackResolverUtil { def isConfigured() : Boolean def resolveRack(hadoopConfiguration: Configuration, host: String): Option[String] } -private[kubernetes] class RackResolverUtilImpl(hadoopConfiguration: Configuration) +private[k8s] class RackResolverUtilImpl(hadoopConfiguration: Configuration) extends RackResolverUtil { val scriptPlugin : String = classOf[ScriptBasedMapping].getCanonicalName diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala similarity index 94% rename from resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala rename to resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala index ca1bbbe17076f..789e5f336edc8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/kubernetes/ShuffleServiceConfig.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ShuffleServiceConfig.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s private[spark] case class ShuffleServiceConfig( shuffleNamespace: String, diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala index 5ded567c84287..72ce41d34d5a9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -14,13 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.kubernetes.constants._ class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala index 8de0f56f007dc..0a41cd5d41d0f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SSLUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SSLUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import java.io.{File, FileOutputStream, OutputStreamWriter} import java.math.BigInteger @@ -30,7 +30,7 @@ import org.bouncycastle.cert.jcajce.{JcaX509CertificateConverter, JcaX509v3Certi import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.bouncycastle.operator.jcajce.JcaContentSignerBuilder -import org.apache.spark.deploy.kubernetes.submit.{KeyAndCertPem, KeyStoreAndTrustStore} +import org.apache.spark.deploy.k8s.submit.{KeyAndCertPem, KeyStoreAndTrustStore} import org.apache.spark.util.Utils private[spark] object SSLUtils { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala index 86420e5a9edfd..dc552bb2fae3a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala @@ -14,13 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes +package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.kubernetes.constants._ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 0100dce454a3f..041f51e912002 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import com.google.common.collect.Iterables import io.fabric8.kubernetes.api.model.{ContainerBuilder, DoneablePod, HasMetadata, Pod, PodBuilder, PodList, Secret, SecretBuilder} @@ -29,8 +29,8 @@ import org.scalatest.mock.MockitoSugar._ import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} class ClientSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala similarity index 93% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index fc9159d08fe23..6f5d5e571c443 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala similarity index 89% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala index 213a62bb833e5..ea25b28823c4b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/KubernetesExternalShuffleServiceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala @@ -14,12 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.internal.config +import org.apache.spark.deploy.k8s.KubernetesExternalShuffleService +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.kubernetes.KubernetesExternalShuffleClientImpl diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala index a23ee667004a2..0921228c795c0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/MountSecretsBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/MountSecretsBootstrapSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala index 5240128743b76..83181c1fe3c4e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SSLFilePairs.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SSLFilePairs.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.File diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala index 860bc6e0438aa..8388c16ded268 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SecretVolumeUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SecretVolumeUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import scala.collection.JavaConverters._ diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala index 96fa92c254297..42114f1a6f82b 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/SubmittedDependencyUploaderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/SubmittedDependencyUploaderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit +package org.apache.spark.deploy.k8s.submit import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.UUID @@ -31,8 +31,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Response} import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.rest.kubernetes.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.rest.k8s.{ResourceStagingServiceRetrofit, RetrofitClientFactory, StagedResourcesOwner} import org.apache.spark.util.Utils private[spark] class SubmittedDependencyUploaderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala index 5e3718800d928..1441287244e0a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -14,14 +14,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala index 3f7ec61074b0c..02b2820acaf3f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DependencyResolutionStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DependencyResolutionStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilde import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.constants._ private[spark] class DependencyResolutionStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala index 08c939987603d..2b9570fdd625c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverAddressConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverAddressConfigurationStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model.Service import org.mockito.{Mock, MockitoAnnotations} @@ -23,8 +23,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Clock private[spark] class DriverAddressConfigurationStepSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala index 3d5664713a2b8..b29edca05f4a0 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/DriverKubernetesCredentialsStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.File @@ -25,8 +25,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.util.Utils private[spark] class DriverKubernetesCredentialsStepSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala similarity index 92% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala index b94e7345cd6e1..a0a010ffe916e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSecretsStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSecretsStepSuite.scala @@ -14,10 +14,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.submit.{MountSecretsBootstrapImpl, SecretVolumeUtils} +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, SecretVolumeUtils} private[spark] class MountSecretsStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala index 5e49ab2fc0e36..812031a306063 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/MountSmallLocalFilesStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/MountSmallLocalFilesStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.{File, RandomAccessFile} @@ -27,9 +27,9 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.MountSmallFilesBootstrap +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.MountSmallFilesBootstrap import org.apache.spark.util.Utils private[spark] class MountSmallLocalFilesStepSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala index ce0dcee6acc46..66218f130b9c4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/PythonStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/PythonStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import io.fabric8.kubernetes.api.model._ import org.scalatest.BeforeAndAfter diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initContainerBootstrapStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initContainerBootstrapStepSuite.scala index b11b487111496..ee3b4229b16c1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initContainerBootstrapStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initContainerBootstrapStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps +package org.apache.spark.deploy.k8s.submit.submitsteps import java.io.StringReader import java.util.Properties @@ -26,9 +26,9 @@ import io.fabric8.kubernetes.api.model.{ConfigMap, Container, ContainerBuilder, import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.{InitContainerConfigurationStep, InitContainerSpec} import org.apache.spark.util.Utils private[spark] class initContainerBootstrapStepSuite extends SparkFunSuite { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala similarity index 94% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala index fe1af4bc5be2a..65df5fcebe382 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/BaseInitContainerConfigurationStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import io.fabric8.kubernetes.api.model._ import org.mockito.{Mock, MockitoAnnotations} @@ -26,8 +26,8 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ class BaseInitContainerConfigurationStepSuite extends SparkFunSuite with BeforeAndAfter{ private val SPARK_JARS = Seq( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala index ea4d60795787c..f29d12a9f26cc 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -14,11 +14,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { private val NAMESPACE = "namespace" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala similarity index 96% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala index 2edaba93fe07f..1488c0d00b7a5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/SubmittedResourcesInitContainerStepSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.submit.submitsteps.initcontainer +package org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer import java.io.File import java.util.UUID @@ -31,10 +31,10 @@ import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.kubernetes.InitContainerResourceStagingServerSecretPlugin -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ -import org.apache.spark.deploy.kubernetes.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} +import org.apache.spark.deploy.k8s.InitContainerResourceStagingServerSecretPlugin +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.{SubmittedDependencyUploader, SubmittedResourceIdAndSecret} import org.apache.spark.util.Utils class SubmittedResourcesInitContainerStepSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala index f2fdf026390cd..4c80637da7843 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/KubernetesSparkDependencyDownloadInitContainerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/KubernetesSparkDependencyDownloadInitContainerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayOutputStream, File} import java.util.UUID @@ -33,8 +33,8 @@ import org.scalatest.mock.MockitoSugar._ import retrofit2.{Call, Callback, Response} import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.CompressionUtils -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.CompressionUtils +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.util.Utils class KubernetesSparkDependencyDownloadInitContainerSuite diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala index 3bb318d713a54..1ab7a7f4f966d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSslOptionsProviderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSslOptionsProviderSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{File, FileInputStream, StringWriter} import java.security.KeyStore @@ -25,7 +25,7 @@ import org.bouncycastle.openssl.jcajce.JcaPEMWriter import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.util.Utils class ResourceStagingServerSslOptionsProviderSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala index 1bcd85a611e00..4e6b4abfb6874 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/ResourceStagingServerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/ResourceStagingServerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.net.ServerSocket import javax.ws.rs.core.MediaType @@ -29,7 +29,7 @@ import org.scalatest.mock.MockitoSugar.mock import retrofit2.Call import org.apache.spark.{SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils +import org.apache.spark.deploy.k8s.SSLUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala index 8b398a9891f34..d9b5c4ac5b9c5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesCleanerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesCleanerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.util.concurrent.{ScheduledExecutorService, TimeUnit} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala index 6b5737ebf2e23..b58046da67800 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/kubernetes/StagedResourcesStoreSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/rest/k8s/StagedResourcesStoreSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.rest.kubernetes +package org.apache.spark.deploy.rest.k8s import java.io.{ByteArrayInputStream, File} import java.nio.file.Paths diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala similarity index 99% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index b30d1c2543bea..a9a2937869edd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.{ExecutorService, ScheduledExecutorService, TimeUnit} @@ -31,8 +31,8 @@ import scala.collection.JavaConverters._ import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.constants._ +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEndpoint, RpcEndpointAddress, RpcEndpointRef, RpcEnv, RpcTimeout} import org.apache.spark.scheduler.{ExecutorExited, LiveListenerBus, SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala index e10b9508530f7..4924b67480048 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSchedulerImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.{Pod, PodSpec, PodStatus} import org.mockito.Matchers._ @@ -22,7 +22,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.FakeTask class KubernetesTaskSchedulerImplSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala similarity index 98% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala index 889758731a6c1..56c576a7dd449 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/KubernetesTaskSetManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSetManagerSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import scala.collection.mutable.ArrayBuffer @@ -23,7 +23,7 @@ import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkContext, SparkFunSuite} -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{FakeTask, FakeTaskScheduler, HostTaskLocation, TaskLocation} class KubernetesTaskSetManagerSuite extends SparkFunSuite with BeforeAndAfter { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala similarity index 97% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala index ee671ce7d6590..ec1428117280f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/kubernetes/RackResolverUtilImplSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/RackResolverUtilImplSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.scheduler.cluster.kubernetes +package org.apache.spark.scheduler.cluster.k8s import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.CommonConfigurationKeysPublic diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile index 1178dd2428448..a52f5e1305598 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/init-container/Dockerfile @@ -21,4 +21,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-init:latest -f dockerfiles/init-container/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.KubernetesSparkDependencyDownloadInitContainer" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.KubernetesSparkDependencyDownloadInitContainer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile index 87ed7d10f3eb3..9ddc2c9bcaa0f 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/resource-staging-server/Dockerfile @@ -22,4 +22,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-resource-staging-server:latest -f dockerfiles/resource-staging-server/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.kubernetes.ResourceStagingServer" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.rest.k8s.ResourceStagingServer" ] diff --git a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile index b76e66d316c5c..a240e76a12a15 100644 --- a/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile +++ b/resource-managers/kubernetes/docker-minimal-bundle/src/main/docker/shuffle-service/Dockerfile @@ -21,4 +21,4 @@ FROM spark-base # command should be invoked from the top level directory of the Spark distribution. E.g.: # docker build -t spark-shuffle:latest -f dockerfiles/shuffle-service/Dockerfile . -ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.kubernetes.KubernetesExternalShuffleService", "1" ] +ENTRYPOINT [ "/opt/entrypoint.sh", "/opt/spark/bin/spark-class", "org.apache.spark.deploy.k8s.KubernetesExternalShuffleService", "1" ] diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java similarity index 94% rename from resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java rename to resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java index 99d982397bb6e..b18f8deea6a20 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/kubernetes/integrationtest/PiHelper.java +++ b/resource-managers/kubernetes/integration-tests-spark-jobs-helpers/src/main/java/org/apache/spark/deploy/k8s/integrationtest/PiHelper.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest; +package org.apache.spark.deploy.k8s.integrationtest; /** * Primarily extracted so that a separate jar can be added as a dependency for the diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala index a9e328f4ff248..be1c3f6dc3af4 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/FileExistenceTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/FileExistenceTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.File import java.nio.file.Paths diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala index fe47d42485b24..7bae66cbf71f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/GroupByTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/GroupByTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.util.Random diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala index 967032eddccb5..62cba7f6b0742 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/JavaOptionsTest.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/JavaOptionsTest.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs import java.io.{File, FileInputStream} import java.util.Properties diff --git a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala similarity index 92% rename from resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala rename to resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala index d3372749f999e..5ae22671e41f7 100644 --- a/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/kubernetes/integrationtest/jobs/SparkPiWithInfiniteWait.scala +++ b/resource-managers/kubernetes/integration-tests-spark-jobs/src/main/scala/org/apache/spark/deploy/k8s/integrationtest/jobs/SparkPiWithInfiniteWait.scala @@ -14,9 +14,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.jobs +package org.apache.spark.deploy.k8s.integrationtest.jobs -import org.apache.spark.deploy.kubernetes.integrationtest.PiHelper +import org.apache.spark.deploy.k8s.integrationtest.PiHelper import org.apache.spark.sql.SparkSession // Equivalent to SparkPi except does not stop the Spark Context diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala similarity index 95% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index e204d0173aff8..0d052626d73d0 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, FileOutputStream} import java.nio.file.Paths @@ -29,12 +29,12 @@ import org.scalatest.time.{Minutes, Seconds, Span} import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite, SSLOptions} -import org.apache.spark.deploy.kubernetes.SSLUtils -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackendFactory -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.Minikube -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} +import org.apache.spark.deploy.k8s.SSLUtils +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackendFactory +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.submit.{Client, ClientArguments, JavaMainAppResource, KeyAndCertPem, MainAppResource, PythonMainAppResource} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.Utils @@ -421,7 +421,7 @@ private[spark] object KubernetesSuite { s"integration-tests-jars/${HELPER_JAR_FILE.getName}" val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) val INTERVAL = PatienceConfiguration.Interval(Span(2, Seconds)) - val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val SPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.SparkPiWithInfiniteWait" val PYSPARK_PI_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" val PYSPARK_PI_CONTAINER_LOCAL_FILE_LOCATION = @@ -429,11 +429,11 @@ private[spark] object KubernetesSuite { val PYSPARK_SORT_CONTAINER_LOCAL_FILE_LOCATION = "local:///opt/spark/examples/src/main/python/sort.py" val PYSPARK_PI_SUBMITTER_LOCAL_FILE_LOCATION = "src/test/python/pi.py" - val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val FILE_EXISTENCE_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.FileExistenceTest" - val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val GROUP_BY_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.GroupByTest" - val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.kubernetes" + + val JAVA_OPTIONS_MAIN_CLASS = "org.apache.spark.deploy.k8s" + ".integrationtest.jobs.JavaOptionsTest" val TEST_EXISTENCE_FILE_CONTENTS = "contents" diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala similarity index 95% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 0ca1f482269db..44eb125fb8e77 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.util.UUID @@ -23,7 +23,7 @@ import org.scalatest.concurrent.Eventually import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.deploy.kubernetes.config._ +import org.apache.spark.deploy.k8s.config._ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index 4008007b72fc4..ed15f66f53458 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{BufferedReader, InputStreamReader} import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala index e5e1b1f085f9f..1aa90f4ce691d 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/ResourceStagingServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ResourceStagingServerLauncher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.io.{File, StringWriter} import java.util.Properties @@ -25,8 +25,8 @@ import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.JavaConverters._ import org.apache.spark.SSLOptions -import org.apache.spark.deploy.kubernetes.config._ -import org.apache.spark.deploy.kubernetes.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{ContainerNameEqualityPredicate, KeyAndCertPem} import org.apache.spark.util.Utils /** diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala index bd604ab94b936..f1fd6dc19ce54 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/SparkReadinessWatcher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SparkReadinessWatcher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import java.util.concurrent.TimeUnit diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala index 6b483769f5254..69a9df63bc4af 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/StaticAssetServerLauncher.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/StaticAssetServerLauncher.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest import io.fabric8.kubernetes.api.model.{HTTPGetActionBuilder, Pod} import io.fabric8.kubernetes.client.KubernetesClient diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala index 1ef096be4af02..09a3118a202eb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/GCE/GCETestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/GCE/GCETestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE +package org.apache.spark.deploy.k8s.integrationtest.backend.GCE import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.config.resolveK8sMaster -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.GCE_TEST_BACKEND +import org.apache.spark.deploy.k8s.config.resolveK8sMaster +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.GCE_TEST_BACKEND private[spark] class GCETestBackend(val master: String) extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala similarity index 78% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index c5bc923dd51a6..b61daf65a4093 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend +package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.GCE.GCETestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.GCE.GCETestBackend +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.{Minikube, MinikubeTestBackend} +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] trait IntegrationTestBackend { def name(): String diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala similarity index 97% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala index 7c4b344e8f72b..bd31bde6dabf4 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import java.nio.file.Paths import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} -import org.apache.spark.deploy.kubernetes.integrationtest.ProcessUtils +import org.apache.spark.deploy.k8s.integrationtest.ProcessUtils import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala similarity index 81% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala index 461264877edc2..8e94f13360536 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -14,13 +14,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.backend.minikube +package org.apache.spark.deploy.k8s.integrationtest.backend.minikube import io.fabric8.kubernetes.client.DefaultKubernetesClient -import org.apache.spark.deploy.kubernetes.integrationtest.backend.IntegrationTestBackend -import org.apache.spark.deploy.kubernetes.integrationtest.constants.MINIKUBE_TEST_BACKEND -import org.apache.spark.deploy.kubernetes.integrationtest.docker.SparkDockerImageBuilder +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.deploy.k8s.integrationtest.constants.MINIKUBE_TEST_BACKEND +import org.apache.spark.deploy.k8s.integrationtest.docker.SparkDockerImageBuilder private[spark] class MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala similarity index 93% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala index bfded1003fc25..0807a68cd823c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/constants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/constants.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest +package org.apache.spark.deploy.k8s.integrationtest package object constants { val MINIKUBE_TEST_BACKEND = "minikube" diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala similarity index 98% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala index e240fcf953f8c..0e2fced70c9f7 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/docker/SparkDockerImageBuilder.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/docker/SparkDockerImageBuilder.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.docker +package org.apache.spark.deploy.k8s.integrationtest.docker import java.io.File import java.net.URI diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala similarity index 96% rename from resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala index 7a3b06b1b5e58..6c7ef3c592431 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/kubernetes/integrationtest/restapis/SparkRestApiV1.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/restapis/SparkRestApiV1.scala @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.deploy.kubernetes.integrationtest.restapis +package org.apache.spark.deploy.k8s.integrationtest.restapis import java.util.{List => JList} import javax.ws.rs._ From 84f4602e92c54861a1c615d63c59fc8531fb0f3e Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 22 Sep 2017 12:46:56 -0700 Subject: [PATCH 11/25] Added reference YAML files for RBAC configs for driver and shuffle service (#502) --- conf/k8s-shuffle-service-rbac.yaml | 80 ++++++++++++++++++++++++++++++ conf/k8s-spark-rbac.yaml | 55 ++++++++++++++++++++ 2 files changed, 135 insertions(+) create mode 100644 conf/k8s-shuffle-service-rbac.yaml create mode 100644 conf/k8s-spark-rbac.yaml diff --git a/conf/k8s-shuffle-service-rbac.yaml b/conf/k8s-shuffle-service-rbac.yaml new file mode 100644 index 0000000000000..7ba4fe8e2f15c --- /dev/null +++ b/conf/k8s-shuffle-service-rbac.yaml @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +apiVersion: v1 +kind: ServiceAccount +metadata: + name: spark-shuffle-service-service-account + namespace: default + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +--- +apiVersion: extensions/v1beta1 +kind: PodSecurityPolicy +metadata: + name: spark-shuffle-service-pod-security-policy + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +spec: + privileged: false + fsGroup: + rule: RunAsAny + runAsUser: + rule: RunAsAny + volumes: + - "hostPath" + - "secret" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: ClusterRole +metadata: + name: spark-shuffle-service-role + labels: + app: spark-shuffle-service + spark-version: 2.2.0 +rules: +- apiGroups: + - "extensions" + resources: + - "podsecuritypolicies" + resourceNames: + - "spark-shuffle-service-pod-security-policy" + verbs: + - "use" +- apiGroups: + - "" # "" indicates the core API group + resources: + - "pods" + verbs: + - "get" + - "list" + - "watch" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: ClusterRoleBinding +metadata: + name: spark-shuffle-service-role-binding +subjects: +- kind: ServiceAccount + name: spark-shuffle-service-service-account + namespace: default +roleRef: + apiGroup: rbac.authorization.k8s.io + kind: ClusterRole + name: spark-shuffle-service-role diff --git a/conf/k8s-spark-rbac.yaml b/conf/k8s-spark-rbac.yaml new file mode 100644 index 0000000000000..40381d39f2b07 --- /dev/null +++ b/conf/k8s-spark-rbac.yaml @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +apiVersion: v1 +kind: ServiceAccount +metadata: + name: spark + namespace: default +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: Role +metadata: + namespace: default + name: spark-role +rules: +- apiGroups: + - "" # "" indicates the core API group + resources: + - "pods" + verbs: + - "*" +- apiGroups: + - "" # "" indicates the core API group + resources: + - "services" + verbs: + - "*" +--- +apiVersion: rbac.authorization.k8s.io/v1beta1 +kind: RoleBinding +metadata: + name: spark-role-binding + namespace: default +subjects: +- kind: ServiceAccount + name: spark + namespace: default +roleRef: + kind: Role + name: spark-role + apiGroup: rbac.authorization.k8s.io From 0dff8be213eadee5483700b691d15c738364b454 Mon Sep 17 00:00:00 2001 From: Yinan Li Date: Fri, 22 Sep 2017 15:57:13 -0700 Subject: [PATCH 12/25] Removing deprecated configuration (#503) Replaced #501 by @foxish. --- docs/running-on-kubernetes.md | 42 ------------------- .../spark/deploy/k8s/ConfigurationUtils.scala | 26 ------------ .../org/apache/spark/deploy/k8s/config.scala | 33 +-------------- ...DriverConfigurationStepsOrchestrator.scala | 5 +-- .../BaseDriverConfigurationStep.scala | 3 +- .../cluster/k8s/ExecutorPodFactory.scala | 7 +--- .../BaseDriverConfigurationStepSuite.scala | 5 --- ... => InitContainerBootstrapStepSuite.scala} | 0 ...rConfigurationStepsOrchestratorSuite.scala | 5 --- .../k8s/integrationtest/KubernetesSuite.scala | 2 +- 10 files changed, 7 insertions(+), 121 deletions(-) rename resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/{initContainerBootstrapStepSuite.scala => InitContainerBootstrapStepSuite.scala} (100%) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 6bf5140af6a3c..ae1e9adb7f0ba 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -608,48 +608,6 @@ from the other deployment modes. See the [configuration page](configuration.html myIdentifier. Multiple annotations can be added by setting multiple configurations with this prefix. - - spark.kubernetes.driver.labels - (none) - - Deprecated. Use spark.kubernetes.driver.label. instead which supports = - and , characters in label values. - Custom labels that will be added to the driver pod. This should be a comma-separated list of label key-value pairs, - where each label is in the format key=value. Note that Spark also adds its own labels to the driver pod - for bookkeeping purposes. - - - - spark.kubernetes.driver.annotations - (none) - - Deprecated. Use spark.kubernetes.driver.annotation. instead which supports - = and , characters in annotation values. - Custom annotations that will be added to the driver pod. This should be a comma-separated list of label key-value - pairs, where each annotation is in the format key=value. - - - - spark.kubernetes.executor.labels - (none) - - Deprecated. Use spark.kubernetes.executor.label. instead which supports - = and , characters in label values. - Custom labels that will be added to the executor pods. This should be a comma-separated list of label key-value - pairs, where each label is in the format key=value. Note that Spark also adds its own labels to the - executor pods for bookkeeping purposes. - - - - spark.kubernetes.executor.annotations - (none) - - Deprecated. Use spark.kubernetes.executor.annotation. instead which supports - = and , characters in annotation values. - Custom annotations that will be added to the executor pods. This should be a comma-separated list of annotation - key-value pairs, where each annotation is in the format key=value. - - spark.kubernetes.driver.pod.name (none) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala index 7645e54e46280..601533615ce9a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/ConfigurationUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.k8s import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.OptionalConfigEntry object ConfigurationUtils extends Logging { def parseKeyValuePairs( @@ -41,31 +40,6 @@ object ConfigurationUtils extends Logging { }).getOrElse(Map.empty[String, String]) } - def combinePrefixedKeyValuePairsWithDeprecatedConf( - sparkConf: SparkConf, - prefix: String, - deprecatedConf: OptionalConfigEntry[String], - configType: String): Map[String, String] = { - val deprecatedKeyValuePairsString = sparkConf.get(deprecatedConf) - deprecatedKeyValuePairsString.foreach { _ => - logWarning(s"Configuration with key ${deprecatedConf.key} is deprecated. Use" + - s" configurations with prefix $prefix instead.") - } - val fromDeprecated = parseKeyValuePairs( - deprecatedKeyValuePairsString, - deprecatedConf.key, - configType) - val fromPrefix = sparkConf.getAllWithPrefix(prefix) - val combined = fromDeprecated.toSeq ++ fromPrefix - combined.groupBy(_._1).foreach { - case (key, values) => - require(values.size == 1, - s"Cannot have multiple values for a given $configType key, got key $key with" + - s" values $values") - } - combined.toMap - } - def parsePrefixedKeyValuePairs( sparkConf: SparkConf, prefix: String, 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 ebabed956365d..fc08c0ad42f82 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 @@ -115,43 +115,12 @@ package object config extends Logging { private[spark] val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." private[spark] val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." + private[spark] val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label." private[spark] val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation." - private[spark] val KUBERNETES_DRIVER_LABELS = - ConfigBuilder("spark.kubernetes.driver.labels") - .doc("Custom labels that will be added to the driver pod. This should be a comma-separated" + - " list of label key-value pairs, where each label is in the format key=value. Note that" + - " Spark also adds its own labels to the driver pod for bookkeeping purposes.") - .stringConf - .createOptional - private[spark] val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv." - private[spark] val KUBERNETES_DRIVER_ANNOTATIONS = - ConfigBuilder("spark.kubernetes.driver.annotations") - .doc("Custom annotations that will be added to the driver pod. This should be a" + - " comma-separated list of annotation key-value pairs, where each annotation is in the" + - " format key=value.") - .stringConf - .createOptional - - private[spark] val KUBERNETES_EXECUTOR_LABELS = - ConfigBuilder("spark.kubernetes.executor.labels") - .doc("Custom labels that will be added to the executor pods. This should be a" + - " comma-separated list of label key-value pairs, where each label is in the format" + - " key=value.") - .stringConf - .createOptional - - private[spark] val KUBERNETES_EXECUTOR_ANNOTATIONS = - ConfigBuilder("spark.kubernetes.executor.annotations") - .doc("Custom annotations that will be added to the executor pods. This should be a" + - " comma-separated list of annotation key-value pairs, where each annotation is in the" + - " format key=value.") - .stringConf - .createOptional - private[spark] val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." private[spark] val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets." diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 17e4a18a07d47..fd251637ce210 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -72,10 +72,9 @@ private[spark] class DriverConfigurationStepsOrchestrator( .getOrElse(Array.empty[String]) ++ additionalMainAppPythonFile.toSeq ++ additionalPythonFiles - val driverCustomLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + val driverCustomLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( submissionSparkConf, KUBERNETES_DRIVER_LABEL_PREFIX, - KUBERNETES_DRIVER_LABELS, "label") require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), s"Label with key " + s" $SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping" + @@ -124,7 +123,7 @@ private[spark] class DriverConfigurationStepsOrchestrator( // Then, indicate to the outer block that the init-container should not handle // those local files simply by filtering them out. val sparkFilesWithoutLocal = KubernetesFileUtils.getNonSubmitterLocalFiles(sparkFiles) - val smallFilesSecretName = s"${kubernetesAppId}-submitted-files" + val smallFilesSecretName = s"$kubernetesAppId-submitted-files" val mountSmallFilesBootstrap = new MountSmallFilesBootstrapImpl( smallFilesSecretName, MOUNTED_SMALL_FILES_SECRET_MOUNT_PATH) val mountSmallLocalFilesStep = new MountSmallLocalFilesStep( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala index 0eb9ee2563045..563662ef80d11 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStep.scala @@ -67,10 +67,9 @@ private[spark] class BaseDriverConfigurationStep( .build() } val driverCustomAnnotations = ConfigurationUtils - .combinePrefixedKeyValuePairsWithDeprecatedConf( + .parsePrefixedKeyValuePairs( submissionSparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX, - KUBERNETES_DRIVER_ANNOTATIONS, "annotation") require(!driverCustomAnnotations.contains(SPARK_APP_NAME_ANNOTATION), s"Annotation with key $SPARK_APP_NAME_ANNOTATION is not allowed as it is reserved for" + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index f39ee1b6710d4..6f4ba1c8b888f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler.cluster.k8s import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} -import org.apache.commons.io.FilenameUtils import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} @@ -56,10 +55,9 @@ private[spark] class ExecutorPodFactoryImpl( org.apache.spark.internal.config.EXECUTOR_CLASS_PATH) private val executorJarsDownloadDir = sparkConf.get(INIT_CONTAINER_JARS_DOWNLOAD_LOCATION) - private val executorLabels = ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + private val executorLabels = ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX, - KUBERNETES_EXECUTOR_LABELS, "executor label") require( !executorLabels.contains(SPARK_APP_ID_LABEL), @@ -70,10 +68,9 @@ private[spark] class ExecutorPodFactoryImpl( s" Spark.") private val executorAnnotations = - ConfigurationUtils.combinePrefixedKeyValuePairsWithDeprecatedConf( + ConfigurationUtils.parsePrefixedKeyValuePairs( sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, - KUBERNETES_EXECUTOR_ANNOTATIONS, "executor annotation") private val nodeSelector = ConfigurationUtils.parsePrefixedKeyValuePairs( diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala index 1441287244e0a..330d8e87a9fe9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/BaseDriverConfigurationStepSuite.scala @@ -34,8 +34,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { private val APP_ARGS = Array("arg1", "arg2") private val CUSTOM_ANNOTATION_KEY = "customAnnotation" private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DEPRECATED_CUSTOM_ANNOTATION_KEY = "customAnnotationDeprecated" - private val DEPRECATED_CUSTOM_ANNOTATION_VALUE = "customAnnotationDeprecatedValue" private val DRIVER_CUSTOM_ENV_KEY1 = "customDriverEnv1" private val DRIVER_CUSTOM_ENV_KEY2 = "customDriverEnv2" @@ -49,8 +47,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { .set(KUBERNETES_DRIVER_MEMORY_OVERHEAD, 200L) .set(DRIVER_DOCKER_IMAGE, "spark-driver:latest") .set(s"spark.kubernetes.driver.annotation.$CUSTOM_ANNOTATION_KEY", CUSTOM_ANNOTATION_VALUE) - .set("spark.kubernetes.driver.annotations", - s"$DEPRECATED_CUSTOM_ANNOTATION_KEY=$DEPRECATED_CUSTOM_ANNOTATION_VALUE") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY1", "customDriverEnv1") .set(s"$KUBERNETES_DRIVER_ENV_KEY$DRIVER_CUSTOM_ENV_KEY2", "customDriverEnv2") @@ -98,7 +94,6 @@ private[spark] class BaseDriverConfigurationStepSuite extends SparkFunSuite { assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) val expectedAnnotations = Map( CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE, - DEPRECATED_CUSTOM_ANNOTATION_KEY -> DEPRECATED_CUSTOM_ANNOTATION_VALUE, SPARK_APP_NAME_ANNOTATION -> APP_NAME) assert(driverPodMetadata.getAnnotations.asScala === expectedAnnotations) assert(preparedDriverSpec.driverPod.getSpec.getRestartPolicy === "Never") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initContainerBootstrapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala similarity index 100% rename from resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initContainerBootstrapStepSuite.scala rename to resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/InitContainerBootstrapStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala index f29d12a9f26cc..3810a324f99b5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/submitsteps/initcontainer/InitContainerConfigurationStepsOrchestratorSuite.scala @@ -46,7 +46,6 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("error thrown if local jars provided without resource staging server") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) assert(sparkConf.get(RESOURCE_STAGING_SERVER_URI).isEmpty) @@ -72,7 +71,6 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("error not thrown with non-local jars and resource staging server provided") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) @@ -97,7 +95,6 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("error not thrown with non-local jars and no resource staging server provided") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) val orchestrator = new InitContainerConfigurationStepsOrchestrator( @@ -120,7 +117,6 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("including step to contact resource staging server") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) .set(RESOURCE_STAGING_SERVER_URI, STAGING_SERVER_URI) @@ -145,7 +141,6 @@ class InitContainerConfigurationStepsOrchestratorSuite extends SparkFunSuite { test ("not including steps because no contact to resource staging server") { val sparkConf = new SparkConf(true) - .set(KUBERNETES_DRIVER_LABELS, s"$DEPRECATED_CUSTOM_LABEL_KEY=$DEPRECATED_CUSTOM_LABEL_VALUE") .set(s"$KUBERNETES_DRIVER_LABEL_PREFIX$CUSTOM_LABEL_KEY", CUSTOM_LABEL_VALUE) val orchestrator = new InitContainerConfigurationStepsOrchestrator( diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 0d052626d73d0..d6e12df9d1ba2 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -64,7 +64,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf = kubernetesTestComponents.newSparkConf() .set(INIT_CONTAINER_DOCKER_IMAGE, s"spark-init:latest") .set(DRIVER_DOCKER_IMAGE, s"spark-driver:latest") - .set(KUBERNETES_DRIVER_LABELS, s"spark-app-locator=$APP_LOCATOR_LABEL") + .set(s"${KUBERNETES_DRIVER_LABEL_PREFIX}spark-app-locator", APP_LOCATOR_LABEL) kubernetesTestComponents.createNamespace() } From c42952d0a14caca8d6f89b2935051f9dd64f4c59 Mon Sep 17 00:00:00 2001 From: Hideaki Tanaka Date: Thu, 17 Aug 2017 22:02:13 +0800 Subject: [PATCH 13/25] [SPARK-21642][CORE] Use FQDN for DRIVER_HOST_ADDRESS instead of ip address ## What changes were proposed in this pull request? The patch lets spark web ui use FQDN as its hostname instead of ip address. In current implementation, ip address of a driver host is set to DRIVER_HOST_ADDRESS. This becomes a problem when we enable SSL using "spark.ssl.enabled", "spark.ssl.trustStore" and "spark.ssl.keyStore" properties. When we configure these properties, spark web ui is launched with SSL enabled and the HTTPS server is configured with the custom SSL certificate you configured in these properties. In this case, client gets javax.net.ssl.SSLPeerUnverifiedException exception when the client accesses the spark web ui because the client fails to verify the SSL certificate (Common Name of the SSL cert does not match with DRIVER_HOST_ADDRESS). To avoid the exception, we should use FQDN of the driver host for DRIVER_HOST_ADDRESS. Error message that client gets when the client accesses spark web ui: javax.net.ssl.SSLPeerUnverifiedException: Certificate for <10.102.138.239> doesn't match any of the subject alternative names: [] ## How was this patch tested? manual tests Author: Hideaki Tanaka Closes #18846 from thideeeee/SPARK-21642. --- .../scala/org/apache/spark/internal/config/package.scala | 2 +- core/src/main/scala/org/apache/spark/util/Utils.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 6f7a86748e240..39bc95994d7c3 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -253,7 +253,7 @@ package object config { private[spark] val DRIVER_HOST_ADDRESS = ConfigBuilder("spark.driver.host") .doc("Address of driver endpoints.") .stringConf - .createWithDefault(Utils.localHostName()) + .createWithDefault(Utils.localCanonicalHostName()) private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") .doc("Address where to bind network listen sockets on the driver.") diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 83765118521fa..836e33c36d9a1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -940,6 +940,13 @@ private[spark] object Utils extends Logging { customHostname = Some(hostname) } + /** + * Get the local machine's FQDN. + */ + def localCanonicalHostName(): String = { + customHostname.getOrElse(localIpAddress.getCanonicalHostName) + } + /** * Get the local machine's hostname. */ From f0a50aaa974be25709560ea7cba378d73ee450fd Mon Sep 17 00:00:00 2001 From: mcheah Date: Tue, 26 Sep 2017 14:37:29 -0700 Subject: [PATCH 14/25] Fix scalastyle --- .../main/scala/org/apache/spark/deploy/k8s/submit/Client.scala | 2 +- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala index ba23caea49498..0190d3eccfe3b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/Client.scala @@ -24,10 +24,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.submitsteps.{DriverConfigurationStep, KubernetesDriverSpec} -import org.apache.spark.deploy.k8s.SparkKubernetesClientFactory import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index d30c88fcc74bf..9b665dc7fcb73 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -25,8 +25,8 @@ import java.util.concurrent.atomic.{AtomicInteger, AtomicLong, AtomicReference} import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action -import scala.collection.mutable import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import org.apache.spark.{SparkEnv, SparkException} From 375fa395051dc2c810c7dff9cdc0f9f0f49fb1d2 Mon Sep 17 00:00:00 2001 From: mcheah Date: Thu, 12 Oct 2017 14:55:29 -0700 Subject: [PATCH 15/25] Empty commit to retrigger build From 3a9a591e05b932e056662ba11bdaca0ea7cfac31 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Mon, 25 Sep 2017 09:54:20 -0700 Subject: [PATCH 16/25] Update poms for 2.2 release 0.4.0 (#508) * Update POMs * Update extensions/v1beta1.Deployment to apps * Modified defaults on rss and ss (cherry picked from commit 562f301e69f01f86f5d2cc4879dcad5728476ed1) --- conf/kubernetes-resource-staging-server.yaml | 6 +++--- conf/kubernetes-shuffle-service.yaml | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/conf/kubernetes-resource-staging-server.yaml b/conf/kubernetes-resource-staging-server.yaml index 80d59b8091903..18b446574fdaa 100644 --- a/conf/kubernetes-resource-staging-server.yaml +++ b/conf/kubernetes-resource-staging-server.yaml @@ -15,7 +15,7 @@ # limitations under the License. # --- -apiVersion: extensions/v1beta1 +apiVersion: apps/v1beta1 kind: Deployment metadata: name: spark-resource-staging-server @@ -32,14 +32,14 @@ spec: name: spark-resource-staging-server-config containers: - name: spark-resource-staging-server - image: kubespark/spark-resource-staging-server:v2.2.0-kubernetes-0.3.0 + image: kubespark/spark-resource-staging-server:v2.2.0-kubernetes-0.4.0 resources: requests: cpu: 100m memory: 256Mi limits: cpu: 100m - memory: 256Mi + memory: 1Gi volumeMounts: - name: resource-staging-server-properties mountPath: '/etc/spark-resource-staging-server' diff --git a/conf/kubernetes-shuffle-service.yaml b/conf/kubernetes-shuffle-service.yaml index 8ab0b362ea32e..6f02106ff06c5 100644 --- a/conf/kubernetes-shuffle-service.yaml +++ b/conf/kubernetes-shuffle-service.yaml @@ -32,13 +32,13 @@ spec: volumes: - name: temp-volume hostPath: - path: '/var/tmp' # change this path according to your cluster configuration. + path: '/tmp' # change this path according to your cluster configuration. containers: - name: shuffle # This is an official image that is built # from the dockerfiles/shuffle directory # in the spark distribution. - image: kubespark/spark-shuffle:v2.2.0-kubernetes-0.3.0 + image: kubespark/spark-shuffle:v2.2.0-kubernetes-0.4.0 imagePullPolicy: IfNotPresent volumeMounts: - mountPath: '/tmp' From 8314cbc4da30c741f2a44cdf1b60d654facf11ef Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Tue, 26 Sep 2017 11:07:39 -0700 Subject: [PATCH 17/25] Update POM to 0.5.0-SNAPSHOT (#512) (cherry picked from commit 3c7dec5c68e3e09aaa8fa31b682dacd35d46af09) From 0fba1194e7cb308ae744b1cdb433f776ea5d0cb2 Mon Sep 17 00:00:00 2001 From: Anirudh Ramanathan Date: Tue, 10 Oct 2017 14:48:16 -0700 Subject: [PATCH 18/25] Add unit-testing for executorpodfactory (#491) * Unit test for executorpodfactory * Fix test * Indentation fix * Fix isEmpty and split between lines * Address issues with multi-line code fragments * Replace == with === * mock shuffleManager * .kubernetes. => .k8s. * move to k8s subdir * fix package clause to k8s * mock nodeAffinityExecutorPodModifier * remove commented code * move when clause to before{} block * mock initContainerBootstrap, smallFiles * insert actual logic into smallFiles mock * verify application of nodeAffinityExecutorPodModifier * avoid cumulative invocation * Fixed env-var check to include values, removed mock for small files (cherry picked from commit 887fdce61af8e71ea02f25cd55f28f3febc78f1a) --- .../cluster/k8s/ExecutorPodFactorySuite.scala | 305 ++++++++++++++++++ 1 file changed, 305 insertions(+) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala new file mode 100644 index 0000000000000..6690217557637 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -0,0 +1,305 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Pod, VolumeBuilder, VolumeMountBuilder, _} +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.commons.io.FilenameUtils +import org.mockito.{AdditionalAnswers, MockitoAnnotations} +import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.Mockito._ +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{constants, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrap, MountSmallFilesBootstrapImpl} + +class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { + private val driverPodName: String = "driver-pod" + private val driverPodUid: String = "driver-uid" + private val driverUrl: String = "driver-url" + private val executorPrefix: String = "base" + private val executorImage: String = "executor-image" + private val driverPod = new PodBuilder() + .withNewMetadata() + .withName(driverPodName) + .withUid(driverPodUid) + .endMetadata() + .withNewSpec() + .withNodeName("some-node") + .endSpec() + .withNewStatus() + .withHostIP("192.168.99.100") + .endStatus() + .build() + private var baseConf: SparkConf = _ + private val nodeAffinityExecutorPodModifier = mock(classOf[NodeAffinityExecutorPodModifier]) + + before { + MockitoAnnotations.initMocks(this) + baseConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, driverPodName) + .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix) + .set(EXECUTOR_DOCKER_IMAGE, executorImage) + } + private var kubernetesClient: KubernetesClient = _ + + override def beforeEach(cmap: org.scalatest.ConfigMap) { + reset(nodeAffinityExecutorPodModifier) + when(nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( + any(classOf[Pod]), + any(classOf[Map[String, Int]]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + } + + test("basic executor pod has reasonable defaults") { + val factory = new ExecutorPodFactoryImpl( + baseConf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + // The executor pod name and default labels. + assert(executor.getMetadata.getName === s"$executorPrefix-exec-1") + assert(executor.getMetadata.getLabels.size() === 3) + + // There is exactly 1 container with no volume mounts and default memory limits. + // Default memory limit is 1024M + 384M (minimum overhead constant). + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getImage === executorImage) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.isEmpty) + assert(executor.getSpec.getContainers.get(0).getResources.getLimits.size() === 1) + assert(executor.getSpec.getContainers.get(0).getResources + .getLimits.get("memory").getAmount === "1408Mi") + + // The pod has no node selector, volumes. + assert(executor.getSpec.getNodeSelector.isEmpty) + assert(executor.getSpec.getVolumes.isEmpty) + + checkEnv(executor, Map()) + checkOwnerReferences(executor, driverPodUid) + } + + test("executor pod hostnames get truncated to 63 characters") { + val conf = baseConf.clone() + conf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, + "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") + + val factory = new ExecutorPodFactoryImpl( + conf, nodeAffinityExecutorPodModifier, None, None, None, None, None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getHostname.length === 63) + } + + test("secrets get mounted") { + val conf = baseConf.clone() + + val secretsBootstrap = new MountSecretsBootstrapImpl(Map("secret1" -> "/var/secret1")) + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + Some(secretsBootstrap), + None, + None, + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName + === "secret1-volume") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + // check volume mounted. + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + } + + test("init-container bootstrap step adds an init container") { + val conf = baseConf.clone() + val initContainerBootstrap = mock(classOf[SparkPodInitContainerBootstrap]) + when(initContainerBootstrap.bootstrapInitContainerAndVolumes( + any(classOf[PodWithDetachedInitContainer]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + Some(initContainerBootstrap), + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getMetadata.getAnnotations.size() === 1) + assert(executor.getMetadata.getAnnotations.containsKey(constants.INIT_CONTAINER_ANNOTATION)) + checkOwnerReferences(executor, driverPodUid) + } + + test("the shuffle-service adds a volume mount") { + val conf = baseConf.clone() + conf.set(KUBERNETES_SHUFFLE_LABELS, "label=value") + conf.set(KUBERNETES_SHUFFLE_NAMESPACE, "default") + conf.set(KUBERNETES_SHUFFLE_DIR, "/tmp") + + val shuffleManager = mock(classOf[KubernetesExternalShuffleManager]) + when(shuffleManager.getExecutorShuffleDirVolumesWithMounts).thenReturn({ + val shuffleDirs = Seq("/tmp") + shuffleDirs.zipWithIndex.map { case (shuffleDir, shuffleDirIndex) => + val volumeName = s"$shuffleDirIndex-${FilenameUtils.getBaseName(shuffleDir)}" + val volume = new VolumeBuilder() + .withName(volumeName) + .withNewHostPath(shuffleDir) + .build() + val volumeMount = new VolumeMountBuilder() + .withName(volumeName) + .withMountPath(shuffleDir) + .build() + (volume, volumeMount) + } + }) + + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + Some(shuffleManager)) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName === "0-tmp") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/tmp") + checkOwnerReferences(executor, driverPodUid) + } + + test("Small-files add a secret & secret volume mount to the container") { + val conf = baseConf.clone() + + val smallFiles = new MountSmallFilesBootstrapImpl("secret1", "/var/secret1") + val factory = new ExecutorPodFactoryImpl( + conf, + nodeAffinityExecutorPodModifier, + None, + Some(smallFiles), + None, + None, + None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getName === "submitted-files") + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) + .getMountPath === "/var/secret1") + + assert(executor.getSpec.getVolumes.size() === 1) + assert(executor.getSpec.getVolumes.get(0).getSecret.getSecretName === "secret1") + + checkOwnerReferences(executor, driverPodUid) + checkEnv(executor, Map("SPARK_MOUNTED_FILES_FROM_SECRET_DIR" -> "/var/secret1")) + } + + test("classpath and extra java options get translated into environment variables") { + val conf = baseConf.clone() + conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") + conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") + + val factory = new ExecutorPodFactoryImpl( + conf, nodeAffinityExecutorPodModifier, None, None, None, None, None) + val executor = factory.createExecutorPod( + "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) + + verify(nodeAffinityExecutorPodModifier, times(1)) + .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) + + checkEnv(executor, + Map("SPARK_JAVA_OPT_0" -> "foo=bar", + "SPARK_EXECUTOR_EXTRA_CLASSPATH" -> "bar=baz", + "qux" -> "quux")) + checkOwnerReferences(executor, driverPodUid) + } + + // There is always exactly one controller reference, and it points to the driver pod. + private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { + assert(executor.getMetadata.getOwnerReferences.size() === 1) + assert(executor.getMetadata.getOwnerReferences.get(0).getUid === driverPodUid) + assert(executor.getMetadata.getOwnerReferences.get(0).getController === true) + } + + // Check that the expected environment variables are present. + private def checkEnv(executor: Pod, additionalEnvVars: Map[String, String]): Unit = { + val defaultEnvs = Map( + constants.ENV_EXECUTOR_ID -> "1", + constants.ENV_DRIVER_URL -> "dummy", + constants.ENV_EXECUTOR_CORES -> "1", + constants.ENV_EXECUTOR_MEMORY -> "1g", + constants.ENV_APPLICATION_ID -> "dummy", + constants.ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", + constants.ENV_EXECUTOR_POD_IP -> null, + constants.ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + + assert(executor.getSpec.getContainers.size() === 1) + assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) + val mapEnvs = executor.getSpec.getContainers.get(0).getEnv.asScala.map { + x => (x.getName, x.getValue) + }.toMap + assert(defaultEnvs === mapEnvs) + } +} From 56310dbff7d7dfa84d0c955833fdff54c825f937 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Oct 2017 14:47:55 -0700 Subject: [PATCH 19/25] Fix compilation error with external shuffle client. --- .../kubernetes/KubernetesExternalShuffleClientImpl.java | 5 +++-- .../scheduler/cluster/k8s/KubernetesClusterManager.scala | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java index 4302eff0c31f2..b33ffd0d97d4e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/kubernetes/KubernetesExternalShuffleClientImpl.java @@ -49,8 +49,9 @@ public class KubernetesExternalShuffleClientImpl public KubernetesExternalShuffleClientImpl( TransportConf conf, SecretKeyHolder secretKeyHolder, - boolean saslEnabled) { - super(conf, secretKeyHolder, saslEnabled); + boolean saslEnabled, + long registrationTimeoutMs) { + super(conf, secretKeyHolder, saslEnabled, registrationTimeoutMs); } @Override diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index cd92df439a7e6..8385cb61dc986 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -117,7 +117,8 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit val kubernetesExternalShuffleClient = new KubernetesExternalShuffleClientImpl( SparkTransportConf.fromSparkConf(sparkConf, "shuffle"), sc.env.securityManager, - sc.env.securityManager.isAuthenticationEnabled()) + sc.env.securityManager.isAuthenticationEnabled(), + sparkConf.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) Some(new KubernetesExternalShuffleManagerImpl( sparkConf, kubernetesClient, From 0e04996fef0a627bdb8ecb359220d81abfc12b13 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Oct 2017 15:02:11 -0700 Subject: [PATCH 20/25] Add hadoop delegation creds --- .../cluster/k8s/KubernetesClusterSchedulerBackend.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index 9b665dc7fcb73..687e89d659fd8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -464,7 +464,8 @@ private[spark] class KubernetesClusterSchedulerBackend( .getShuffleServiceConfigurationForExecutor(runningExecutorPod) val reply = SparkAppConfig( sparkProperties ++ shuffleSpecificProperties, - SparkEnv.get.securityManager.getIOEncryptionKey()) + SparkEnv.get.securityManager.getIOEncryptionKey(), + hadoopDelegationCreds) context.reply(reply) } } From ea36f4c5e20cb8c0cab0d6da0b3e74479f4160ec Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Oct 2017 15:14:03 -0700 Subject: [PATCH 21/25] Fix missing import --- .../scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala index e4e36c6a942ae..ec56d662195f1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesTaskSchedulerImpl.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.scheduler.cluster.k8s +import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.config._ import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, TaskSet, TaskSetManager} import org.apache.spark.util.Utils From 6a392448dbcd029f29dcd176a4f373a157b73ec0 Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Oct 2017 15:35:57 -0700 Subject: [PATCH 22/25] Fix more imports --- .../spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala | 1 - .../k8s/submit/KubernetesExternalShuffleServiceSuite.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala index dc552bb2fae3a..78abea5754e56 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/SparkPodInitContainerBootstrapSuite.scala @@ -22,7 +22,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s.constants._ -import org.apache.spark.deploy.kubernetes.constants._ class SparkPodInitContainerBootstrapSuite extends SparkFunSuite with BeforeAndAfter { private val INIT_CONTAINER_IMAGE = "spark-init:latest" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala index ea25b28823c4b..2a568c95d21b3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesExternalShuffleServiceSuite.scala @@ -36,7 +36,7 @@ private[spark] class KubernetesExternalShuffleServiceSuite extends SparkFunSuite SparkTransportConf.fromSparkConf(SPARK_CONF, "shuffle"), new SecurityManager(SPARK_CONF), false, - SPARK_CONF.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) + SPARK_CONF.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) shuffleService.start() shuffleClient.init("newapp") From a86c00202d554e15014b3e3058921e964926993b Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 16 Oct 2017 16:08:03 -0700 Subject: [PATCH 23/25] Fix more imports --- .../InitContainerResourceStagingServerSecretPluginSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala index 72ce41d34d5a9..b4e9ee104fa52 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/InitContainerResourceStagingServerSecretPluginSuite.scala @@ -22,7 +22,6 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s.constants._ -import org.apache.spark.deploy.kubernetes.constants._ class InitContainerResourceStagingServerSecretPluginSuite extends SparkFunSuite with BeforeAndAfter{ private val INIT_CONTAINER_SECRET_NAME = "init-secret" From 79605105cd641cccea46d27870531fe25741fa7d Mon Sep 17 00:00:00 2001 From: mccheah Date: Mon, 16 Oct 2017 16:07:35 -0700 Subject: [PATCH 24/25] Mount emptyDir volumes for temporary directories on executors in static allocation mode (rebased) (#522) * Use emptyDir volume mounts for executor local directories. * Mount local dirs in the driver. Remove shuffle dir configuration. * Arrange imports * Fix style and integration tests. * Add TODO note for volume types to change. * Add unit test and extra documentation. * Fix existing unit tests and add tests for empty dir volumes * Remove extraneous constant (cherry picked from commit 49932d6018a13f1885733dce0589d24bc7d76c8b) Conflicts: resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala --- conf/kubernetes-shuffle-service.yaml | 4 +- docs/running-on-kubernetes.md | 11 +- .../org/apache/spark/deploy/k8s/config.scala | 6 - .../apache/spark/deploy/k8s/constants.scala | 1 + ...DriverConfigurationStepsOrchestrator.scala | 7 +- ...LocalDirectoryMountConfigurationStep.scala | 98 ++++++++++++++ .../k8s/ExecutorLocalDirVolumeProvider.scala | 61 +++++++++ .../cluster/k8s/ExecutorPodFactory.scala | 11 +- .../k8s/KubernetesClusterManager.scala | 21 +-- .../KubernetesExternalShuffleManager.scala | 10 +- ...rConfigurationStepsOrchestratorSuite.scala | 9 +- ...DirectoryMountConfigurationStepSuite.scala | 91 +++++++++++++ .../ExecutorLocalDirVolumeProviderSuite.scala | 64 +++++++++ .../cluster/k8s/ExecutorPodFactorySuite.scala | 123 +++++++++--------- .../k8s/integrationtest/KubernetesSuite.scala | 1 + 15 files changed, 424 insertions(+), 94 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala diff --git a/conf/kubernetes-shuffle-service.yaml b/conf/kubernetes-shuffle-service.yaml index 6f02106ff06c5..b22adb265fd18 100644 --- a/conf/kubernetes-shuffle-service.yaml +++ b/conf/kubernetes-shuffle-service.yaml @@ -32,7 +32,7 @@ spec: volumes: - name: temp-volume hostPath: - path: '/tmp' # change this path according to your cluster configuration. + path: '/tmp/spark-local' # change this path according to your cluster configuration. containers: - name: shuffle # This is an official image that is built @@ -41,7 +41,7 @@ spec: image: kubespark/spark-shuffle:v2.2.0-kubernetes-0.4.0 imagePullPolicy: IfNotPresent volumeMounts: - - mountPath: '/tmp' + - mountPath: '/tmp/spark-local' name: temp-volume # more volumes can be mounted here. # The spark job must be configured to use these diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index ae1e9adb7f0ba..7b01cf4c3c3d8 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -224,7 +224,7 @@ Below is an example submission: local:///opt/spark/examples/src/main/python/pi.py 100 ``` -## Dynamic Executor Scaling +## Dynamic Allocation in Kubernetes Spark on Kubernetes supports Dynamic Allocation with cluster mode. This mode requires running an external shuffle service. This is typically a [daemonset](https://kubernetes.io/docs/concepts/workloads/controllers/daemonset/) @@ -247,6 +247,7 @@ the command may then look like the following: --class org.apache.spark.examples.GroupByTest \ --master k8s://: \ --kubernetes-namespace default \ + --conf spark.local.dir=/tmp/spark-local --conf spark.app.name=group-by-test \ --conf spark.kubernetes.driver.docker.image=kubespark/spark-driver:latest \ --conf spark.kubernetes.executor.docker.image=kubespark/spark-executor:latest \ @@ -256,6 +257,14 @@ the command may then look like the following: --conf spark.kubernetes.shuffle.labels="app=spark-shuffle-service,spark-version=2.2.0" \ local:///opt/spark/examples/jars/spark-examples_2.11-2.2.0-k8s-0.3.0.jar 10 400000 2 +The external shuffle service has to mount directories that can be shared with the executor pods. The provided example +YAML spec mounts a hostPath volume to the external shuffle service pods, but these hostPath volumes must also be mounted +into the executors. When using the external shuffle service, the directories specified in the `spark.local.dir` +configuration are mounted as hostPath volumes into all of the executor containers. To ensure that one does not +accidentally mount the incorrect hostPath volumes, the value of `spark.local.dir` must be specified in your +application's configuration when using Kubernetes, even though it defaults to the JVM's temporary directory when using +other cluster managers. + ## Advanced ### Securing the Resource Staging Server with TLS 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 fc08c0ad42f82..0e35e04ff5803 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 @@ -157,12 +157,6 @@ package object config extends Logging { .stringConf .createOptional - private[spark] val KUBERNETES_SHUFFLE_DIR = - ConfigBuilder("spark.kubernetes.shuffle.dir") - .doc("Path to the shared shuffle directories.") - .stringConf - .createOptional - private[spark] val KUBERNETES_SHUFFLE_APISERVER_URI = ConfigBuilder("spark.kubernetes.shuffle.apiServer.url") .doc("URL to the Kubernetes API server that the shuffle service will monitor for Spark pods.") 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 d8aec7d3c5bd7..95d7f284f86da 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 @@ -102,4 +102,5 @@ package object constants { private[spark] val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" private[spark] val MEMORY_OVERHEAD_FACTOR = 0.10 private[spark] val MEMORY_OVERHEAD_MIN_MIB = 384L + private[spark] val GENERATED_LOCAL_DIR_MOUNT_ROOT = "/mnt/tmp/spark-local" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index fd251637ce210..92ec8e5d85260 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -22,6 +22,7 @@ import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator +import org.apache.spark.deploy.k8s.submit.submitsteps.LocalDirectoryMountConfigurationStep import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.{SystemClock, Utils} @@ -104,6 +105,9 @@ private[spark] class DriverConfigurationStepsOrchestrator( val kubernetesCredentialsStep = new DriverKubernetesCredentialsStep( submissionSparkConf, kubernetesResourceNamePrefix) + val localDirectoryMountConfigurationStep = new LocalDirectoryMountConfigurationStep( + submissionSparkConf) + val pythonStep = mainAppResource match { case PythonMainAppResource(mainPyResource) => Option(new PythonStep(mainPyResource, additionalPythonFiles, filesDownloadPath)) @@ -181,7 +185,8 @@ private[spark] class DriverConfigurationStepsOrchestrator( initialSubmissionStep, driverAddressStep, kubernetesCredentialsStep, - dependencyResolutionStep) ++ + dependencyResolutionStep, + localDirectoryMountConfigurationStep) ++ submittedDependenciesBootstrapSteps ++ pythonStep.toSeq ++ mountSecretsStep.toSeq diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala new file mode 100644 index 0000000000000..3f9ba8af74162 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/submitsteps/LocalDirectoryMountConfigurationStep.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit.submitsteps + +import java.nio.file.Paths +import java.util.UUID + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, PodBuilder, VolumeBuilder, VolumeMountBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.constants._ + +/** + * Configures local directories that the driver and executors should use for temporary storage. + * + * Note that we have different semantics for scratch space in Kubernetes versus the other cluster + * managers. In Kubernetes, we cannot allow the local directories to resolve to the Java temporary + * directory. This is because we will mount either emptyDir volumes for both the driver and + * executors, or hostPath volumes for the executors and an emptyDir for the driver. In either + * case, the mount paths need to be directories that do not exist in the base container images. + * But the Java temporary directory is typically a directory like /tmp which exists in most + * container images. + * + * The solution is twofold: + * - When not using an external shuffle service, a reasonable default is to create a new directory + * with a random name and set that to be the value of `spark.local.dir`. + * - When using the external shuffle service, it is risky to assume that the user intends to mount + * the JVM temporary directory into the pod as a hostPath volume. We therefore enforce that + * spark.local.dir must be set in dynamic allocation mode so that the user explicitly sets the + * paths that have to be mounted. + */ +private[spark] class LocalDirectoryMountConfigurationStep( + submissionSparkConf: SparkConf, + randomDirProvider: () => String = () => s"spark-${UUID.randomUUID()}") + extends DriverConfigurationStep { + + override def configureDriver(driverSpec: KubernetesDriverSpec): KubernetesDriverSpec = { + val configuredLocalDirs = submissionSparkConf.getOption("spark.local.dir") + val isUsingExternalShuffle = submissionSparkConf.get( + org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED) + val resolvedLocalDirsSingleString = if (isUsingExternalShuffle) { + require(configuredLocalDirs.isDefined, "spark.local.dir must be provided explicitly when" + + " using the external shuffle service in Kubernetes. These directories should map to" + + " the paths that are mounted into the external shuffle service pods.") + configuredLocalDirs.get + } else { + // If we don't use the external shuffle service, local directories should be randomized if + // not provided. + configuredLocalDirs.getOrElse(s"$GENERATED_LOCAL_DIR_MOUNT_ROOT/${randomDirProvider()}") + } + val resolvedLocalDirs = resolvedLocalDirsSingleString.split(",") + // It's worth noting that we always use an emptyDir volume for the directories on the driver, + // because the driver does not need a hostPath to share its scratch space with any other pod. + // The driver itself will decide on whether to use a hostPath volume or an emptyDir volume for + // these directories on the executors. (see ExecutorPodFactory and + // KubernetesExternalClusterManager) + val localDirVolumes = resolvedLocalDirs.zipWithIndex.map { case (dir, index) => + new VolumeBuilder() + .withName(s"spark-local-dir-$index-${Paths.get(dir).getFileName.toString}") + .withNewEmptyDir().endEmptyDir() + .build() + } + val localDirVolumeMounts = localDirVolumes.zip(resolvedLocalDirs).map { + case (volume, path) => + new VolumeMountBuilder() + .withName(volume.getName) + .withMountPath(path) + .build() + } + val resolvedDriverSparkConf = driverSpec.driverSparkConf.clone().set( + "spark.local.dir", resolvedLocalDirsSingleString) + driverSpec.copy( + driverPod = new PodBuilder(driverSpec.driverPod) + .editSpec() + .addToVolumes(localDirVolumes: _*) + .endSpec() + .build(), + driverContainer = new ContainerBuilder(driverSpec.driverContainer) + .addToVolumeMounts(localDirVolumeMounts: _*) + .build(), + driverSparkConf = resolvedDriverSparkConf + ) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala new file mode 100644 index 0000000000000..2b35fd6a513f5 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProvider.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.k8s + +import java.nio.file.Paths + +import io.fabric8.kubernetes.api.model.{Volume, VolumeBuilder, VolumeMount, VolumeMountBuilder} + +import org.apache.spark.SparkConf +import org.apache.spark.util.Utils + +private[spark] trait ExecutorLocalDirVolumeProvider { + def getExecutorLocalDirVolumesWithMounts: Seq[(Volume, VolumeMount)] +} + +private[spark] class ExecutorLocalDirVolumeProviderImpl( + sparkConf: SparkConf, + kubernetesExternalShuffleManager: Option[KubernetesExternalShuffleManager]) + extends ExecutorLocalDirVolumeProvider { + override def getExecutorLocalDirVolumesWithMounts: Seq[(Volume, VolumeMount)] = { + kubernetesExternalShuffleManager.map(_.getExecutorShuffleDirVolumesWithMounts) + .getOrElse { + // If we're not using the external shuffle manager, we should use emptyDir volumes for + // shuffle directories since it's important for disk I/O for these directories to be + // performant. If the user has not provided a local directory, instead of using the + // Java temporary directory, we create one instead, because we want to avoid + // mounting an emptyDir which overlaps with an existing path in the Docker image. + // Java's temporary directory path is typically /tmp or a similar path, which is + // likely to exist in most images. + val resolvedLocalDirs = Utils.getConfiguredLocalDirs(sparkConf) + val localDirVolumes = resolvedLocalDirs.zipWithIndex.map { case (dir, index) => + new VolumeBuilder() + .withName(s"spark-local-dir-$index-${Paths.get(dir).getFileName.toString}") + .withNewEmptyDir().endEmptyDir() + .build() + } + val localDirVolumeMounts = localDirVolumes.zip(resolvedLocalDirs).map { + case (volume, path) => + new VolumeMountBuilder() + .withName(volume.getName) + .withMountPath(path) + .build() + } + localDirVolumes.zip(localDirVolumeMounts) + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala index 6f4ba1c8b888f..98a0d879b6a58 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactory.scala @@ -16,10 +16,9 @@ */ package org.apache.spark.scheduler.cluster.k8s +import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder, VolumeBuilder, VolumeMountBuilder} import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerBuilder, ContainerPortBuilder, EnvVar, EnvVarBuilder, EnvVarSourceBuilder, Pod, PodBuilder, QuantityBuilder} - import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.k8s.{ConfigurationUtils, InitContainerResourceStagingServerSecretPlugin, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.k8s.config._ @@ -46,7 +45,7 @@ private[spark] class ExecutorPodFactoryImpl( mountSmallFilesBootstrap: Option[MountSmallFilesBootstrap], executorInitContainerBootstrap: Option[SparkPodInitContainerBootstrap], executorMountInitContainerSecretPlugin: Option[InitContainerResourceStagingServerSecretPlugin], - shuffleManager: Option[KubernetesExternalShuffleManager]) + executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider) extends ExecutorPodFactory { import ExecutorPodFactoryImpl._ @@ -175,9 +174,8 @@ private[spark] class ExecutorPodFactoryImpl( .withContainerPort(port._2) .build() }) - val shuffleVolumesWithMounts = - shuffleManager.map(_.getExecutorShuffleDirVolumesWithMounts) - .getOrElse(Seq.empty) + val shuffleVolumesWithMounts = executorLocalDirVolumeProvider + .getExecutorLocalDirVolumesWithMounts val executorContainer = new ContainerBuilder() .withName(s"executor") @@ -262,6 +260,7 @@ private[spark] class ExecutorPodFactoryImpl( val executorPodWithNodeAffinity = nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( executorPodWithInitContainer, nodeToLocalTaskCount) + new PodBuilder(executorPodWithNodeAffinity) .editSpec() .addToContainers(initBootstrappedExecutorContainer) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 8385cb61dc986..6ae76c4a7bda3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -113,18 +113,21 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_TOKEN_PATH)), Some(new File(Config.KUBERNETES_SERVICE_ACCOUNT_CA_CRT_PATH))) - val kubernetesShuffleManager = if (Utils.isDynamicAllocationEnabled(sparkConf)) { + val kubernetesShuffleManager = if (sparkConf.get( + org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED)) { val kubernetesExternalShuffleClient = new KubernetesExternalShuffleClientImpl( - SparkTransportConf.fromSparkConf(sparkConf, "shuffle"), - sc.env.securityManager, - sc.env.securityManager.isAuthenticationEnabled(), - sparkConf.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) + SparkTransportConf.fromSparkConf(sparkConf, "shuffle"), + sc.env.securityManager, + sc.env.securityManager.isAuthenticationEnabled(), + sparkConf.get(org.apache.spark.internal.config.SHUFFLE_REGISTRATION_TIMEOUT)) Some(new KubernetesExternalShuffleManagerImpl( - sparkConf, - kubernetesClient, - kubernetesExternalShuffleClient)) + sparkConf, + kubernetesClient, + kubernetesExternalShuffleClient)) } else None + val executorLocalDirVolumeProvider = new ExecutorLocalDirVolumeProviderImpl( + sparkConf, kubernetesShuffleManager) val executorPodFactory = new ExecutorPodFactoryImpl( sparkConf, NodeAffinityExecutorPodModifierImpl, @@ -132,7 +135,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit mountSmallFilesBootstrap, executorInitContainerBootstrap, executorInitContainerSecretVolumePlugin, - kubernetesShuffleManager) + executorLocalDirVolumeProvider) val allocatorExecutor = ThreadUtils .newDaemonSingleThreadScheduledExecutor("kubernetes-pod-allocator") val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala index 181f22dfe3dcc..388e2b17f4fdd 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExternalShuffleManager.scala @@ -67,9 +67,7 @@ private[spark] class KubernetesExternalShuffleManagerImpl( s"but no ${KUBERNETES_SHUFFLE_LABELS.key} specified") } private val externalShufflePort = sparkConf.getInt("spark.shuffle.service.port", 7337) - private val shuffleDirs = sparkConf.get(KUBERNETES_SHUFFLE_DIR).map { - _.split(",") - }.getOrElse(Utils.getConfiguredLocalDirs(sparkConf)) + private val shuffleDirs = Utils.getConfiguredLocalDirs(sparkConf) private var shufflePodCache = scala.collection.mutable.Map[String, String]() private var watcher: Watch = _ @@ -140,6 +138,12 @@ private[spark] class KubernetesExternalShuffleManagerImpl( } override def getExecutorShuffleDirVolumesWithMounts(): Seq[(Volume, VolumeMount)] = { + // TODO: Using hostPath for the local directory will also make it such that the + // other uses of the local directory - broadcasting and caching - will also write + // to the directory that the shuffle service is aware of. It would be better for + // these directories to be separate so that the lifetime of the non-shuffle scratch + // space is tied to an emptyDir instead of the hostPath. This requires a change in + // core Spark as well. shuffleDirs.zipWithIndex.map { case (shuffleDir, shuffleDirIndex) => val volumeName = s"$shuffleDirIndex-${FilenameUtils.getBaseName(shuffleDir)}" diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala index 6f5d5e571c443..1199f033cf06a 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestratorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.submit import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s.config._ -import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} +import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, LocalDirectoryMountConfigurationStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunSuite { @@ -52,7 +52,8 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[BaseDriverConfigurationStep], classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], - classOf[DependencyResolutionStep]) + classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep]) } test("Submission steps with an init-container.") { @@ -76,6 +77,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[InitContainerBootstrapStep]) } @@ -98,6 +100,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[PythonStep]) } @@ -120,6 +123,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[MountSmallLocalFilesStep]) } @@ -144,6 +148,7 @@ private[spark] class DriverConfigurationStepsOrchestratorSuite extends SparkFunS classOf[DriverAddressConfigurationStep], classOf[DriverKubernetesCredentialsStep], classOf[DependencyResolutionStep], + classOf[LocalDirectoryMountConfigurationStep], classOf[MountSecretsStep]) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala new file mode 100644 index 0000000000000..5ce199a5df857 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/kubernetes/submit/submitsteps/LocalDirectoryMountConfigurationStepSuite.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.kubernetes.submit.submitsteps + +import java.nio.file.Paths + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.constants._ +import org.apache.spark.deploy.k8s.submit.submitsteps.{KubernetesDriverSpec, LocalDirectoryMountConfigurationStep} + +private[spark] class LocalDirectoryMountConfigurationStepSuite extends SparkFunSuite { + + test("When using the external shuffle service, the local directories must be provided.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, true) + val configurationStep = new LocalDirectoryMountConfigurationStep(sparkConf) + try { + configurationStep.configureDriver(KubernetesDriverSpec.initialSpec(sparkConf)) + fail("The configuration step should have failed without local dirs.") + } catch { + case e: Throwable => + assert(e.getMessage === "requirement failed: spark.local.dir must be provided explicitly" + + " when using the external shuffle service in Kubernetes. These directories should map" + + " to the paths that are mounted into the external shuffle service pods.") + } + } + + test("When not using the external shuffle service, a random directory should be set" + + " for local dirs if one is not provided.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, false) + val configurationStep = new LocalDirectoryMountConfigurationStep( + sparkConf, () => "local-dir") + val resolvedDriverSpec = configurationStep.configureDriver( + KubernetesDriverSpec.initialSpec(sparkConf)) + testLocalDirsMatch(resolvedDriverSpec, Seq(s"$GENERATED_LOCAL_DIR_MOUNT_ROOT/local-dir")) + } + + test("When not using the external shuffle service, provided local dirs should be mounted as" + + " emptyDirs.") { + val sparkConf = new SparkConf(false) + .set(org.apache.spark.internal.config.SHUFFLE_SERVICE_ENABLED, false) + .set("spark.local.dir", "/mnt/tmp/spark-local,/var/tmp/spark-local") + val configurationStep = new LocalDirectoryMountConfigurationStep( + sparkConf) + val resolvedDriverSpec = configurationStep.configureDriver( + KubernetesDriverSpec.initialSpec(sparkConf)) + testLocalDirsMatch(resolvedDriverSpec, Seq("/mnt/tmp/spark-local", "/var/tmp/spark-local")) + } + + private def testLocalDirsMatch( + resolvedDriverSpec: KubernetesDriverSpec, expectedLocalDirs: Seq[String]): Unit = { + assert(resolvedDriverSpec.driverSparkConf.get("spark.local.dir").split(",") === + expectedLocalDirs) + expectedLocalDirs + .zip(resolvedDriverSpec.driverPod.getSpec.getVolumes.asScala) + .zipWithIndex + .foreach { + case ((dir, volume), index) => + assert(volume.getEmptyDir != null) + val fileName = Paths.get(dir).getFileName.toString + assert(volume.getName === s"spark-local-dir-$index-$fileName") + } + + expectedLocalDirs + .zip(resolvedDriverSpec.driverContainer.getVolumeMounts.asScala) + .zipWithIndex + .foreach { + case ((dir, volumeMount), index) => + val fileName = Paths.get(dir).getFileName.toString + assert(volumeMount.getName === s"spark-local-dir-$index-$fileName") + assert(volumeMount.getMountPath === dir) + } + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala new file mode 100644 index 0000000000000..f3baf5b9f739a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLocalDirVolumeProviderSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.scheduler.cluster.k8s + +import io.fabric8.kubernetes.api.model.{VolumeBuilder, VolumeMountBuilder} +import org.mockito.Mockito.{verify, verifyNoMoreInteractions, when} +import org.scalatest.mock.MockitoSugar.mock + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class ExecutorLocalDirVolumeProviderSuite extends SparkFunSuite { + + test("Delegates to the external shuffle manager implementation if present.") { + val externalShuffleManager = mock[KubernetesExternalShuffleManager] + val mockVolume = new VolumeBuilder() + .withName("local-dir") + .withNewHostPath("/tmp/spark-local-dirs") + .build() + val mockVolumeMount = new VolumeMountBuilder() + .withName("local-dir") + .withMountPath("/tmp/spark-local-dirs-mount") + .build() + when(externalShuffleManager.getExecutorShuffleDirVolumesWithMounts) + .thenReturn(Seq((mockVolume, mockVolumeMount))) + val volumeProvider = new ExecutorLocalDirVolumeProviderImpl( + new SparkConf(false), Some(externalShuffleManager)) + assert(volumeProvider.getExecutorLocalDirVolumesWithMounts === + Seq((mockVolume, mockVolumeMount))) + verify(externalShuffleManager).getExecutorShuffleDirVolumesWithMounts + verifyNoMoreInteractions(externalShuffleManager) + } + + test("Provides EmptyDir volumes for each local dir in spark.local.dirs.") { + val localDirs = Seq("/tmp/test-local-dir-1", "/tmp/test-local-dir-2") + val sparkConf = new SparkConf(false).set("spark.local.dir", localDirs.mkString(",")) + val volumeProvider = new ExecutorLocalDirVolumeProviderImpl(sparkConf, None) + val localDirVolumesWithMounts = volumeProvider.getExecutorLocalDirVolumesWithMounts + assert(localDirVolumesWithMounts.size === 2) + localDirVolumesWithMounts.zip(localDirs).zipWithIndex.foreach { + case (((localDirVolume, localDirVolumeMount), expectedDirMountPath), index) => + val dirName = expectedDirMountPath.substring( + expectedDirMountPath.lastIndexOf('/') + 1, expectedDirMountPath.length) + assert(localDirVolume.getName === s"spark-local-dir-$index-$dirName") + assert(localDirVolume.getEmptyDir != null) + assert(localDirVolumeMount.getName === localDirVolume.getName) + assert(localDirVolumeMount.getMountPath === expectedDirMountPath) + case unknown => throw new IllegalArgumentException("Unexpected object: $unknown") + } + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala index 6690217557637..bb09cb801b5a9 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodFactorySuite.scala @@ -16,21 +16,18 @@ */ package org.apache.spark.scheduler.cluster.k8s -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{Pod, VolumeBuilder, VolumeMountBuilder, _} +import io.fabric8.kubernetes.api.model.{Pod, PodBuilder, VolumeBuilder, VolumeMountBuilder} import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.commons.io.FilenameUtils -import org.mockito.{AdditionalAnswers, MockitoAnnotations} -import org.mockito.Matchers.{any, eq => mockitoEq} +import org.mockito.{AdditionalAnswers, Mock, Mockito, MockitoAnnotations} +import org.mockito.Matchers.any import org.mockito.Mockito._ -import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} +import scala.collection.JavaConverters._ import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{constants, PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} +import org.apache.spark.deploy.k8s.{PodWithDetachedInitContainer, SparkPodInitContainerBootstrap} import org.apache.spark.deploy.k8s.config._ +import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.{MountSecretsBootstrapImpl, MountSmallFilesBootstrap, MountSmallFilesBootstrapImpl} class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { @@ -52,7 +49,12 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef .endStatus() .build() private var baseConf: SparkConf = _ - private val nodeAffinityExecutorPodModifier = mock(classOf[NodeAffinityExecutorPodModifier]) + + @Mock + private var nodeAffinityExecutorPodModifier: NodeAffinityExecutorPodModifier = _ + + @Mock + private var executorLocalDirVolumeProvider: ExecutorLocalDirVolumeProvider = _ before { MockitoAnnotations.initMocks(this) @@ -60,15 +62,12 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef .set(KUBERNETES_DRIVER_POD_NAME, driverPodName) .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, executorPrefix) .set(EXECUTOR_DOCKER_IMAGE, executorImage) - } - private var kubernetesClient: KubernetesClient = _ - - override def beforeEach(cmap: org.scalatest.ConfigMap) { - reset(nodeAffinityExecutorPodModifier) when(nodeAffinityExecutorPodModifier.addNodeAffinityAnnotationIfUseful( any(classOf[Pod]), any(classOf[Map[String, Int]]))).thenAnswer(AdditionalAnswers.returnsFirstArg()) + when(executorLocalDirVolumeProvider.getExecutorLocalDirVolumesWithMounts).thenReturn(Seq.empty) } + private var kubernetesClient: KubernetesClient = _ test("basic executor pod has reasonable defaults") { val factory = new ExecutorPodFactoryImpl( @@ -78,7 +77,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - None) + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -112,7 +111,13 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple") val factory = new ExecutorPodFactoryImpl( - conf, nodeAffinityExecutorPodModifier, None, None, None, None, None) + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -133,7 +138,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, None, None, - None) + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -167,7 +172,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef None, Some(initContainerBootstrap), None, - None) + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -175,53 +180,37 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) assert(executor.getMetadata.getAnnotations.size() === 1) - assert(executor.getMetadata.getAnnotations.containsKey(constants.INIT_CONTAINER_ANNOTATION)) + assert(executor.getMetadata.getAnnotations.containsKey(INIT_CONTAINER_ANNOTATION)) checkOwnerReferences(executor, driverPodUid) } - test("the shuffle-service adds a volume mount") { - val conf = baseConf.clone() - conf.set(KUBERNETES_SHUFFLE_LABELS, "label=value") - conf.set(KUBERNETES_SHUFFLE_NAMESPACE, "default") - conf.set(KUBERNETES_SHUFFLE_DIR, "/tmp") - - val shuffleManager = mock(classOf[KubernetesExternalShuffleManager]) - when(shuffleManager.getExecutorShuffleDirVolumesWithMounts).thenReturn({ - val shuffleDirs = Seq("/tmp") - shuffleDirs.zipWithIndex.map { case (shuffleDir, shuffleDirIndex) => - val volumeName = s"$shuffleDirIndex-${FilenameUtils.getBaseName(shuffleDir)}" - val volume = new VolumeBuilder() - .withName(volumeName) - .withNewHostPath(shuffleDir) - .build() - val volumeMount = new VolumeMountBuilder() - .withName(volumeName) - .withMountPath(shuffleDir) - .build() - (volume, volumeMount) - } - }) - + test("The local dir volume provider's returned volumes and volume mounts should be added.") { + Mockito.reset(executorLocalDirVolumeProvider) + val localDirVolume = new VolumeBuilder() + .withName("local-dir") + .withNewEmptyDir().endEmptyDir() + .build() + val localDirVolumeMount = new VolumeMountBuilder() + .withName("local-dir") + .withMountPath("/tmp") + .build() + when(executorLocalDirVolumeProvider.getExecutorLocalDirVolumesWithMounts) + .thenReturn(Seq((localDirVolume, localDirVolumeMount))) val factory = new ExecutorPodFactoryImpl( - conf, + baseConf, nodeAffinityExecutorPodModifier, None, None, None, None, - Some(shuffleManager)) + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) - - verify(nodeAffinityExecutorPodModifier, times(1)) - .addNodeAffinityAnnotationIfUseful(any(classOf[Pod]), any(classOf[Map[String, Int]])) - + assert(executor.getSpec.getVolumes.size === 1) + assert(executor.getSpec.getVolumes.contains(localDirVolume)) assert(executor.getSpec.getContainers.size() === 1) - assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size() === 1) - assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0).getName === "0-tmp") - assert(executor.getSpec.getContainers.get(0).getVolumeMounts.get(0) - .getMountPath === "/tmp") - checkOwnerReferences(executor, driverPodUid) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.size === 1) + assert(executor.getSpec.getContainers.get(0).getVolumeMounts.contains(localDirVolumeMount)) } test("Small-files add a secret & secret volume mount to the container") { @@ -235,7 +224,7 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef Some(smallFiles), None, None, - None) + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)](), driverPod, Map[String, Int]()) @@ -262,7 +251,13 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") val factory = new ExecutorPodFactoryImpl( - conf, nodeAffinityExecutorPodModifier, None, None, None, None, None) + conf, + nodeAffinityExecutorPodModifier, + None, + None, + None, + None, + executorLocalDirVolumeProvider) val executor = factory.createExecutorPod( "1", "dummy", "dummy", Seq[(String, String)]("qux" -> "quux"), driverPod, Map[String, Int]()) @@ -286,14 +281,14 @@ class ExecutorPodFactorySuite extends SparkFunSuite with BeforeAndAfter with Bef // Check that the expected environment variables are present. private def checkEnv(executor: Pod, additionalEnvVars: Map[String, String]): Unit = { val defaultEnvs = Map( - constants.ENV_EXECUTOR_ID -> "1", - constants.ENV_DRIVER_URL -> "dummy", - constants.ENV_EXECUTOR_CORES -> "1", - constants.ENV_EXECUTOR_MEMORY -> "1g", - constants.ENV_APPLICATION_ID -> "dummy", - constants.ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", - constants.ENV_EXECUTOR_POD_IP -> null, - constants.ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars + ENV_EXECUTOR_ID -> "1", + ENV_DRIVER_URL -> "dummy", + ENV_EXECUTOR_CORES -> "1", + ENV_EXECUTOR_MEMORY -> "1g", + ENV_APPLICATION_ID -> "dummy", + ENV_MOUNTED_CLASSPATH -> "/var/spark-data/spark-jars/*", + ENV_EXECUTOR_POD_IP -> null, + ENV_EXECUTOR_PORT -> "10000") ++ additionalEnvVars assert(executor.getSpec.getContainers.size() === 1) assert(executor.getSpec.getContainers.get(0).getEnv().size() === defaultEnvs.size) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index d6e12df9d1ba2..27041207ffdce 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -150,6 +150,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfter { sparkConf.setJars(Seq(CONTAINER_LOCAL_HELPER_JAR_PATH)) sparkConf.set("spark.dynamicAllocation.enabled", "true") + sparkConf.set("spark.local.dir", "/tmp") sparkConf.set("spark.shuffle.service.enabled", "true") sparkConf.set("spark.kubernetes.shuffle.labels", "app=spark-shuffle-service") sparkConf.set("spark.kubernetes.shuffle.namespace", kubernetesTestComponents.namespace) From 76a554411bb5111de9ea023c2258e5d0a3cbe848 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 16 Oct 2017 16:26:47 -0700 Subject: [PATCH 25/25] Reorder imports --- .../k8s/submit/DriverConfigurationStepsOrchestrator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala index 92ec8e5d85260..3203cebba71e3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/DriverConfigurationStepsOrchestrator.scala @@ -21,8 +21,8 @@ import org.apache.spark.deploy.k8s.ConfigurationUtils import org.apache.spark.deploy.k8s.config._ import org.apache.spark.deploy.k8s.constants._ import org.apache.spark.deploy.k8s.submit.submitsteps.{BaseDriverConfigurationStep, DependencyResolutionStep, DriverAddressConfigurationStep, DriverConfigurationStep, DriverKubernetesCredentialsStep, InitContainerBootstrapStep, MountSecretsStep, MountSmallLocalFilesStep, PythonStep} -import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.deploy.k8s.submit.submitsteps.LocalDirectoryMountConfigurationStep +import org.apache.spark.deploy.k8s.submit.submitsteps.initcontainer.InitContainerConfigurationStepsOrchestrator import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util.{SystemClock, Utils}