Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
7570eab
[SPARK-22788][STREAMING] Use correct hadoop config for fs append supp…
Dec 20, 2017
7798c9e
[SPARK-22824] Restore old offset for binary compatibility
jose-torres Dec 20, 2017
d762d11
[SPARK-22832][ML] BisectingKMeans unpersist unused datasets
zhengruifeng Dec 20, 2017
c89b431
[SPARK-22849] ivy.retrieve pattern should also consider `classifier`
gatorsmile Dec 20, 2017
792915c
[SPARK-22830] Scala Coding style has been improved in Spark Examples
chetkhatri Dec 20, 2017
b176014
[SPARK-22847][CORE] Remove redundant code in AppStatusListener while …
Ngone51 Dec 20, 2017
0114c89
[SPARK-22845][SCHEDULER] Modify spark.kubernetes.allocation.batch.del…
foxish Dec 21, 2017
fb0562f
[SPARK-22810][ML][PYSPARK] Expose Python API for LinearRegression wit…
yanboliang Dec 21, 2017
9c289a5
[SPARK-22387][SQL] Propagate session configs to data source read/writ…
jiangxb1987 Dec 21, 2017
d3ae3e1
[SPARK-19634][SQL][ML][FOLLOW-UP] Improve interface of dataframe vect…
WeichenXu123 Dec 21, 2017
cb9fc8d
[SPARK-22848][SQL] Eliminate mutable state from Stack
kiszk Dec 21, 2017
59d5263
[SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0
BryanCutler Dec 21, 2017
0abaf31
[SPARK-22852][BUILD] Exclude -Xlint:unchecked from sbt javadoc flags
easel Dec 21, 2017
4c2efde
[SPARK-22855][BUILD] Add -no-java-comments to sbt docs/scalacOptions
easel Dec 21, 2017
8a0ed5a
[SPARK-22668][SQL] Ensure no global variables in arguments of method …
cloud-fan Dec 21, 2017
d3a1d95
[SPARK-22786][SQL] only use AppStatusPlugin in history server
cloud-fan Dec 21, 2017
4e107fd
[SPARK-22822][TEST] Basic tests for WindowFrameCoercion and DecimalPr…
wangyum Dec 21, 2017
fe65361
[SPARK-22042][FOLLOW-UP][SQL] ReorderJoinPredicates can break when ch…
tejasapatil Dec 21, 2017
7beb375
[SPARK-22861][SQL] SQLAppStatusListener handles multi-job executions.
squito Dec 21, 2017
7ab165b
[SPARK-22648][K8S] Spark on Kubernetes - Documentation
foxish Dec 22, 2017
c0abb1d
[SPARK-22854][UI] Read Spark version from event logs.
Dec 22, 2017
c6f01ca
[SPARK-22750][SQL] Reuse mutable states when possible
mgaido91 Dec 22, 2017
a36b78b
[SPARK-22450][CORE][MLLIB][FOLLOWUP] safely register class for mllib …
zhengruifeng Dec 22, 2017
22e1849
[SPARK-22866][K8S] Fix path issue in Kubernetes dockerfile
foxish Dec 22, 2017
8df1da3
[SPARK-22862] Docs on lazy elimination of columns missing from an enc…
marmbrus Dec 22, 2017
13190a4
[SPARK-22874][PYSPARK][SQL] Modify checking pandas version to use Loo…
ueshin Dec 22, 2017
d23dc5b
[SPARK-22346][ML] VectorSizeHint Transformer for using VectorAssemble…
MrBago Dec 22, 2017
d3cbbdd
[SPARK-22757][Kubernetes] Enable use of remote dependencies in Kubern…
liyinan926 Dec 12, 2017
5d2cbc8
Addressed first round of comments
liyinan926 Dec 15, 2017
4ee76af
Addressed the second round of comments
liyinan926 Dec 16, 2017
9c8051a
Create one task per jar/file to download in the init-container
liyinan926 Dec 16, 2017
1f65417
More review comments
liyinan926 Dec 18, 2017
109ad80
Shorten variable names
liyinan926 Dec 19, 2017
c21fdcf
Removed traits that have only a single implementation
liyinan926 Dec 19, 2017
a3cd71d
Remove unused class arguments
liyinan926 Dec 19, 2017
23c5cd9
Improved documentation
liyinan926 Dec 19, 2017
2ec15c4
Addressed latest round of comments
liyinan926 Dec 20, 2017
5d1f889
Addressed more comments
liyinan926 Dec 21, 2017
9d9c841
Updated names of two configuration properties
liyinan926 Dec 22, 2017
c51bc56
Addressed more comments
liyinan926 Dec 25, 2017
28343fb
Addressed one more comment
liyinan926 Dec 26, 2017
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
[SPARK-22757][Kubernetes] Enable use of remote dependencies in Kubern…
…etes mode
  • Loading branch information
liyinan926 committed Dec 23, 2017
commit d3cbbdd8ce45592caa706d0b4d29c873a176e40c
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import java.util.concurrent.TimeUnit

import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.ConfigBuilder
import org.apache.spark.network.util.ByteUnit

private[spark] object Config extends Logging {

Expand Down Expand Up @@ -132,30 +131,78 @@ private[spark] object Config extends Logging {

val JARS_DOWNLOAD_LOCATION =
ConfigBuilder("spark.kubernetes.mountDependencies.jarsDownloadDir")
.doc("Location to download jars to in the driver and executors. When using" +
" spark-submit, this directory must be empty and will be mounted as an empty directory" +
" volume on the driver and executor pod.")
.doc("Location to download jars to in the driver and executors. When using " +
"spark-submit, this directory must be empty and will be mounted as an empty directory " +
"volume on the driver and executor pod.")
.stringConf
.createWithDefault("/var/spark-data/spark-jars")

val FILES_DOWNLOAD_LOCATION =
ConfigBuilder("spark.kubernetes.mountDependencies.filesDownloadDir")
.doc("Location to download files to in the driver and executors. When using" +
" spark-submit, this directory must be empty and will be mounted as an empty directory" +
" volume on the driver and executor pods.")
.doc("Location to download files to in the driver and executors. When using " +
"spark-submit, this directory must be empty and will be mounted as an empty directory " +
"volume on the driver and executor pods.")
.stringConf
.createWithDefault("/var/spark-data/spark-files")

val INIT_CONTAINER_DOCKER_IMAGE =
ConfigBuilder("spark.kubernetes.initContainer.docker.image")
.doc("Image for the driver and executor's init-container that downloads dependencies.")
.stringConf
.createOptional

val INIT_CONTAINER_MOUNT_TIMEOUT =
ConfigBuilder("spark.kubernetes.mountDependencies.mountTimeout")
.doc("Timeout before aborting the attempt to download and unpack local dependencies from " +
"remote locations and the resource staging server when initializing the driver and " +
"executor pods.")
.timeConf(TimeUnit.MINUTES)
.createWithDefault(5)

val INIT_CONTAINER_REMOTE_JARS =
ConfigBuilder("spark.kubernetes.initContainer.remoteJars")
.doc("Comma-separated list of jar URIs to download in the init-container. This is " +
"calculated from spark.jars.")
.internal()
.stringConf
.createOptional

val INIT_CONTAINER_REMOTE_FILES =
ConfigBuilder("spark.kubernetes.initContainer.remoteFiles")
.doc("Comma-separated list of file URIs to download in the init-container. This is " +
"calculated from spark.files.")
.internal()
.stringConf
.createOptional

val INIT_CONTAINER_CONFIG_MAP_NAME =
ConfigBuilder("spark.kubernetes.initContainer.configMapName")
.doc("Name of the config map to use in the init-container that retrieves submitted files " +
"for the executor.")
.internal()
.stringConf
.createOptional

val INIT_CONTAINER_CONFIG_MAP_KEY_CONF =
ConfigBuilder("spark.kubernetes.initContainer.configMapKey")
.doc("Key for the entry in the init container config map for submitted files that " +
"corresponds to the properties for this init-container.")
.internal()
.stringConf
.createOptional

val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX =
"spark.kubernetes.authenticate.submission"

val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector."

val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label."
val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation."
val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets."

val KUBERNETES_EXECUTOR_LABEL_PREFIX = "spark.kubernetes.executor.label."
val KUBERNETES_EXECUTOR_ANNOTATION_PREFIX = "spark.kubernetes.executor.annotation."
val KUBERNETES_EXECUTOR_SECRETS_PREFIX = "spark.kubernetes.executor.secrets."

val KUBERNETES_DRIVER_ENV_KEY = "spark.kubernetes.driverEnv."
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,17 @@ private[spark] object Constants {
val ENV_DRIVER_JAVA_OPTS = "SPARK_DRIVER_JAVA_OPTS"
val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS"
val ENV_DRIVER_MEMORY = "SPARK_DRIVER_MEMORY"
val ENV_MOUNTED_FILES_DIR = "SPARK_MOUNTED_FILES_DIR"

// Bootstrapping dependencies with the init-container
val INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME = "download-jars-volume"
val INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME = "download-files-volume"
val INIT_CONTAINER_PROPERTIES_FILE_VOLUME = "spark-init-properties"
val INIT_CONTAINER_PROPERTIES_FILE_DIR = "/etc/spark-init"
val INIT_CONTAINER_PROPERTIES_FILE_NAME = "spark-init.properties"
val INIT_CONTAINER_PROPERTIES_FILE_PATH =
s"$INIT_CONTAINER_PROPERTIES_FILE_DIR/$INIT_CONTAINER_PROPERTIES_FILE_NAME"
val INIT_CONTAINER_SECRET_VOLUME_NAME = "spark-init-secret"

// Miscellaneous
val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc"
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,128 @@
/*
* 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

import scala.collection.JavaConverters._

import io.fabric8.kubernetes.api.model.{ContainerBuilder, EmptyDirVolumeSource, EnvVarBuilder, PodBuilder, VolumeMount, VolumeMountBuilder}

import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.deploy.k8s.Config._
import org.apache.spark.deploy.k8s.Constants._

/**
* This is separated out from the init-container steps API because this component can be reused to
* set up the init-container for executors as well.
*/
private[spark] trait InitContainerBootstrap {
/**
* Bootstraps an init-container that downloads dependencies to be used by a main container.
*/
def bootstrapInitContainer(
originalPodWithInitContainer: PodWithDetachedInitContainer)
: PodWithDetachedInitContainer
}

private[spark] class InitContainerBootstrapImpl(
initContainerImage: String,
dockerImagePullPolicy: String,
jarsDownloadPath: String,
filesDownloadPath: String,
downloadTimeoutMinutes: Long,
initContainerConfigMapName: String,
initContainerConfigMapKey: String,
sparkRole: String,
sparkConf: SparkConf)
extends InitContainerBootstrap {

override def bootstrapInitContainer(
podWithDetachedInitContainer: PodWithDetachedInitContainer): PodWithDetachedInitContainer = {
val sharedVolumeMounts = Seq[VolumeMount](
new VolumeMountBuilder()
.withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
.withMountPath(jarsDownloadPath)
.build(),
new VolumeMountBuilder()
.withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
.withMountPath(filesDownloadPath)
.build())

val initContainerCustomEnvVarKeyPrefix = sparkRole match {
case SPARK_POD_DRIVER_ROLE => KUBERNETES_DRIVER_ENV_KEY
case SPARK_POD_EXECUTOR_ROLE => "spark.executorEnv."
case _ => throw new SparkException(s"$sparkRole is not a valid Spark pod role")
}
val initContainerCustomEnvVars = sparkConf.getAllWithPrefix(initContainerCustomEnvVarKeyPrefix)
.toSeq
.map(env =>
new EnvVarBuilder()
.withName(env._1)
.withValue(env._2)
.build())

val initContainer = new ContainerBuilder(podWithDetachedInitContainer.initContainer)
.withName(s"spark-init")
.withImage(initContainerImage)
.withImagePullPolicy(dockerImagePullPolicy)
.addAllToEnv(initContainerCustomEnvVars.asJava)
.addNewVolumeMount()
.withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
.withMountPath(INIT_CONTAINER_PROPERTIES_FILE_DIR)
.endVolumeMount()
.addToVolumeMounts(sharedVolumeMounts: _*)
.addToArgs(INIT_CONTAINER_PROPERTIES_FILE_PATH)
.build()

val podWithBasicVolumes = new PodBuilder(podWithDetachedInitContainer.pod)
.editSpec()
.addNewVolume()
.withName(INIT_CONTAINER_PROPERTIES_FILE_VOLUME)
.withNewConfigMap()
.withName(initContainerConfigMapName)
.addNewItem()
.withKey(initContainerConfigMapKey)
.withPath(INIT_CONTAINER_PROPERTIES_FILE_NAME)
.endItem()
.endConfigMap()
.endVolume()
.addNewVolume()
.withName(INIT_CONTAINER_DOWNLOAD_JARS_VOLUME_NAME)
.withEmptyDir(new EmptyDirVolumeSource())
.endVolume()
.addNewVolume()
.withName(INIT_CONTAINER_DOWNLOAD_FILES_VOLUME_NAME)
.withEmptyDir(new EmptyDirVolumeSource())
.endVolume()
.endSpec()
.build()

val mainContainerWithMountedFiles = new ContainerBuilder(
podWithDetachedInitContainer.mainContainer)
.addToVolumeMounts(sharedVolumeMounts: _*)
.addNewEnv()
.withName(ENV_MOUNTED_FILES_DIR)
.withValue(filesDownloadPath)
.endEnv()
.build()

PodWithDetachedInitContainer(
podWithBasicVolumes,
initContainer,
mainContainerWithMountedFiles)
}

}
Original file line number Diff line number Diff line change
@@ -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.k8s

import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder}

/**
* Bootstraps a driver or executor container or an init-container 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"
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.deploy.k8s

import io.fabric8.kubernetes.api.model.{Container, Pod}

/**
* Represents a pod with a detached init-container (not yet added to the pod).
*
* @param pod the pod
* @param initContainer the init-container in the pod
* @param mainContainer the main container in the pod
*/
private[spark] case class PodWithDetachedInitContainer(
pod: Pod,
initContainer: Container,
mainContainer: Container)
Loading