From 7f86e231922648a9da07ae73026a3a430d2c83f5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 19 Oct 2018 13:33:47 -0700 Subject: [PATCH 01/10] [SPARK-25815][k8s] Support kerbers in client mode, keytab-based token renewal. This change hooks up the k8s backed to the updated HadoopDelegationTokenManager, so that delegation tokens are also available in client mode. It also closes the remaining gap in kerberos support which is to support the current keytab-based token renewal supported by YARN (and partially by Mesos). That is done by stashing the keytab in a secret on the driver pod, and then providing it to the HadoopDelegationTokenManager when the driver starts. Tested in cluster mode with and without a keytab, and also in client mode. --- .../apache/spark/deploy/k8s/Constants.scala | 2 + .../KerberosConfDriverFeatureStep.scala | 81 +++++++++++++++---- .../k8s/KubernetesClusterManager.scala | 2 +- .../KubernetesClusterSchedulerBackend.scala | 15 +++- ...bernetesClusterSchedulerBackendSuite.scala | 9 ++- 5 files changed, 88 insertions(+), 21 deletions(-) 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 1c6d53c16871e..1155eadf34011 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 @@ -109,6 +109,8 @@ private[spark] object Constants { val KERBEROS_SPARK_USER_NAME = "spark.kubernetes.kerberos.spark-user-name" val KERBEROS_SECRET_KEY = "hadoop-tokens" + val KERBEROS_KEYTAB_VOLUME = "kerberos-keytab" + val KERBEROS_KEYTAB_MOUNT_POINT = "/mnt/secrets/kerberos-keytab" // Hadoop credentials secrets for the Spark app. val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index ce47933b7f700..6d678d6ef5c56 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -16,7 +16,11 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.HasMetadata +import java.io.File + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.codec.binary.Base64 import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} @@ -25,6 +29,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf import org.apache.spark.deploy.k8s.features.hadooputils._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ /** * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the @@ -107,13 +112,16 @@ private[spark] class KerberosConfDriverFeatureStep( } ) + private def ktSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-kerberos-keytab" + override def configurePod(pod: SparkPod): SparkPod = { val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( hadoopConfDirSpec.hadoopConfDir, newHadoopConfigMapName, hadoopConfDirSpec.hadoopConfigMapName, pod) - kerberosConfSpec.map { hSpec => + + val kerberizedPod = kerberosConfSpec.map { hSpec => HadoopBootstrapUtil.bootstrapKerberosPod( hSpec.dtSecretName, hSpec.dtSecretItemKey, @@ -126,20 +134,53 @@ private[spark] class KerberosConfDriverFeatureStep( HadoopBootstrapUtil.bootstrapSparkUserPod( kubeTokenManager.getCurrentUser.getShortUserName, hadoopBasedSparkPod)) + + if (keytab.isDefined) { + val podWitKeytab = new PodBuilder(kerberizedPod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(KERBEROS_KEYTAB_VOLUME) + .withNewSecret() + .withSecretName(ktSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithKeytab = new ContainerBuilder(kerberizedPod.container) + .addNewVolumeMount() + .withName(KERBEROS_KEYTAB_VOLUME) + .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) + .endVolumeMount() + .build() + + SparkPod(podWitKeytab, containerWithKeytab) + } else { + kerberizedPod + } } override def getAdditionalPodSystemProperties(): Map[String, String] = { - val resolvedConfValues = kerberosConfSpec.map { hSpec => - Map(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) - }.getOrElse( - Map(KERBEROS_SPARK_USER_NAME -> - kubeTokenManager.getCurrentUser.getShortUserName)) - Map(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse( - kubernetesConf.hadoopConfigMapName)) ++ resolvedConfValues + val krbConfValues = kerberosConfSpec match { + case Some(hSpec) => + Seq(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, + KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, + KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, + KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) + + case _ => + Seq(KERBEROS_SPARK_USER_NAME -> kubeTokenManager.getCurrentUser.getShortUserName) + } + + val keytabConf = keytab.map { path => + val ktName = new File(path).getName() + (KEYTAB.key -> s"$KERBEROS_KEYTAB_MOUNT_POINT/$ktName") + } + + val hadoopConf = Seq(HADOOP_CONFIG_MAP_NAME -> + hadoopConfDirSpec.hadoopConfigMapName.getOrElse(kubernetesConf.hadoopConfigMapName)) + + (hadoopConf ++ krbConfValues ++ keytabConf).toMap } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { @@ -158,8 +199,20 @@ private[spark] class KerberosConfDriverFeatureStep( val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) + val keytabSecret = keytab.map { kt => + val ktName = new File(kt).getName() + val ktData = Files.toByteArray(new File(kt)) + new SecretBuilder() + .withNewMetadata() + .withName(ktSecretName) + .endMetadata() + .addToData(ktName, Base64.encodeBase64String(ktData)) + .build() + } + hadoopConfConfigMap.toSeq ++ krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq + kerberosDTSecret.toSeq ++ + keytabSecret } } 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 ce10f766334ff..739766a17cb47 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 @@ -110,7 +110,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], - sc.env.rpcEnv, + sc, kubernetesClient, requestExecutorsService, snapshotsStore, 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 fa6dc2c479bbf..cb3458fbfc5c1 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 @@ -18,10 +18,13 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService -import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.SparkContext import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} @@ -29,7 +32,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, - rpcEnv: RpcEnv, + sc: SparkContext, kubernetesClient: KubernetesClient, requestExecutorsService: ExecutorService, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -37,7 +40,7 @@ private[spark] class KubernetesClusterSchedulerBackend( lifecycleEventHandler: ExecutorPodsLifecycleManager, watchEvents: ExecutorPodsWatchSnapshotSource, pollEvents: ExecutorPodsPollingSnapshotSource) - extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( requestExecutorsService) @@ -123,7 +126,11 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new KubernetesDriverEndpoint(rpcEnv, properties) + new KubernetesDriverEndpoint(sc.env.rpcEnv, properties) + } + + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration)) } private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 52e7a12dbaf06..75232f7b98b04 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -23,7 +23,7 @@ import org.mockito.Matchers.{eq => mockitoEq} import org.mockito.Mockito.{never, verify, when} import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} @@ -41,6 +41,9 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn @Mock private var sc: SparkContext = _ + @Mock + private var env: SparkEnv = _ + @Mock private var rpcEnv: RpcEnv = _ @@ -81,6 +84,8 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn MockitoAnnotations.initMocks(this) when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(env) + when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) when(rpcEnv.setupEndpoint( mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) @@ -88,7 +93,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(kubernetesClient.pods()).thenReturn(podOperations) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, - rpcEnv, + sc, kubernetesClient, requestExecutorsService, eventQueue, From d0ec409b518cf5a5fed324770545c6869f077316 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 7 Nov 2018 14:53:37 -0800 Subject: [PATCH 02/10] Big refactoring of client mode support for kerberos. This change re-works the k8s feature steps related to kerberos so that the driver and executor steps are more clearly defined, and less things need to be done in executor pods. The main driver behind a lot of these changes is that, in client mode, no driver-related config steps are run. So all executor-related steps must be able to be applied even when no driver steps have previously run and potentially modified the pod or Spark configuration. The second is that based on the above changes, a bunch of things that were being done on executor pods are actually not needed. These are: - adding the Hadoop config to the executor pods: this is not needed since the Spark driver will serialize the Hadoop config and send it to executors when running tasks. - mounting the kerberos config file in the executor pods: this is not needed once you remove the above. The Hadoop conf sent by the driver with the tasks is already resolved (i.e. has all the kerberos names properly defined), so executors do not need access to the kerberos realm information anymore. So there is a single thing that executor pods need: mounting the delegation token secret when one is provided. Everything else is handled in the driver, regardless of whether it's client or cluster mode. The change also avoids creating delegation tokens unnecessarily. This means that they'll only be created if a secret with tokens was not provided, and if a keytab is not provided. In either of those cases, the driver code will handle delegation tokens: in cluster mode by creating a secret and stashing them, in client mode by using existing mechanisms to send DTs to executors. One last feature: the change also allows defining a keytab with a "local:" URI. This is supported in client mode (although that's the same as not saying "local:"), and in k8s cluster mode. This allows the keytab to be mounted onto the image from a pre-existing secret, for example. Finally, the new code always sets SPARK_USER in the driver and executor pods. This is in line with how other resource managers behave: the submitting user reflects which user will access Hadoop services in the app. (With kerberos, that's overridden by the logged in user.) That user is unrelated to the OS user the app is running as inside the containers. Tested: - client and cluster mode with kinit - cluster mode with keytab - cluster mode with local: keytab - YARN cluster with keytab (to make sure it isn't broken) --- .../org/apache/spark/deploy/SparkSubmit.scala | 29 +- .../HadoopDelegationTokenManager.scala | 8 +- .../scala/org/apache/spark/util/Utils.scala | 8 + project/SparkBuild.scala | 3 +- .../apache/spark/deploy/k8s/Constants.scala | 7 +- .../spark/deploy/k8s/KubernetesConf.scala | 4 - .../apache/spark/deploy/k8s/SparkPod.scala | 25 +- .../k8s/features/BasicDriverFeatureStep.scala | 4 + .../features/BasicExecutorFeatureStep.scala | 4 + .../features/DelegationTokenFeatureStep.scala | 106 +++++++ .../HadoopConfDriverFeatureStep.scala | 115 +++++++ .../HadoopConfExecutorFeatureStep.scala | 48 --- .../HadoopSparkUserExecutorFeatureStep.scala | 43 --- .../KerberosConfDriverFeatureStep.scala | 218 ++++++-------- .../KerberosConfExecutorFeatureStep.scala | 53 ---- .../hadooputils/HadoopBootstrapUtil.scala | 283 ------------------ .../hadooputils/HadoopKerberosLogin.scala | 64 ---- .../hadooputils/KerberosConfigSpec.scala | 33 -- ...bernetesHadoopDelegationTokenManager.scala | 37 --- .../k8s/submit/KubernetesDriverBuilder.scala | 19 +- .../k8s/KubernetesExecutorBuilder.scala | 31 +- .../BasicDriverFeatureStepSuite.scala | 9 +- .../BasicExecutorFeatureStepSuite.scala | 11 +- .../submit/KubernetesDriverBuilderSuite.scala | 119 ++------ .../k8s/KubernetesExecutorBuilderSuite.scala | 91 +----- .../org/apache/spark/deploy/yarn/Client.scala | 24 +- 26 files changed, 451 insertions(+), 945 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala create mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0fc8c9bd789e0..8138d05063c26 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy import java.io._ import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} -import java.net.URL +import java.net.{URI, URL} import java.security.PrivilegedExceptionAction import java.text.ParseException import java.util.UUID @@ -334,19 +334,20 @@ private[spark] class SparkSubmit extends Logging { val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf)) val targetDir = Utils.createTempDir() - // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient || isKubernetesCluster) { - if (args.principal != null) { - if (args.keytab != null) { - require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") - // Add keytab and principal configurations in sysProps to make them available - // for later use; e.g. in spark sql, the isolated class loader used to talk - // to HiveMetastore will use these settings. They will be set as Java system - // properties and then loaded by SparkConf - sparkConf.set(KEYTAB, args.keytab) - sparkConf.set(PRINCIPAL, args.principal) - UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) - } + // Kerberos is not supported in standalone mode, and keytab support is not yet available + // in Mesos cluster mode. + if (clusterManager != STANDALONE + && !isMesosCluster + && args.principal != null + && args.keytab != null) { + // If client mode, make sure the keytab is just a local path. + if (deployMode == CLIENT && Utils.isLocalUri(args.keytab)) { + args.keytab = new URI(args.keytab).getPath() + } + + if (!Utils.isLocalUri(args.keytab)) { + require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index 10cd8742f2b49..d81b78cb47173 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.security import java.io.File +import java.net.URI import java.security.PrivilegedExceptionAction import java.util.concurrent.{ScheduledExecutorService, TimeUnit} import java.util.concurrent.atomic.AtomicReference @@ -71,11 +72,13 @@ private[spark] class HadoopDelegationTokenManager( private val providerEnabledConfig = "spark.security.credentials.%s.enabled" private val principal = sparkConf.get(PRINCIPAL).orNull - private val keytab = sparkConf.get(KEYTAB).orNull + + // The keytab can be a local: URI for cluster mode, so translate it to a regular path. If it is + // needed later on, the code will check that it exists. + private val keytab = sparkConf.get(KEYTAB).map { uri => new URI(uri).getPath() }.orNull require((principal == null) == (keytab == null), "Both principal and keytab must be defined, or neither.") - require(keytab == null || new File(keytab).isFile(), s"Cannot find keytab at $keytab.") private val delegationTokenProviders = loadProviders() logDebug("Using the following builtin delegation token providers: " + @@ -264,6 +267,7 @@ private[spark] class HadoopDelegationTokenManager( private def doLogin(): UserGroupInformation = { logInfo(s"Attempting to login to KDC using principal: $principal") + require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.") val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) logInfo("Successfully logged into KDC.") ugi 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 a07eee6ad8a4b..033e263ee99b3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -100,6 +100,9 @@ private[spark] object Utils extends Logging { */ val DEFAULT_MAX_TO_STRING_FIELDS = 25 + /** Scheme used for files that are locally available on worker nodes in the cluster. */ + val LOCAL_SCHEME = "local" + private[spark] def maxNumToStringFields = { if (SparkEnv.get != null) { SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) @@ -2864,6 +2867,11 @@ private[spark] object Utils extends Logging { def stringHalfWidth(str: String): Int = { if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size } + + /** Returns whether the URI is a "local:" URI. */ + def isLocalUri(uri: String): Boolean = { + uri.startsWith(s"$LOCAL_SCHEME:") + } } private[util] object CallerContext extends Logging { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5e034f9fe2a95..06bc5a72d556b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -516,7 +516,8 @@ object KubernetesIntegrationTests { s"-Dspark.kubernetes.test.unpackSparkDir=$sparkHome" ), // Force packaging before building images, so that the latest code is tested. - dockerBuild := dockerBuild.dependsOn(packageBin in Compile in assembly).value + dockerBuild := dockerBuild.dependsOn(packageBin in Compile in assembly) + .dependsOn(packageBin in Compile in examples).value ) } 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 1741486d3de01..76041e7de5182 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 @@ -87,24 +87,19 @@ private[spark] object Constants { val NON_JVM_MEMORY_OVERHEAD_FACTOR = 0.4d // Hadoop Configuration - val HADOOP_FILE_VOLUME = "hadoop-properties" + val HADOOP_CONF_VOLUME = "hadoop-properties" val KRB_FILE_VOLUME = "krb5-file" val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" val KRB_FILE_DIR_PATH = "/etc" val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" val HADOOP_CONFIG_MAP_NAME = "spark.kubernetes.executor.hadoopConfigMapName" - val KRB5_CONFIG_MAP_NAME = - "spark.kubernetes.executor.krb5ConfigMapName" // Kerberos Configuration - val KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME = "delegation-tokens" val KERBEROS_DT_SECRET_NAME = "spark.kubernetes.kerberos.dt-secret-name" val KERBEROS_DT_SECRET_KEY = "spark.kubernetes.kerberos.dt-secret-key" - val KERBEROS_SPARK_USER_NAME = - "spark.kubernetes.kerberos.spark-user-name" val KERBEROS_SECRET_KEY = "hadoop-tokens" val KERBEROS_KEYTAB_VOLUME = "kerberos-keytab" val KERBEROS_KEYTAB_MOUNT_POINT = "/mnt/secrets/kerberos-keytab" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index ebb81540bbbbe..aa1b56d9e6c88 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry @@ -82,9 +81,6 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def krbConfigMapName: String = s"$appResourceNamePrefix-krb5-file" - def tokenManager(conf: SparkConf, hConf: Configuration): KubernetesHadoopDelegationTokenManager = - new KubernetesHadoopDelegationTokenManager(conf, hConf) - def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala index 345dd117fd35f..fd1196368a7ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala @@ -18,7 +18,30 @@ package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -private[spark] case class SparkPod(pod: Pod, container: Container) +private[spark] case class SparkPod(pod: Pod, container: Container) { + + /** + * Convenience method to apply a series of chained transformations to a pod. + * + * Use it like: + * + * original.modify { case pod => + * // update pod and return new one + * }.modify { case pod => + * // more changes that create a new pod + * }.modify { + * case pod if someCondition => // new pod + * } + * + * This makes it cleaner to apply multiple transformations, avoiding having to create + * a bunch of awkwardly-named local variables. Since the argument is a partial function, + * it can do matching without needing to exhaust all the possibilities. If the function + * is not applied, then the original pod will be kept. + */ + def transform(fn: PartialFunction[SparkPod, SparkPod]): SparkPod = fn.lift(this).getOrElse(this) + +} + private[spark] object SparkPod { def initialPod(): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 5ddf73cb16a6f..55c49788b3d5a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -112,6 +112,10 @@ private[spark] class BasicDriverFeatureStep( .withContainerPort(driverUIPort) .withProtocol("TCP") .endPort() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(driverCustomEnvs.asJava) .addNewEnv() .withName(ENV_DRIVER_BIND_ADDRESS) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index 7f397e6e84fa5..385ec17c4a175 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -144,6 +144,10 @@ private[spark] class BasicExecutorFeatureStep( .addToLimits("memory", executorMemoryQuantity) .addToRequests("cpu", executorCpuQuantity) .endResources() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(executorEnv.asJava) .withPorts(requiredPorts.asJava) .addToArgs("executor") diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala new file mode 100644 index 0000000000000..50c5f5147cc79 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.config._ + +/** + * Delegation token support for Spark apps on kubernetes. + * + * When preparing driver resources, this step will generate delegation tokens for the app if + * they're needed. + * + * When preparing pods, this step will mount the delegation token secret (either pre-defined, + * or generated by this step when preparing the driver). + */ +private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDriver: Boolean) + extends KubernetesFeatureConfigStep { + + private val existingSecret = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val existingItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val shouldCreateTokens = isDriver && !conf.sparkConf.contains(KEYTAB) && + existingSecret.isEmpty && UserGroupInformation.isSecurityEnabled() + + KubernetesUtils.requireBothOrNeitherDefined( + existingSecret, + existingItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret", + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the item-key where the data is stored") + + private def dtSecretName: String = s"${conf.appResourceNamePrefix}-delegation-tokens" + + override def configurePod(pod: SparkPod): SparkPod = { + pod.transform { case pod if shouldCreateTokens | existingSecret.isDefined => + val secretName = existingSecret.getOrElse(dtSecretName) + val itemKey = existingItemKey.getOrElse(KERBEROS_SECRET_KEY) + + val podWithTokens = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithTokens = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") + .endEnv() + .build() + + SparkPod(podWithTokens, containerWithTokens) + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (shouldCreateTokens) { + val tokenManager = new HadoopDelegationTokenManager(conf.sparkConf, + SparkHadoopUtil.get.newConfiguration(conf.sparkConf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + val tokenData = SparkHadoopUtil.get.serialize(creds) + Seq(new SecretBuilder() + .withNewMetadata() + .withName(dtSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) + .build()) + } else { + Nil + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala new file mode 100644 index 0000000000000..74f933fce2c29 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration + * directory - on the driver pod. + */ +private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) + extends KubernetesFeatureConfigStep { + + private val confSpec = conf.hadoopConfSpec + private val confDir = confSpec.flatMap(_.hadoopConfDir) + private val confMap = confSpec.flatMap(_.hadoopConfigMapName) + + private lazy val confFiles: Seq[File] = { + val dir = new File(confDir.get) + if (dir.isDirectory) { + dir.listFiles.filter(_.isFile).toSeq + } else { + Nil + } + } + + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if confSpec.isDefined => + val confVolume = if (confDir.isDefined) { + val keyPaths = confFiles.map { file => + new KeyToPathBuilder() + .withKey(file.getName()) + .withPath(file.getName()) + .build() + } + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(conf.hadoopConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .build() + } else { + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(confMap.get) + .endConfigMap() + .build() + } + + val podWithConf = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(confVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_CONF_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + + SparkPod(podWithConf, containerWithMount) + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (confDir.isDefined) { + val fileMap = confFiles.map { file => + (file.getName(), Files.toString(file, StandardCharsets.UTF_8)) + }.toMap.asJava + + Seq(new ConfigMapBuilder() + .withNewMetadata() + .withName(conf.hadoopConfigMapName) + .endMetadata() + .addToData(fileMap) + .build()) + } else { + Nil + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala deleted file mode 100644 index fd09de2a918a1..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfExecutorFeatureStep.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for bootstraping the container with ConfigMaps - * containing Hadoop config files mounted as volumes and an ENV variable - * pointed to the mounted file directory. - */ -private[spark] class HadoopConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val sparkConf = kubernetesConf.sparkConf - val hadoopConfDirCMapName = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - require(hadoopConfDirCMapName.isDefined, - "Ensure that the env `HADOOP_CONF_DIR` is defined either in the client or " + - " using pre-existing ConfigMaps") - logInfo("HADOOP_CONF_DIR defined") - HadoopBootstrapUtil.bootstrapHadoopConfDir(None, None, hadoopConfDirCMapName, pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala deleted file mode 100644 index 5b6a6d5a7db45..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopSparkUserExecutorFeatureStep.scala +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for setting ENV_SPARK_USER when HADOOP_FILES are detected - * however, this step would not be run if Kerberos is enabled, as Kerberos sets SPARK_USER - */ -private[spark] class HadoopSparkUserExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - override def configurePod(pod: SparkPod): SparkPod = { - val sparkUserName = kubernetesConf.sparkConf.get(KERBEROS_SPARK_USER_NAME) - HadoopBootstrapUtil.bootstrapSparkUserPod(sparkUserName, pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 6d678d6ef5c56..b6a31b2f4a516 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -17,49 +17,33 @@ package org.apache.spark.deploy.k8s.features import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ import org.apache.commons.codec.binary.Base64 -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesDriverSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils /** * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. */ -private[spark] class KerberosConfDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesConf[_]) extends KubernetesFeatureConfigStep with Logging { - require(kubernetesConf.hadoopConfSpec.isDefined, - "Ensure that HADOOP_CONF_DIR is defined either via env or a pre-defined ConfigMap") - private val hadoopConfDirSpec = kubernetesConf.hadoopConfSpec.get private val conf = kubernetesConf.sparkConf private val principal = conf.get(org.apache.spark.internal.config.PRINCIPAL) private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) - private val existingSecretName = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val existingSecretItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) - private val kubeTokenManager = kubernetesConf.tokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) - private val isKerberosEnabled = - (hadoopConfDirSpec.hadoopConfDir.isDefined && kubeTokenManager.isSecurityEnabled) || - (hadoopConfDirSpec.hadoopConfigMapName.isDefined && - (krb5File.isDefined || krb5CMap.isDefined)) - require(keytab.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Keytab") - - require(existingSecretName.isEmpty || isKerberosEnabled, - "You must enable Kerberos support if you are specifying a Kerberos Secret") KubernetesUtils.requireNandDefined( krb5File, @@ -73,70 +57,60 @@ private[spark] class KerberosConfDriverFeatureStep( "If a Kerberos principal is specified you must also specify a Kerberos keytab", "If a Kerberos keytab is specified you must also specify a Kerberos principal") - KubernetesUtils.requireBothOrNeitherDefined( - existingSecretName, - existingSecretItemKey, - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret", - "If a secret storing a Kerberos Delegation Token is specified you must also" + - " specify the item-key where the data is stored") - - private val hadoopConfigurationFiles = hadoopConfDirSpec.hadoopConfDir.map { hConfDir => - HadoopBootstrapUtil.getHadoopConfFiles(hConfDir) + if (!hasKerberosConf) { + logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + + "Make sure that you have the krb5.conf locally on the driver image.") } - private val newHadoopConfigMapName = - if (hadoopConfDirSpec.hadoopConfigMapName.isEmpty) { - Some(kubernetesConf.hadoopConfigMapName) - } else { - None - } - // Either use pre-existing secret or login to create new Secret with DT stored within - private val kerberosConfSpec: Option[KerberosConfigSpec] = (for { - secretName <- existingSecretName - secretItemKey <- existingSecretItemKey - } yield { - KerberosConfigSpec( - dtSecret = None, - dtSecretName = secretName, - dtSecretItemKey = secretItemKey, - jobUserName = kubeTokenManager.getCurrentUser.getShortUserName) - }).orElse( - if (isKerberosEnabled) { - Some(HadoopKerberosLogin.buildSpec( - conf, - kubernetesConf.appResourceNamePrefix, - kubeTokenManager)) - } else { - None - } - ) + private val needKeytabUpload = keytab.exists(!Utils.isLocalUri(_)) private def ktSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-kerberos-keytab" - override def configurePod(pod: SparkPod): SparkPod = { - val hadoopBasedSparkPod = HadoopBootstrapUtil.bootstrapHadoopConfDir( - hadoopConfDirSpec.hadoopConfDir, - newHadoopConfigMapName, - hadoopConfDirSpec.hadoopConfigMapName, - pod) - - val kerberizedPod = kerberosConfSpec.map { hSpec => - HadoopBootstrapUtil.bootstrapKerberosPod( - hSpec.dtSecretName, - hSpec.dtSecretItemKey, - hSpec.jobUserName, - krb5File, - Some(kubernetesConf.krbConfigMapName), - krb5CMap, - hadoopBasedSparkPod) - }.getOrElse( - HadoopBootstrapUtil.bootstrapSparkUserPod( - kubeTokenManager.getCurrentUser.getShortUserName, - hadoopBasedSparkPod)) - - if (keytab.isDefined) { - val podWitKeytab = new PodBuilder(kerberizedPod.pod) + private def hasKerberosConf: Boolean = krb5CMap.isDefined | krb5File.isDefined + + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if hasKerberosConf => + val configMapVolume = if (krb5CMap.isDefined) { + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5CMap.get) + .endConfigMap() + .build() + } else { + val krb5Conf = new File(krb5File.get) + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(kubernetesConf.krbConfigMapName) + .withItems(new KeyToPathBuilder() + .withKey(krb5Conf.getName()) + .withPath(krb5Conf.getName()) + .build()) + .endConfigMap() + .build() + } + + val podWithVolume = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(configMapVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") + .endVolumeMount() + .build() + + SparkPod(podWithVolume, containerWithMount) + }.transform { case pod if needKeytabUpload => + // If keytab is defined and is a submission-local file (not local: URI), then create a + // secret for it. The keytab data will be stored in this secret below. + val podWitKeytab = new PodBuilder(pod.pod) .editOrNewSpec() .addNewVolume() .withName(KERBEROS_KEYTAB_VOLUME) @@ -147,7 +121,7 @@ private[spark] class KerberosConfDriverFeatureStep( .endSpec() .build() - val containerWithKeytab = new ContainerBuilder(kerberizedPod.container) + val containerWithKeytab = new ContainerBuilder(pod.container) .addNewVolumeMount() .withName(KERBEROS_KEYTAB_VOLUME) .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) @@ -155,64 +129,46 @@ private[spark] class KerberosConfDriverFeatureStep( .build() SparkPod(podWitKeytab, containerWithKeytab) - } else { - kerberizedPod } } override def getAdditionalPodSystemProperties(): Map[String, String] = { - val krbConfValues = kerberosConfSpec match { - case Some(hSpec) => - Seq(KERBEROS_DT_SECRET_NAME -> hSpec.dtSecretName, - KERBEROS_DT_SECRET_KEY -> hSpec.dtSecretItemKey, - KERBEROS_SPARK_USER_NAME -> hSpec.jobUserName, - KRB5_CONFIG_MAP_NAME -> krb5CMap.getOrElse(kubernetesConf.krbConfigMapName)) - - case _ => - Seq(KERBEROS_SPARK_USER_NAME -> kubeTokenManager.getCurrentUser.getShortUserName) - } - - val keytabConf = keytab.map { path => - val ktName = new File(path).getName() - (KEYTAB.key -> s"$KERBEROS_KEYTAB_MOUNT_POINT/$ktName") + // If a submission-local keytab is provided, update the Spark config so that it knows the + // path of the keytab in the driver container. + if (needKeytabUpload) { + val ktName = new File(keytab.get).getName() + Map(KEYTAB.key -> s"$KERBEROS_KEYTAB_MOUNT_POINT/$ktName") + } else { + Map.empty } - - val hadoopConf = Seq(HADOOP_CONFIG_MAP_NAME -> - hadoopConfDirSpec.hadoopConfigMapName.getOrElse(kubernetesConf.hadoopConfigMapName)) - - (hadoopConf ++ krbConfValues ++ keytabConf).toMap } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - val hadoopConfConfigMap = for { - hName <- newHadoopConfigMapName - hFiles <- hadoopConfigurationFiles - } yield { - HadoopBootstrapUtil.buildHadoopConfigMap(hName, hFiles) - } - - val krb5ConfigMap = krb5File.map { fileLocation => - HadoopBootstrapUtil.buildkrb5ConfigMap( - kubernetesConf.krbConfigMapName, - fileLocation) + Seq[HasMetadata]() ++ { + krb5File.map { path => + val file = new File(path) + new ConfigMapBuilder() + .withNewMetadata() + .withName(kubernetesConf.krbConfigMapName) + .endMetadata() + .addToData( + Map(file.getName() -> Files.toString(file, StandardCharsets.UTF_8)).asJava) + .build() + } + } ++ { + // If a submission-local keytab is provided, stash it in a secret. + if (needKeytabUpload) { + val kt = new File(keytab.get) + Seq(new SecretBuilder() + .withNewMetadata() + .withName(ktSecretName) + .endMetadata() + .addToData(kt.getName(), Base64.encodeBase64String(Files.toByteArray(kt))) + .build()) + } else { + Nil + } } - - val kerberosDTSecret = kerberosConfSpec.flatMap(_.dtSecret) - - val keytabSecret = keytab.map { kt => - val ktName = new File(kt).getName() - val ktData = Files.toByteArray(new File(kt)) - new SecretBuilder() - .withNewMetadata() - .withName(ktSecretName) - .endMetadata() - .addToData(ktName, Base64.encodeBase64String(ktData)) - .build() - } - - hadoopConfConfigMap.toSeq ++ - krb5ConfigMap.toSeq ++ - kerberosDTSecret.toSeq ++ - keytabSecret } + } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala deleted file mode 100644 index 06a88b6c229f7..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfExecutorFeatureStep.scala +++ /dev/null @@ -1,53 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.HasMetadata - -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.KubernetesExecutorSpecificConf -import org.apache.spark.deploy.k8s.features.hadooputils.HadoopBootstrapUtil -import org.apache.spark.internal.Logging - -/** - * This step is responsible for mounting the DT secret for the executors - */ -private[spark] class KerberosConfExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) - extends KubernetesFeatureConfigStep with Logging { - - private val sparkConf = kubernetesConf.sparkConf - private val maybeKrb5CMap = sparkConf.getOption(KRB5_CONFIG_MAP_NAME) - require(maybeKrb5CMap.isDefined, "HADOOP_CONF_DIR ConfigMap not found") - - override def configurePod(pod: SparkPod): SparkPod = { - logInfo(s"Mounting Resources for Kerberos") - HadoopBootstrapUtil.bootstrapKerberosPod( - sparkConf.get(KERBEROS_DT_SECRET_NAME), - sparkConf.get(KERBEROS_DT_SECRET_KEY), - sparkConf.get(KERBEROS_SPARK_USER_NAME), - None, - None, - maybeKrb5CMap, - pod) - } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty[HasMetadata] -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala deleted file mode 100644 index 5bee766caf2be..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopBootstrapUtil.scala +++ /dev/null @@ -1,283 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import java.io.File -import java.nio.charset.StandardCharsets - -import scala.collection.JavaConverters._ - -import com.google.common.io.Files -import io.fabric8.kubernetes.api.model._ - -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.SparkPod -import org.apache.spark.internal.Logging - -private[spark] object HadoopBootstrapUtil extends Logging { - - /** - * Mounting the DT secret for both the Driver and the executors - * - * @param dtSecretName Name of the secret that stores the Delegation Token - * @param dtSecretItemKey Name of the Item Key storing the Delegation Token - * @param userName Name of the SparkUser to set SPARK_USER - * @param fileLocation Optional Location of the krb5 file - * @param newKrb5ConfName Optional location of the ConfigMap for Krb5 - * @param existingKrb5ConfName Optional name of ConfigMap for Krb5 - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapKerberosPod( - dtSecretName: String, - dtSecretItemKey: String, - userName: String, - fileLocation: Option[String], - newKrb5ConfName: Option[String], - existingKrb5ConfName: Option[String], - pod: SparkPod): SparkPod = { - - val preConfigMapVolume = existingKrb5ConfName.map { kconf => - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(kconf) - .endConfigMap() - .build() - } - - val createConfigMapVolume = for { - fLocation <- fileLocation - krb5ConfName <- newKrb5ConfName - } yield { - val krb5File = new File(fLocation) - val fileStringPath = krb5File.toPath.getFileName.toString - new VolumeBuilder() - .withName(KRB_FILE_VOLUME) - .withNewConfigMap() - .withName(krb5ConfName) - .withItems(new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build()) - .endConfigMap() - .build() - } - - // Breaking up Volume creation for clarity - val configMapVolume = preConfigMapVolume.orElse(createConfigMapVolume) - if (configMapVolume.isEmpty) { - logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + - "Make sure that you have the krb5.conf locally on the Driver and Executor images") - } - - val kerberizedPodWithDTSecret = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(dtSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() - - // Optionally add the krb5.conf ConfigMap - val kerberizedPod = configMapVolume.map { cmVolume => - new PodBuilder(kerberizedPodWithDTSecret) - .editSpec() - .addNewVolumeLike(cmVolume) - .endVolume() - .endSpec() - .build() - }.getOrElse(kerberizedPodWithDTSecret) - - val kerberizedContainerWithMounts = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$dtSecretItemKey") - .endEnv() - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(userName) - .endEnv() - .build() - - // Optionally add the krb5.conf Volume Mount - val kerberizedContainer = - if (configMapVolume.isDefined) { - new ContainerBuilder(kerberizedContainerWithMounts) - .addNewVolumeMount() - .withName(KRB_FILE_VOLUME) - .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") - .withSubPath("krb5.conf") - .endVolumeMount() - .build() - } else { - kerberizedContainerWithMounts - } - - SparkPod(kerberizedPod, kerberizedContainer) - } - - /** - * setting ENV_SPARK_USER when HADOOP_FILES are detected - * - * @param sparkUserName Name of the SPARK_USER - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapSparkUserPod(sparkUserName: String, pod: SparkPod): SparkPod = { - val envModifiedContainer = new ContainerBuilder(pod.container) - .addNewEnv() - .withName(ENV_SPARK_USER) - .withValue(sparkUserName) - .endEnv() - .build() - SparkPod(pod.pod, envModifiedContainer) - } - - /** - * Grabbing files in the HADOOP_CONF_DIR - * - * @param path location of HADOOP_CONF_DIR - * @return a list of File object - */ - def getHadoopConfFiles(path: String): Seq[File] = { - val dir = new File(path) - if (dir.isDirectory) { - dir.listFiles.filter(_.isFile).toSeq - } else { - Seq.empty[File] - } - } - - /** - * Bootstraping the container with ConfigMaps that store - * Hadoop configuration files - * - * @param hadoopConfDir directory location of HADOOP_CONF_DIR env - * @param newHadoopConfigMapName name of the new configMap for HADOOP_CONF_DIR - * @param existingHadoopConfigMapName name of the pre-defined configMap for HADOOP_CONF_DIR - * @param pod Input pod to be appended to - * @return a modified SparkPod - */ - def bootstrapHadoopConfDir( - hadoopConfDir: Option[String], - newHadoopConfigMapName: Option[String], - existingHadoopConfigMapName: Option[String], - pod: SparkPod): SparkPod = { - val preConfigMapVolume = existingHadoopConfigMapName.map { hConf => - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConf) - .endConfigMap() - .build() } - - val createConfigMapVolume = for { - dirLocation <- hadoopConfDir - hConfName <- newHadoopConfigMapName - } yield { - val hadoopConfigFiles = getHadoopConfFiles(dirLocation) - val keyPaths = hadoopConfigFiles.map { file => - val fileStringPath = file.toPath.getFileName.toString - new KeyToPathBuilder() - .withKey(fileStringPath) - .withPath(fileStringPath) - .build() - } - new VolumeBuilder() - .withName(HADOOP_FILE_VOLUME) - .withNewConfigMap() - .withName(hConfName) - .withItems(keyPaths.asJava) - .endConfigMap() - .build() - } - - // Breaking up Volume Creation for clarity - val configMapVolume = preConfigMapVolume.getOrElse(createConfigMapVolume.get) - - val hadoopSupportedPod = new PodBuilder(pod.pod) - .editSpec() - .addNewVolumeLike(configMapVolume) - .endVolume() - .endSpec() - .build() - - val hadoopSupportedContainer = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(HADOOP_FILE_VOLUME) - .withMountPath(HADOOP_CONF_DIR_PATH) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_CONF_DIR) - .withValue(HADOOP_CONF_DIR_PATH) - .endEnv() - .build() - SparkPod(hadoopSupportedPod, hadoopSupportedContainer) - } - - /** - * Builds ConfigMap given the file location of the - * krb5.conf file - * - * @param configMapName name of configMap for krb5 - * @param fileLocation location of krb5 file - * @return a ConfigMap - */ - def buildkrb5ConfigMap( - configMapName: String, - fileLocation: String): ConfigMap = { - val file = new File(fileLocation) - new ConfigMapBuilder() - .withNewMetadata() - .withName(configMapName) - .endMetadata() - .addToData(Map(file.toPath.getFileName.toString -> - Files.toString(file, StandardCharsets.UTF_8)).asJava) - .build() - } - - /** - * Builds ConfigMap given the ConfigMap name - * and a list of Hadoop Conf files - * - * @param hadoopConfigMapName name of hadoopConfigMap - * @param hadoopConfFiles list of hadoopFiles - * @return a ConfigMap - */ - def buildHadoopConfigMap( - hadoopConfigMapName: String, - hadoopConfFiles: Seq[File]): ConfigMap = { - new ConfigMapBuilder() - .withNewMetadata() - .withName(hadoopConfigMapName) - .endMetadata() - .addToData(hadoopConfFiles.map { file => - (file.toPath.getFileName.toString, - Files.toString(file, StandardCharsets.UTF_8)) - }.toMap.asJava) - .build() - } - -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala deleted file mode 100644 index 0022d8f242a72..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/HadoopKerberosLogin.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.SecretBuilder -import org.apache.commons.codec.binary.Base64 - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.security.KubernetesHadoopDelegationTokenManager - -/** - * This logic does all the heavy lifting for Delegation Token creation. This step - * assumes that the job user has either specified a principal and keytab or ran - * $kinit before running spark-submit. By running UGI.getCurrentUser we are able - * to obtain the current user, either signed in via $kinit or keytab. With the - * Job User principal you then retrieve the delegation token from the NameNode - * and store values in DelegationToken. Lastly, the class puts the data into - * a secret. All this is defined in a KerberosConfigSpec. - */ -private[spark] object HadoopKerberosLogin { - def buildSpec( - submissionSparkConf: SparkConf, - kubernetesResourceNamePrefix: String, - tokenManager: KubernetesHadoopDelegationTokenManager): KerberosConfigSpec = { - // The JobUserUGI will be taken fom the Local Ticket Cache or via keytab+principal - // The login happens in the SparkSubmit so login logic is not necessary to include - val jobUserUGI = tokenManager.getCurrentUser - val originalCredentials = jobUserUGI.getCredentials - tokenManager.obtainDelegationTokens(originalCredentials) - - val tokenData = SparkHadoopUtil.get.serialize(originalCredentials) - - val initialTokenDataKeyName = KERBEROS_SECRET_KEY - val newSecretName = s"$kubernetesResourceNamePrefix-$KERBEROS_DELEGEGATION_TOKEN_SECRET_NAME" - val secretDT = - new SecretBuilder() - .withNewMetadata() - .withName(newSecretName) - .endMetadata() - .addToData(initialTokenDataKeyName, Base64.encodeBase64String(tokenData)) - .build() - KerberosConfigSpec( - dtSecret = Some(secretDT), - dtSecretName = newSecretName, - dtSecretItemKey = initialTokenDataKeyName, - jobUserName = jobUserUGI.getShortUserName) - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala deleted file mode 100644 index 7f7ef216cf485..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/hadooputils/KerberosConfigSpec.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.hadooputils - -import io.fabric8.kubernetes.api.model.Secret - -/** - * Represents a given configuration of the Kerberos Configuration logic - *

- * - The secret containing a DT, either previously specified or built on the fly - * - The name of the secret where the DT will be stored - * - The data item-key on the secret which correlates with where the current DT data is stored - * - The Job User's username - */ -private[spark] case class KerberosConfigSpec( - dtSecret: Option[Secret], - dtSecretName: String, - dtSecretItemKey: String, - jobUserName: String) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala deleted file mode 100644 index 3e98d5811d83f..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/security/KubernetesHadoopDelegationTokenManager.scala +++ /dev/null @@ -1,37 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.k8s.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.HadoopDelegationTokenManager - -/** - * Adds Kubernetes-specific functionality to HadoopDelegationTokenManager. - */ -private[spark] class KubernetesHadoopDelegationTokenManager( - _sparkConf: SparkConf, - _hadoopConf: Configuration) - extends HadoopDelegationTokenManager(_sparkConf, _hadoopConf) { - - def getCurrentUser: UserGroupInformation = UserGroupInformation.getCurrentUser - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled - -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index be4daec3b1bb9..250c30368fd2d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -48,10 +48,12 @@ private[spark] class KubernetesDriverBuilder( KubernetesConf[KubernetesDriverSpecificConf] => DriverCommandFeatureStep) = new DriverCommandFeatureStep(_), - provideHadoopGlobalStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => KerberosConfDriverFeatureStep) = - new KerberosConfDriverFeatureStep(_), + provideHadoopConfStep: (KubernetesConf[_] => HadoopConfDriverFeatureStep) = + new HadoopConfDriverFeatureStep(_), + provideKerberosConfStep: (KubernetesConf[_] => KerberosConfDriverFeatureStep) = + new KerberosConfDriverFeatureStep(_), + provideDelegationTokenStep: (KubernetesConf[_] => DelegationTokenFeatureStep) = + new DelegationTokenFeatureStep(_, true), providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => PodTemplateConfigMapStep) = new PodTemplateConfigMapStep(_), @@ -81,14 +83,15 @@ private[spark] class KubernetesDriverBuilder( val driverCommandStep = provideDriverCommandStep(kubernetesConf) - val maybeHadoopConfigStep = - kubernetesConf.hadoopConfSpec.map { _ => - provideHadoopGlobalStep(kubernetesConf)} + val otherSteps = Seq( + provideHadoopConfStep(kubernetesConf), + provideKerberosConfStep(kubernetesConf), + provideDelegationTokenStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ Seq(driverCommandStep) ++ secretFeature ++ envSecretFeature ++ volumesFeature ++ - maybeHadoopConfigStep.toSeq ++ podTemplateFeature + otherSteps ++ podTemplateFeature var spec = KubernetesDriverSpec( provideInitialPod(), diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 089f84dec277f..0e90e548061b9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -41,27 +41,13 @@ private[spark] class KubernetesExecutorBuilder( provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideHadoopConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopConfExecutorFeatureStep) = - new HadoopConfExecutorFeatureStep(_), - provideKerberosConfStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => KerberosConfExecutorFeatureStep) = - new KerberosConfExecutorFeatureStep(_), - provideHadoopSparkUserStep: ( - KubernetesConf[KubernetesExecutorSpecificConf] - => HadoopSparkUserExecutorFeatureStep) = - new HadoopSparkUserExecutorFeatureStep(_), + provideDelegationTokenStep: (KubernetesConf[_] => DelegationTokenFeatureStep) = + new DelegationTokenFeatureStep(_, false), provideInitialPod: () => SparkPod = SparkPod.initialPod) { def buildFromFeatures( kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { val sparkConf = kubernetesConf.sparkConf - val maybeHadoopConfigMap = sparkConf.getOption(HADOOP_CONFIG_MAP_NAME) - val maybeDTSecretName = sparkConf.getOption(KERBEROS_DT_SECRET_NAME) - val maybeDTDataItem = sparkConf.getOption(KERBEROS_DT_SECRET_KEY) - val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { Seq(provideSecretsStep(kubernetesConf)) @@ -73,23 +59,14 @@ private[spark] class KubernetesExecutorBuilder( Seq(provideVolumesStep(kubernetesConf)) } else Nil - val maybeHadoopConfFeatureSteps = maybeHadoopConfigMap.map { _ => - val maybeKerberosStep = - if (maybeDTSecretName.isDefined && maybeDTDataItem.isDefined) { - provideKerberosConfStep(kubernetesConf) - } else { - provideHadoopSparkUserStep(kubernetesConf) - } - Seq(provideHadoopConfStep(kubernetesConf)) :+ - maybeKerberosStep - }.getOrElse(Seq.empty[KubernetesFeatureConfigStep]) + val dtSecretStep = Seq(provideDelegationTokenStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ secretFeature ++ secretEnvFeature ++ volumesFeature ++ - maybeHadoopConfFeatureSteps + dtSecretStep var executorPod = provideInitialPod() for (feature <- allFeatures) { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 1e7dfbeffdb24..3c620bd0666e1 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ import org.apache.spark.ui.SparkUI +import org.apache.spark.util.Utils class BasicDriverFeatureStepSuite extends SparkFunSuite { @@ -95,15 +96,15 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val foundPortNames = configuredPod.container.getPorts.asScala.toSet assert(expectedPortNames === foundPortNames) - assert(configuredPod.container.getEnv.size === 3) val envs = configuredPod.container .getEnv .asScala .map(env => (env.getName, env.getValue)) .toMap - assert(envs(DRIVER_CUSTOM_ENV1) === DRIVER_ENVS(DRIVER_CUSTOM_ENV1)) - assert(envs(DRIVER_CUSTOM_ENV2) === DRIVER_ENVS(DRIVER_CUSTOM_ENV2)) - + DRIVER_ENVS.foreach { case (k, v) => + assert(envs(v) === v) + } + assert(envs(ENV_SPARK_USER) === Utils.getCurrentUserName()) assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index e9a16aab6ccc2..e2e7cecf1c79f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ +import org.apache.hadoop.security.UserGroupInformation import org.mockito.MockitoAnnotations import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} @@ -28,6 +29,7 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { @@ -201,12 +203,15 @@ class BasicExecutorFeatureStepSuite ENV_EXECUTOR_MEMORY -> "1g", ENV_APPLICATION_ID -> APP_ID, ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, - ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars + ENV_EXECUTOR_POD_IP -> null, + ENV_SPARK_USER -> Utils.getCurrentUserName()) - assert(executorPod.container.getEnv.size() === defaultEnvs.size) + val allEnvs = defaultEnvs ++ additionalEnvVars + + assert(executorPod.container.getEnv.size() === allEnvs.size) val mapEnvs = executorPod.container.getEnv.asScala.map { x => (x.getName, x.getValue) }.toMap - assert(defaultEnvs === mapEnvs) + assert(allEnvs === mapEnvs) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index fe900fda6e545..2cc18077cb695 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -34,7 +34,9 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val SECRETS_STEP_TYPE = "mount-secrets" private val DRIVER_CMD_STEP_TYPE = "driver-command" private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val HADOOP_GLOBAL_STEP_TYPE = "hadoop-global" + private val HADOOP_CONF_STEP_TYPE = "hadoop-conf" + private val KERBEROS_CONF_STEP_TYPE = "kerberos-conf" + private val DELEGATION_TOKEN_CONF_STEP_TYPE = "delegation-tokens" private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" private val TEMPLATE_VOLUME_STEP_TYPE = "template-volume" @@ -59,8 +61,14 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - private val hadoopGlobalStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_GLOBAL_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) + private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + HADOOP_CONF_STEP_TYPE, classOf[HadoopConfDriverFeatureStep]) + + private val kerberosStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) + + private val delegationTokenStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + DELEGATION_TOKEN_CONF_STEP_TYPE, classOf[DelegationTokenFeatureStep]) private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) @@ -69,6 +77,15 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { TEMPLATE_VOLUME_STEP_TYPE, classOf[PodTemplateConfigMapStep] ) + private val ALWAYS_ON_STEPS = Seq(BASIC_STEP_TYPE, + CREDENTIALS_STEP_TYPE, + SERVICE_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + DRIVER_CMD_STEP_TYPE, + HADOOP_CONF_STEP_TYPE, + KERBEROS_CONF_STEP_TYPE, + DELEGATION_TOKEN_CONF_STEP_TYPE) + private val builderUnderTest: KubernetesDriverBuilder = new KubernetesDriverBuilder( _ => basicFeatureStep, @@ -79,7 +96,9 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => localDirsStep, _ => mountVolumesStep, _ => driverCommandStep, - _ => hadoopGlobalStep, + _ => hadoopConfStep, + _ => kerberosStep, + _ => delegationTokenStep, _ => templateVolumeStep) test("Apply fundamental steps all the time.") { @@ -99,13 +118,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Map.empty, Nil, hadoopConfSpec = None) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + validateStepTypesApplied(builderUnderTest.buildFromFeatures(conf)) } test("Apply secrets step if secrets are present.") { @@ -127,13 +140,8 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, SECRETS_STEP_TYPE, - ENV_SECRETS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + ENV_SECRETS_STEP_TYPE) } test("Apply volumes step if mounts are present.") { @@ -160,12 +168,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { hadoopConfSpec = None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE) + MOUNT_VOLUMES_STEP_TYPE) } test("Apply template volume step if executor template is present.") { @@ -190,78 +193,14 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { Option.empty) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, TEMPLATE_VOLUME_STEP_TYPE) } - test("Apply HadoopSteps if HADOOP_CONF_DIR is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - Some("/var/hadoop-conf"), - None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - - test("Apply HadoopSteps if HADOOP_CONF ConfigMap is defined.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - JavaMainAppResource(None), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some( - HadoopConfSpec( - None, - Some("pre-defined-configMapName")))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) - } - private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) : Unit = { - assert(resolvedSpec.systemProperties.size === stepTypes.size) - stepTypes.foreach { stepType => + val validSteps = (stepTypes ++ ALWAYS_ON_STEPS).toSet + assert(resolvedSpec.systemProperties.keys === validSteps) + validSteps.foreach { stepType => assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) assert(resolvedSpec.driverKubernetesResources.containsSlice( KubernetesFeaturesTestUtils.getSecretsForStepType(stepType))) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index 1fea08c37ccc6..19a5569ab7119 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -16,6 +16,8 @@ */ package org.apache.spark.scheduler.cluster.k8s +import scala.collection.JavaConverters._ + import io.fabric8.kubernetes.api.model.{Config => _, _} import io.fabric8.kubernetes.client.KubernetesClient import org.mockito.Mockito.{mock, never, verify} @@ -31,9 +33,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { private val SECRETS_STEP_TYPE = "mount-secrets" private val ENV_SECRETS_STEP_TYPE = "env-secrets" private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val HADOOP_CONF_STEP_TYPE = "hadoop-conf-step" - private val HADOOP_SPARK_USER_STEP_TYPE = "hadoop-spark-user" - private val KERBEROS_CONF_STEP_TYPE = "kerberos-step" + private val DELEGATION_TOKEN_CONF_STEP_TYPE = "delegation-token-step" private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( @@ -44,24 +44,22 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - private val hadoopConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_CONF_STEP_TYPE, classOf[HadoopConfExecutorFeatureStep]) - private val hadoopSparkUser = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - HADOOP_SPARK_USER_STEP_TYPE, classOf[HadoopSparkUserExecutorFeatureStep]) - private val kerberosConf = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfExecutorFeatureStep]) + private val dtokenConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( + DELEGATION_TOKEN_CONF_STEP_TYPE, classOf[DelegationTokenFeatureStep]) private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) + private val ALWAYS_ON_STEPS = Seq(BASIC_STEP_TYPE, + LOCAL_DIRS_STEP_TYPE, + DELEGATION_TOKEN_CONF_STEP_TYPE) + private val builderUnderTest = new KubernetesExecutorBuilder( _ => basicFeatureStep, _ => mountSecretsStep, _ => envSecretsStep, _ => localDirsStep, _ => mountVolumesStep, - _ => hadoopConfStep, - _ => kerberosConf, - _ => hadoopSparkUser) + _ => dtokenConfStep) test("Basic steps are consistently applied.") { val conf = KubernetesConf( @@ -77,8 +75,7 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { Map.empty, Nil, None) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) + validateStepTypesApplied(builderUnderTest.buildFromFeatures(conf)) } test("Apply secrets step if secrets are present.") { @@ -97,8 +94,6 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, SECRETS_STEP_TYPE, ENV_SECRETS_STEP_TYPE) } @@ -124,72 +119,12 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { None) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, MOUNT_VOLUMES_STEP_TYPE) } - test("Apply basicHadoop step if HADOOP_CONF_DIR is defined") { - // HADOOP_DELEGATION_TOKEN - val HADOOP_CREDS_PREFIX = "spark.security.credentials." - val HADOOPFS_PROVIDER = s"$HADOOP_CREDS_PREFIX.hadoopfs.enabled" - val conf = KubernetesConf( - new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(HADOOPFS_PROVIDER, "true"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(Some("/var/hadoop-conf"), None))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - HADOOP_SPARK_USER_STEP_TYPE) - } - - test("Apply kerberos step if DT secrets created") { - val conf = KubernetesConf( - new SparkConf(false) - .set(HADOOP_CONFIG_MAP_NAME, "hadoop-conf-map-name") - .set(KRB5_CONFIG_MAP_NAME, "krb5-conf-map-name") - .set(KERBEROS_SPARK_USER_NAME, "spark-user") - .set(KERBEROS_DT_SECRET_NAME, "dt-secret") - .set(KERBEROS_DT_SECRET_KEY, "dt-key"), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Some(HadoopConfSpec(None, Some("pre-defined-onfigMapName")))) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - HADOOP_CONF_STEP_TYPE, - KERBEROS_CONF_STEP_TYPE) - } - private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { - assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) - stepTypes.foreach { stepType => - assert(resolvedPod.pod.getMetadata.getLabels.get(stepType) === stepType) - } + val validSteps = (stepTypes ++ ALWAYS_ON_STEPS).toSet + assert(resolvedPod.pod.getMetadata.getLabels.keySet.asScala === validSteps) } test("Starts with empty executor pod if template is not specified") { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 6240f7b68d2c8..184fb6a8ad13e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -116,6 +116,8 @@ private[spark] class Client( } } + require(keytab == null || !Utils.isLocalUri(keytab), "Keytab should reference a local file.") + private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sparkConf @@ -472,7 +474,7 @@ private[spark] class Client( appMasterOnly: Boolean = false): (Boolean, String) = { val trimmedPath = path.trim() val localURI = Utils.resolveURI(trimmedPath) - if (localURI.getScheme != LOCAL_SCHEME) { + if (localURI.getScheme != Utils.LOCAL_SCHEME) { if (addDistributedUri(localURI)) { val localPath = getQualifiedLocalPath(localURI, hadoopConf) val linkname = targetDir.map(_ + "/").getOrElse("") + @@ -515,7 +517,7 @@ private[spark] class Client( val sparkArchive = sparkConf.get(SPARK_ARCHIVE) if (sparkArchive.isDefined) { val archive = sparkArchive.get - require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") + require(!Utils.isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") distribute(Utils.resolveURI(archive).toString, resType = LocalResourceType.ARCHIVE, destName = Some(LOCALIZED_LIB_DIR)) @@ -525,7 +527,7 @@ private[spark] class Client( // Break the list of jars to upload, and resolve globs. val localJars = new ArrayBuffer[String]() jars.foreach { jar => - if (!isLocalUri(jar)) { + if (!Utils.isLocalUri(jar)) { val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf) val pathFs = FileSystem.get(path.toUri(), hadoopConf) pathFs.globStatus(path).filter(_.isFile()).foreach { entry => @@ -814,7 +816,7 @@ private[spark] class Client( } (pySparkArchives ++ pyArchives).foreach { path => val uri = Utils.resolveURI(path) - if (uri.getScheme != LOCAL_SCHEME) { + if (uri.getScheme != Utils.LOCAL_SCHEME) { pythonPath += buildPath(Environment.PWD.$$(), new Path(uri).getName()) } else { pythonPath += uri.getPath() @@ -1183,9 +1185,6 @@ private object Client extends Logging { // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" - // URI scheme that identifies local resources - val LOCAL_SCHEME = "local" - // Staging directory for any temporary jars or files val SPARK_STAGING: String = ".sparkStaging" @@ -1307,7 +1306,7 @@ private object Client extends Logging { addClasspathEntry(buildPath(Environment.PWD.$$(), LOCALIZED_LIB_DIR, "*"), env) if (sparkConf.get(SPARK_ARCHIVE).isEmpty) { sparkConf.get(SPARK_JARS).foreach { jars => - jars.filter(isLocalUri).foreach { jar => + jars.filter(Utils.isLocalUri).foreach { jar => val uri = new URI(jar) addClasspathEntry(getClusterPath(sparkConf, uri.getPath()), env) } @@ -1340,7 +1339,7 @@ private object Client extends Logging { private def getMainJarUri(mainJar: Option[String]): Option[URI] = { mainJar.flatMap { path => val uri = Utils.resolveURI(path) - if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None + if (uri.getScheme == Utils.LOCAL_SCHEME) Some(uri) else None }.orElse(Some(new URI(APP_JAR_NAME))) } @@ -1368,7 +1367,7 @@ private object Client extends Logging { uri: URI, fileName: String, env: HashMap[String, String]): Unit = { - if (uri != null && uri.getScheme == LOCAL_SCHEME) { + if (uri != null && uri.getScheme == Utils.LOCAL_SCHEME) { addClasspathEntry(getClusterPath(conf, uri.getPath), env) } else if (fileName != null) { addClasspathEntry(buildPath(Environment.PWD.$$(), fileName), env) @@ -1489,11 +1488,6 @@ private object Client extends Logging { components.mkString(Path.SEPARATOR) } - /** Returns whether the URI is a "local:" URI. */ - def isLocalUri(uri: String): Boolean = { - uri.startsWith(s"$LOCAL_SCHEME:") - } - def createAppReport(report: ApplicationReport): YarnAppReport = { val diags = report.getDiagnostics() val diagsOpt = if (diags != null && diags.nonEmpty) Some(diags) else None From 30f42cdd46c5ccad5736697aa2aaacbfe45e965e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 13 Nov 2018 16:29:59 -0800 Subject: [PATCH 03/10] Unit tests. --- .../features/DelegationTokenFeatureStep.scala | 20 ++- .../KerberosConfDriverFeatureStep.scala | 4 +- .../DelegationTokenFeatureStepSuite.scala | 100 ++++++++++++++ .../HadoopConfDriverFeatureStepSuite.scala | 94 +++++++++++++ .../KerberosConfDriverFeatureStepSuite.scala | 125 ++++++++++++++++++ .../KubernetesFeaturesTestUtils.scala | 6 + 6 files changed, 341 insertions(+), 8 deletions(-) create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala create mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala index 50c5f5147cc79..54bf3d5cdbf7a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala @@ -42,7 +42,7 @@ private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDrive private val existingSecret = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) private val existingItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) private val shouldCreateTokens = isDriver && !conf.sparkConf.contains(KEYTAB) && - existingSecret.isEmpty && UserGroupInformation.isSecurityEnabled() + existingSecret.isEmpty && isSecurityEnabled KubernetesUtils.requireBothOrNeitherDefined( existingSecret, @@ -87,11 +87,7 @@ private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDrive override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { if (shouldCreateTokens) { - val tokenManager = new HadoopDelegationTokenManager(conf.sparkConf, - SparkHadoopUtil.get.newConfiguration(conf.sparkConf)) - val creds = UserGroupInformation.getCurrentUser().getCredentials() - tokenManager.obtainDelegationTokens(creds) - val tokenData = SparkHadoopUtil.get.serialize(creds) + val tokenData = createDelegationTokens() Seq(new SecretBuilder() .withNewMetadata() .withName(dtSecretName) @@ -103,4 +99,16 @@ private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDrive } } + // Visible for testing. + def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled() + + // Visible for testing. + def createDelegationTokens(): Array[Byte] = { + val tokenManager = new HadoopDelegationTokenManager(conf.sparkConf, + SparkHadoopUtil.get.newConfiguration(conf.sparkConf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + SparkHadoopUtil.get.serialize(creds) + } + } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index b6a31b2f4a516..50e4377c908b0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -33,8 +33,8 @@ import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** - * Runs the necessary Hadoop-based logic based on Kerberos configs and the presence of the - * HADOOP_CONF_DIR. This runs various bootstrap methods defined in HadoopBootstrapUtil. + * Mount the kerberos configuration defined by the user in the driver pod, and make the + * user's keytab available to the driver if provided. */ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesConf[_]) extends KubernetesFeatureConfigStep with Logging { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala new file mode 100644 index 0000000000000..77f1a6b53e8b1 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.Secret + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.internal.config._ + +class DelegationTokenFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + test("mount delegation tokens if provided") { + val dtSecret = "tokenSecret" + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, dtSecret) + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtokens") + val conf = createConf(sparkConf) + val step = new DelegationTokenFeatureStep(conf, true) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("do nothing if a keytab is provided") { + val sparkConf = new SparkConf(false).set(KEYTAB, "fakeKeytab") + val conf = createConf(sparkConf) + val step = new DelegationTokenFeatureStep(conf, true) + + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create delegation tokens if needed") { + val conf = createConf(new SparkConf(false)) + val step = new DelegationTokenFeatureStep(conf, true) { + override def isSecurityEnabled: Boolean = true + override def createDelegationTokens(): Array[Byte] = Array(0x4, 0x2) + } + + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()) + assert(dtSecret.size === 1) + checkPodForTokens(step.configurePod(SparkPod.initialPod()), + dtSecret.head.getMetadata().getName()) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + private def checkPodForTokens(pod: SparkPod, dtSecretName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala + .find(_.getName() == SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, + SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_TOKEN_FILE_LOCATION)) + assert(podVolume.get.getSecret().getSecretName() === dtSecretName) + } + + private def createConf(conf: SparkConf): KubernetesConf[_] = { + KubernetesConf( + conf, + KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), + "resource-name-prefix", + "app-id", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Nil, + None) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..f58f62b0cdaa2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.Utils + +class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + test("mount hadoop config map if defined") { + val configMap = "testConfigMap" + val hadoopSpec = HadoopConfSpec(None, Some(configMap)) + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), + "resource-name-prefix", + "app-id", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Nil, + hadoopConfSpec = Some(hadoopSpec)) + + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create hadoop config map if config dir is defined") { + val confDir = Utils.createTempDir() + val confFiles = Set("core-site.xml", "hdfs-site.xml") + + confFiles.foreach { f => + Files.write("some data", new File(confDir, f), UTF_8) + } + + val hadoopSpec = HadoopConfSpec(Some(confDir.getAbsolutePath()), None) + val conf = KubernetesConf( + new SparkConf(false), + KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), + "resource-name-prefix", + "app-id", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Nil, + hadoopConfSpec = Some(hadoopSpec)) + + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + + val hadoopConfMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(hadoopConfMap.getData().keySet().asScala === confFiles) + } + + private def checkPod(pod: SparkPod): Unit = { + assert(podHasVolume(pod.pod, HADOOP_CONF_VOLUME)) + assert(containerHasVolume(pod.container, HADOOP_CONF_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_CONF_DIR)) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..e2ab971b7f38a --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + private val tmpDir = Utils.createTempDir() + + test("mount krb5 config map if defined") { + val configMap = "testConfigMap" + val conf = createConf( + new SparkConf(false).set(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP, configMap)) + val step = new KerberosConfDriverFeatureStep(conf) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), configMap) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create krb5.conf config map if local config provided") { + val krbConf = File.createTempFile("krb5", ".conf", tmpDir) + Files.write("some data", krbConf, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_KRB5_FILE, krbConf.getAbsolutePath()) + val conf = createConf(sparkConf) + + val step = new KerberosConfDriverFeatureStep(conf) + + val confMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(confMap.getData().keySet().asScala === Set(krbConf.getName())) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), confMap.getMetadata().getName()) + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("create keytab secret if client keytab file used") { + val keytab = File.createTempFile("keytab", ".bin", tmpDir) + Files.write("some data", keytab, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KEYTAB, keytab.getAbsolutePath()) + .set(PRINCIPAL, "alice") + val conf = createConf(sparkConf) + val step = new KerberosConfDriverFeatureStep(conf) + + val pod = step.configurePod(SparkPod.initialPod()) + assert(podHasVolume(pod.pod, KERBEROS_KEYTAB_VOLUME)) + assert(containerHasVolume(pod.container, KERBEROS_KEYTAB_VOLUME, KERBEROS_KEYTAB_MOUNT_POINT)) + + assert(step.getAdditionalPodSystemProperties().keys === Set(KEYTAB.key)) + + val secret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(secret.getData().keySet().asScala === Set(keytab.getName())) + } + + test("do nothing if container-local keytab used") { + val sparkConf = new SparkConf(false) + .set(KEYTAB, "local:/my.keytab") + .set(PRINCIPAL, "alice") + val conf = createConf(sparkConf) + val step = new KerberosConfDriverFeatureStep(conf) + + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + private def checkPodForKrbConf(pod: SparkPod, confMapName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala.find(_.getName() == KRB_FILE_VOLUME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, KRB_FILE_VOLUME, KRB_FILE_DIR_PATH + "/krb5.conf")) + assert(podVolume.get.getConfigMap().getName() === confMapName) + } + + private def createConf(conf: SparkConf): KubernetesConf[_] = { + KubernetesConf( + conf, + KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), + "resource-name-prefix", + "app-id", + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Map.empty, + Nil, + None) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala index f90380e30e52a..076b681be2397 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import io.fabric8.kubernetes.api.model.{Container, HasMetadata, PodBuilder, SecretBuilder} import org.mockito.Matchers @@ -63,4 +64,9 @@ object KubernetesFeaturesTestUtils { def containerHasEnvVar(container: Container, envVarName: String): Boolean = { container.getEnv.asScala.exists(envVar => envVar.getName == envVarName) } + + def filter[T: ClassTag](list: Seq[HasMetadata]): Seq[T] = { + val desired = implicitly[ClassTag[T]].runtimeClass + list.filter(_.getClass() == desired).map(_.asInstanceOf[T]).toSeq + } } From d8416cda0e23af43a86b129a4a102f37bcbdaf4c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Nov 2018 09:37:18 -0800 Subject: [PATCH 04/10] Fix YARN tests. --- .../scala/org/apache/spark/deploy/yarn/ClientSuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index b3286e8fd824e..a6f57fcdb2461 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -100,7 +100,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => val uri = new URI(entry) - if (LOCAL_SCHEME.equals(uri.getScheme())) { + if (Utils.LOCAL_SCHEME.equals(uri.getScheme())) { cp should contain (uri.getPath()) } else { cp should not contain (uri.getPath()) @@ -136,7 +136,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val expected = ADDED.split(",") .map(p => { val uri = new URI(p) - if (LOCAL_SCHEME == uri.getScheme()) { + if (Utils.LOCAL_SCHEME == uri.getScheme()) { p } else { Option(uri.getFragment()).getOrElse(new File(p).getName()) @@ -249,7 +249,7 @@ class ClientSuite extends SparkFunSuite with Matchers { any(classOf[MutableHashMap[URI, Path]]), anyBoolean(), any()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) - sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath()) + sparkConf.set(SPARK_ARCHIVE, Utils.LOCAL_SCHEME + ":" + archive.getPath()) intercept[IllegalArgumentException] { client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) } From d4a84bc32d6e07c91b72d83913a24719e411369a Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Nov 2018 12:37:45 -0800 Subject: [PATCH 05/10] Let the driver handle all the kerberos needs. --- .../features/DelegationTokenFeatureStep.scala | 114 --------------- .../KerberosConfDriverFeatureStep.scala | 132 +++++++++++++++--- .../k8s/submit/KubernetesDriverBuilder.scala | 5 +- .../k8s/KubernetesExecutorBuilder.scala | 7 +- .../DelegationTokenFeatureStepSuite.scala | 100 ------------- .../KerberosConfDriverFeatureStepSuite.scala | 63 +++++++-- .../submit/KubernetesDriverBuilderSuite.scala | 7 +- .../k8s/KubernetesExecutorBuilderSuite.scala | 9 +- 8 files changed, 166 insertions(+), 271 deletions(-) delete mode 100644 resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala delete mode 100644 resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala deleted file mode 100644 index 54bf3d5cdbf7a..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStep.scala +++ /dev/null @@ -1,114 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, SecretBuilder} -import org.apache.commons.codec.binary.Base64 -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.config._ - -/** - * Delegation token support for Spark apps on kubernetes. - * - * When preparing driver resources, this step will generate delegation tokens for the app if - * they're needed. - * - * When preparing pods, this step will mount the delegation token secret (either pre-defined, - * or generated by this step when preparing the driver). - */ -private[spark] class DelegationTokenFeatureStep(conf: KubernetesConf[_], isDriver: Boolean) - extends KubernetesFeatureConfigStep { - - private val existingSecret = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) - private val existingItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) - private val shouldCreateTokens = isDriver && !conf.sparkConf.contains(KEYTAB) && - existingSecret.isEmpty && isSecurityEnabled - - KubernetesUtils.requireBothOrNeitherDefined( - existingSecret, - existingItemKey, - "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + - " you must also specify the name of the secret", - "If a secret storing a Kerberos Delegation Token is specified you must also" + - " specify the item-key where the data is stored") - - private def dtSecretName: String = s"${conf.appResourceNamePrefix}-delegation-tokens" - - override def configurePod(pod: SparkPod): SparkPod = { - pod.transform { case pod if shouldCreateTokens | existingSecret.isDefined => - val secretName = existingSecret.getOrElse(dtSecretName) - val itemKey = existingItemKey.getOrElse(KERBEROS_SECRET_KEY) - - val podWithTokens = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withNewSecret() - .withSecretName(secretName) - .endSecret() - .endVolume() - .endSpec() - .build() - - val containerWithTokens = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) - .endVolumeMount() - .addNewEnv() - .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) - .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") - .endEnv() - .build() - - SparkPod(podWithTokens, containerWithTokens) - } - } - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { - if (shouldCreateTokens) { - val tokenData = createDelegationTokens() - Seq(new SecretBuilder() - .withNewMetadata() - .withName(dtSecretName) - .endMetadata() - .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(tokenData)) - .build()) - } else { - Nil - } - } - - // Visible for testing. - def isSecurityEnabled: Boolean = UserGroupInformation.isSecurityEnabled() - - // Visible for testing. - def createDelegationTokens(): Array[Byte] = { - val tokenManager = new HadoopDelegationTokenManager(conf.sparkConf, - SparkHadoopUtil.get.newConfiguration(conf.sparkConf)) - val creds = UserGroupInformation.getCurrentUser().getCredentials() - tokenManager.obtainDelegationTokens(creds) - SparkHadoopUtil.get.serialize(creds) - } - -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 50e4377c908b0..c3e77556a24bb 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -24,17 +24,29 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** - * Mount the kerberos configuration defined by the user in the driver pod, and make the - * user's keytab available to the driver if provided. + * Provide kerberos / service credentials to the Spark driver. + * + * There are three use cases, in order of precedence: + * + * - keytab: if a kerberos keytab is defined, it is provided to the driver, and the driver will + * manage the kerberos login and the creation of delegation tokens. + * - existing tokens: if a secret containing delegation tokens is provided, it will be mounted + * on the driver pod, and the driver will handle distribution of those tokens to executors. + * - tgt only: if Hadoop security is enabled, the local TGT will be used to create delegation + * tokens which will be provided to the driver. The driver will handle distribution of the + * tokens to executors. */ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesConf[_]) extends KubernetesFeatureConfigStep with Logging { @@ -44,6 +56,8 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon private val keytab = conf.get(org.apache.spark.internal.config.KEYTAB) private val krb5File = conf.get(KUBERNETES_KERBEROS_KRB5_FILE) private val krb5CMap = conf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + private val existingDtSecret = conf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val existingDtItemKey = conf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) KubernetesUtils.requireNandDefined( krb5File, @@ -57,12 +71,32 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon "If a Kerberos principal is specified you must also specify a Kerberos keytab", "If a Kerberos keytab is specified you must also specify a Kerberos principal") + KubernetesUtils.requireBothOrNeitherDefined( + existingDtSecret, + existingDtItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret", + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the item-key where the data is stored") + if (!hasKerberosConf) { logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + "Make sure that you have the krb5.conf locally on the driver image.") } - private val needKeytabUpload = keytab.exists(!Utils.isLocalUri(_)) + // Create delegation tokens if needed. This is a lazy val so that it's not populated + // unnecessarily. But it needs to be accessible to different methods in this class, + // since it's not clear based solely on available configuration options that delegation + // tokens are needed when other credentials are not available. + private lazy val delegationTokens: Array[Byte] = if (keytab.isEmpty && existingDtSecret.isEmpty) { + createDelegationTokens() + } else { + null + } + + private def needKeytabUpload: Boolean = keytab.exists(!Utils.isLocalUri(_)) + + private def dtSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-delegation-tokens" private def ktSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-kerberos-keytab" @@ -107,28 +141,57 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon .build() SparkPod(podWithVolume, containerWithMount) - }.transform { case pod if needKeytabUpload => - // If keytab is defined and is a submission-local file (not local: URI), then create a - // secret for it. The keytab data will be stored in this secret below. - val podWitKeytab = new PodBuilder(pod.pod) - .editOrNewSpec() - .addNewVolume() + }.transform { + case pod if needKeytabUpload => + // If keytab is defined and is a submission-local file (not local: URI), then create a + // secret for it. The keytab data will be stored in this secret below. + val podWitKeytab = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(KERBEROS_KEYTAB_VOLUME) + .withNewSecret() + .withSecretName(ktSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithKeytab = new ContainerBuilder(pod.container) + .addNewVolumeMount() .withName(KERBEROS_KEYTAB_VOLUME) - .withNewSecret() - .withSecretName(ktSecretName) - .endSecret() - .endVolume() - .endSpec() - .build() + .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) + .endVolumeMount() + .build() - val containerWithKeytab = new ContainerBuilder(pod.container) - .addNewVolumeMount() - .withName(KERBEROS_KEYTAB_VOLUME) - .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) - .endVolumeMount() - .build() + SparkPod(podWitKeytab, containerWithKeytab) - SparkPod(podWitKeytab, containerWithKeytab) + case pod if existingDtSecret.isDefined | delegationTokens != null => + val secretName = existingDtSecret.getOrElse(dtSecretName) + val itemKey = existingDtItemKey.getOrElse(KERBEROS_SECRET_KEY) + + val podWithTokens = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithTokens = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") + .endEnv() + .build() + + SparkPod(podWithTokens, containerWithTokens) } } @@ -168,7 +231,32 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon } else { Nil } + } ++ { + if (delegationTokens != null) { + Seq(new SecretBuilder() + .withNewMetadata() + .withName(dtSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(delegationTokens)) + .build()) + } else { + Nil + } } } + // Visible for testing. + def createDelegationTokens(): Array[Byte] = { + val tokenManager = new HadoopDelegationTokenManager(conf, + SparkHadoopUtil.get.newConfiguration(conf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, to + // avoid creating an unnecessary secret. + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 250c30368fd2d..2d307bb6f3283 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -52,8 +52,6 @@ private[spark] class KubernetesDriverBuilder( new HadoopConfDriverFeatureStep(_), provideKerberosConfStep: (KubernetesConf[_] => KerberosConfDriverFeatureStep) = new KerberosConfDriverFeatureStep(_), - provideDelegationTokenStep: (KubernetesConf[_] => DelegationTokenFeatureStep) = - new DelegationTokenFeatureStep(_, true), providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => PodTemplateConfigMapStep) = new PodTemplateConfigMapStep(_), @@ -85,8 +83,7 @@ private[spark] class KubernetesDriverBuilder( val otherSteps = Seq( provideHadoopConfStep(kubernetesConf), - provideKerberosConfStep(kubernetesConf), - provideDelegationTokenStep(kubernetesConf)) + provideKerberosConfStep(kubernetesConf)) val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ Seq(driverCommandStep) ++ diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 0e90e548061b9..d6ea6ee88d1b5 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -41,8 +41,6 @@ private[spark] class KubernetesExecutorBuilder( provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] => MountVolumesFeatureStep) = new MountVolumesFeatureStep(_), - provideDelegationTokenStep: (KubernetesConf[_] => DelegationTokenFeatureStep) = - new DelegationTokenFeatureStep(_, false), provideInitialPod: () => SparkPod = SparkPod.initialPod) { def buildFromFeatures( @@ -59,14 +57,11 @@ private[spark] class KubernetesExecutorBuilder( Seq(provideVolumesStep(kubernetesConf)) } else Nil - val dtSecretStep = Seq(provideDelegationTokenStep(kubernetesConf)) - val allFeatures: Seq[KubernetesFeatureConfigStep] = baseFeatures ++ secretFeature ++ secretEnvFeature ++ - volumesFeature ++ - dtSecretStep + volumesFeature var executorPod = provideInitialPod() for (feature <- allFeatures) { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala deleted file mode 100644 index 77f1a6b53e8b1..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DelegationTokenFeatureStepSuite.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.Secret - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource -import org.apache.spark.internal.config._ - -class DelegationTokenFeatureStepSuite extends SparkFunSuite { - - import KubernetesFeaturesTestUtils._ - import SecretVolumeUtils._ - - test("mount delegation tokens if provided") { - val dtSecret = "tokenSecret" - val sparkConf = new SparkConf(false) - .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, dtSecret) - .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtokens") - val conf = createConf(sparkConf) - val step = new DelegationTokenFeatureStep(conf, true) - - checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret) - assert(step.getAdditionalPodSystemProperties().isEmpty) - assert(step.getAdditionalKubernetesResources().isEmpty) - } - - test("do nothing if a keytab is provided") { - val sparkConf = new SparkConf(false).set(KEYTAB, "fakeKeytab") - val conf = createConf(sparkConf) - val step = new DelegationTokenFeatureStep(conf, true) - - val initial = SparkPod.initialPod() - assert(step.configurePod(initial) === initial) - - assert(step.getAdditionalPodSystemProperties().isEmpty) - assert(step.getAdditionalKubernetesResources().isEmpty) - } - - test("create delegation tokens if needed") { - val conf = createConf(new SparkConf(false)) - val step = new DelegationTokenFeatureStep(conf, true) { - override def isSecurityEnabled: Boolean = true - override def createDelegationTokens(): Array[Byte] = Array(0x4, 0x2) - } - - val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()) - assert(dtSecret.size === 1) - checkPodForTokens(step.configurePod(SparkPod.initialPod()), - dtSecret.head.getMetadata().getName()) - - assert(step.getAdditionalPodSystemProperties().isEmpty) - } - - private def checkPodForTokens(pod: SparkPod, dtSecretName: String): Unit = { - val podVolume = pod.pod.getSpec().getVolumes().asScala - .find(_.getName() == SPARK_APP_HADOOP_SECRET_VOLUME_NAME) - assert(podVolume.isDefined) - assert(containerHasVolume(pod.container, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, - SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) - assert(containerHasEnvVar(pod.container, ENV_HADOOP_TOKEN_FILE_LOCATION)) - assert(podVolume.get.getSecret().getSecretName() === dtSecretName) - } - - private def createConf(conf: SparkConf): KubernetesConf[_] = { - KubernetesConf( - conf, - KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - None) - } - -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala index e2ab971b7f38a..a0d300ffb2a87 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} +import org.mockito.Mockito._ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ @@ -41,13 +42,12 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { test("mount krb5 config map if defined") { val configMap = "testConfigMap" - val conf = createConf( + val step = createStep( new SparkConf(false).set(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP, configMap)) - val step = new KerberosConfDriverFeatureStep(conf) checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), configMap) assert(step.getAdditionalPodSystemProperties().isEmpty) - assert(step.getAdditionalKubernetesResources().isEmpty) + assert(filter[ConfigMap](step.getAdditionalKubernetesResources()).isEmpty) } test("create krb5.conf config map if local config provided") { @@ -56,9 +56,7 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(KUBERNETES_KERBEROS_KRB5_FILE, krbConf.getAbsolutePath()) - val conf = createConf(sparkConf) - - val step = new KerberosConfDriverFeatureStep(conf) + val step = createStep(sparkConf) val confMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head assert(confMap.getData().keySet().asScala === Set(krbConf.getName())) @@ -74,8 +72,7 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(KEYTAB, keytab.getAbsolutePath()) .set(PRINCIPAL, "alice") - val conf = createConf(sparkConf) - val step = new KerberosConfDriverFeatureStep(conf) + val step = createStep(sparkConf) val pod = step.configurePod(SparkPod.initialPod()) assert(podHasVolume(pod.pod, KERBEROS_KEYTAB_VOLUME)) @@ -91,9 +88,40 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { val sparkConf = new SparkConf(false) .set(KEYTAB, "local:/my.keytab") .set(PRINCIPAL, "alice") - val conf = createConf(sparkConf) - val step = new KerberosConfDriverFeatureStep(conf) + val step = createStep(sparkConf) + + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("mount delegation tokens if provided") { + val dtSecret = "tokenSecret" + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, dtSecret) + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtokens") + val step = createStep(sparkConf) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + test("create delegation tokens if needed") { + val step = spy(createStep(new SparkConf(false))) + doReturn(Array[Byte](0x4, 0x2)).when(step).createDelegationTokens() + + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()) + assert(dtSecret.size === 1) + checkPodForTokens(step.configurePod(SparkPod.initialPod()), + dtSecret.head.getMetadata().getName()) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("do nothing if no config and no tokens") { + val step = createStep(new SparkConf(false)) val initial = SparkPod.initialPod() assert(step.configurePod(initial) === initial) assert(step.getAdditionalPodSystemProperties().isEmpty) @@ -107,8 +135,18 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { assert(podVolume.get.getConfigMap().getName() === confMapName) } - private def createConf(conf: SparkConf): KubernetesConf[_] = { - KubernetesConf( + private def checkPodForTokens(pod: SparkPod, dtSecretName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala + .find(_.getName() == SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, + SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_TOKEN_FILE_LOCATION)) + assert(podVolume.get.getSecret().getSecretName() === dtSecretName) + } + + private def createStep(conf: SparkConf): KerberosConfDriverFeatureStep = { + val kconf = KubernetesConf( conf, KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), "resource-name-prefix", @@ -120,6 +158,7 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { Map.empty, Nil, None) + new KerberosConfDriverFeatureStep(kconf) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 2cc18077cb695..65c002c541d31 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -67,9 +67,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { private val kerberosStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( KERBEROS_CONF_STEP_TYPE, classOf[KerberosConfDriverFeatureStep]) - private val delegationTokenStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - DELEGATION_TOKEN_CONF_STEP_TYPE, classOf[DelegationTokenFeatureStep]) - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) @@ -83,8 +80,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { LOCAL_DIRS_STEP_TYPE, DRIVER_CMD_STEP_TYPE, HADOOP_CONF_STEP_TYPE, - KERBEROS_CONF_STEP_TYPE, - DELEGATION_TOKEN_CONF_STEP_TYPE) + KERBEROS_CONF_STEP_TYPE) private val builderUnderTest: KubernetesDriverBuilder = new KubernetesDriverBuilder( @@ -98,7 +94,6 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { _ => driverCommandStep, _ => hadoopConfStep, _ => kerberosStep, - _ => delegationTokenStep, _ => templateVolumeStep) test("Apply fundamental steps all the time.") { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index 19a5569ab7119..96e158fdc8a91 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -44,22 +44,17 @@ class KubernetesExecutorBuilderSuite extends SparkFunSuite { ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - private val dtokenConfStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - DELEGATION_TOKEN_CONF_STEP_TYPE, classOf[DelegationTokenFeatureStep]) private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) - private val ALWAYS_ON_STEPS = Seq(BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - DELEGATION_TOKEN_CONF_STEP_TYPE) + private val ALWAYS_ON_STEPS = Seq(BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) private val builderUnderTest = new KubernetesExecutorBuilder( _ => basicFeatureStep, _ => mountSecretsStep, _ => envSecretsStep, _ => localDirsStep, - _ => mountVolumesStep, - _ => dtokenConfStep) + _ => mountVolumesStep) test("Basic steps are consistently applied.") { val conf = KubernetesConf( From 7e9b3a2218359846176136cc98d63eb841d97f15 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Nov 2018 14:54:57 -0800 Subject: [PATCH 06/10] Test dt secret content. --- .../KerberosConfDriverFeatureStepSuite.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala index a0d300ffb2a87..5028a88410a45 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} +import org.apache.commons.codec.binary.Base64 import org.mockito.Mockito._ import org.apache.spark.{SparkConf, SparkFunSuite} @@ -109,13 +110,14 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { } test("create delegation tokens if needed") { + val tokens = Array[Byte](0x4, 0x2) val step = spy(createStep(new SparkConf(false))) - doReturn(Array[Byte](0x4, 0x2)).when(step).createDelegationTokens() + doReturn(tokens).when(step).createDelegationTokens() - val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()) - assert(dtSecret.size === 1) - checkPodForTokens(step.configurePod(SparkPod.initialPod()), - dtSecret.head.getMetadata().getName()) + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(dtSecret.getData().get(KERBEROS_SECRET_KEY) === Base64.encodeBase64String(tokens)) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret.getMetadata().getName()) assert(step.getAdditionalPodSystemProperties().isEmpty) } From 78b76a8ace35b77a6f005dfb26a6e2fd836c1ff4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 14 Nov 2018 16:06:15 -0800 Subject: [PATCH 07/10] Don't mock. --- .../KerberosConfDriverFeatureStep.scala | 26 +++++++-------- .../KerberosConfDriverFeatureStepSuite.scala | 32 ++++++++++++++----- 2 files changed, 35 insertions(+), 23 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index c3e77556a24bb..25ae19dbaa0a2 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -89,7 +89,17 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon // since it's not clear based solely on available configuration options that delegation // tokens are needed when other credentials are not available. private lazy val delegationTokens: Array[Byte] = if (keytab.isEmpty && existingDtSecret.isEmpty) { - createDelegationTokens() + val tokenManager = new HadoopDelegationTokenManager(conf, + SparkHadoopUtil.get.newConfiguration(conf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, to + // avoid creating an unnecessary secret. + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } } else { null } @@ -245,18 +255,4 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesCon } } - // Visible for testing. - def createDelegationTokens(): Array[Byte] = { - val tokenManager = new HadoopDelegationTokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) - val creds = UserGroupInformation.getCurrentUser().getCredentials() - tokenManager.obtainDelegationTokens(creds) - // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, to - // avoid creating an unnecessary secret. - if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { - SparkHadoopUtil.get.serialize(creds) - } else { - null - } - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala index 5028a88410a45..9169ba4842290 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -18,15 +18,18 @@ package org.apache.spark.deploy.k8s.features import java.io.File import java.nio.charset.StandardCharsets.UTF_8 +import java.security.PrivilegedExceptionAction import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} import org.apache.commons.codec.binary.Base64 -import org.mockito.Mockito._ +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -110,16 +113,29 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { } test("create delegation tokens if needed") { - val tokens = Array[Byte](0x4, 0x2) - val step = spy(createStep(new SparkConf(false))) - doReturn(tokens).when(step).createDelegationTokens() + // Since HadoopDelegationTokenManager does not create any tokens without proper configs and + // services, start with a test user that already has some tokens that will just be piped + // through to the driver. + val testUser = UserGroupInformation.createUserForTesting("k8s", Array()) + testUser.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val creds = testUser.getCredentials() + creds.addSecretKey(new Text("K8S_TEST_KEY"), Array[Byte](0x4, 0x2)) + testUser.addCredentials(creds) - val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()).head - assert(dtSecret.getData().get(KERBEROS_SECRET_KEY) === Base64.encodeBase64String(tokens)) + val tokens = SparkHadoopUtil.get.serialize(creds) - checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret.getMetadata().getName()) + val step = createStep(new SparkConf(false)) - assert(step.getAdditionalPodSystemProperties().isEmpty) + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(dtSecret.getData().get(KERBEROS_SECRET_KEY) === Base64.encodeBase64String(tokens)) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), + dtSecret.getMetadata().getName()) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + }) } test("do nothing if no config and no tokens") { From 05333ea11bc50ade8f3ba8e1d2116bad9d3d7c89 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 3 Dec 2018 13:23:56 -0800 Subject: [PATCH 08/10] Post-merge fixes. --- .../spark/deploy/k8s/KubernetesConf.scala | 4 -- .../HadoopConfDriverFeatureStep.scala | 27 +++++++---- .../KerberosConfDriverFeatureStep.scala | 47 ++++++++++--------- .../k8s/submit/KubernetesDriverBuilder.scala | 9 ++-- .../HadoopConfDriverFeatureStepSuite.scala | 35 +++----------- .../KerberosConfDriverFeatureStepSuite.scala | 13 +---- .../submit/KubernetesDriverBuilderSuite.scala | 19 ++++---- 7 files changed, 66 insertions(+), 88 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index a06c21b47f15e..6febad981af56 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -42,10 +42,6 @@ private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { def appName: String = get("spark.app.name", "spark") - def hadoopConfigMapName: String = s"$resourceNamePrefix-hadoop-config" - - def krbConfigMapName: String = s"$resourceNamePrefix-krb5-file" - def namespace: String = get(KUBERNETES_NAMESPACE) def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala index 74f933fce2c29..d602ed5481e65 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -24,7 +24,7 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import io.fabric8.kubernetes.api.model._ -import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ @@ -32,12 +32,17 @@ import org.apache.spark.deploy.k8s.Constants._ * Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration * directory - on the driver pod. */ -private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) +private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { - private val confSpec = conf.hadoopConfSpec - private val confDir = confSpec.flatMap(_.hadoopConfDir) - private val confMap = confSpec.flatMap(_.hadoopConfigMapName) + private val confDir = Option(conf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) + private val existingConfMap = conf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + + KubernetesUtils.requireNandDefined( + confDir, + existingConfMap, + "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + + "as the creation of an additional ConfigMap, when one is already specified is extraneous") private lazy val confFiles: Seq[File] = { val dir = new File(confDir.get) @@ -48,8 +53,12 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) } } + private def newConfigMapName: String = s"${conf.resourceNamePrefix}-hadoop-config" + + private def hasHadoopConf: Boolean = confDir.isDefined || existingConfMap.isDefined + override def configurePod(original: SparkPod): SparkPod = { - original.transform { case pod if confSpec.isDefined => + original.transform { case pod if hasHadoopConf => val confVolume = if (confDir.isDefined) { val keyPaths = confFiles.map { file => new KeyToPathBuilder() @@ -60,7 +69,7 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) new VolumeBuilder() .withName(HADOOP_CONF_VOLUME) .withNewConfigMap() - .withName(conf.hadoopConfigMapName) + .withName(newConfigMapName) .withItems(keyPaths.asJava) .endConfigMap() .build() @@ -68,7 +77,7 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) new VolumeBuilder() .withName(HADOOP_CONF_VOLUME) .withNewConfigMap() - .withName(confMap.get) + .withName(existingConfMap.get) .endConfigMap() .build() } @@ -103,7 +112,7 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf[_]) Seq(new ConfigMapBuilder() .withNewMetadata() - .withName(conf.hadoopConfigMapName) + .withName(newConfigMapName) .endMetadata() .addToData(fileMap) .build()) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala index 3b7268026207b..721d7e97b21f8 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, Spark import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.util.Utils @@ -48,7 +49,7 @@ import org.apache.spark.util.Utils * tokens to executors. */ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) - extends KubernetesFeatureConfigStep { + extends KubernetesFeatureConfigStep with Logging { private val principal = kubernetesConf.get(org.apache.spark.internal.config.PRINCIPAL) private val keytab = kubernetesConf.get(org.apache.spark.internal.config.KEYTAB) @@ -71,8 +72,8 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri "If a Kerberos keytab is specified you must also specify a Kerberos principal") KubernetesUtils.requireBothOrNeitherDefined( - existingDtSecret, - existingDtItemKey, + existingSecretName, + existingSecretItemKey, "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + " you must also specify the name of the secret", "If a secret storing a Kerberos Delegation Token is specified you must also" + @@ -87,30 +88,34 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri // unnecessarily. But it needs to be accessible to different methods in this class, // since it's not clear based solely on available configuration options that delegation // tokens are needed when other credentials are not available. - private lazy val delegationTokens: Array[Byte] = if (keytab.isEmpty && existingDtSecret.isEmpty) { - val tokenManager = new HadoopDelegationTokenManager(conf, - SparkHadoopUtil.get.newConfiguration(conf)) - val creds = UserGroupInformation.getCurrentUser().getCredentials() - tokenManager.obtainDelegationTokens(creds) - // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, to - // avoid creating an unnecessary secret. - if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { - SparkHadoopUtil.get.serialize(creds) + private lazy val delegationTokens: Array[Byte] = { + if (keytab.isEmpty && existingSecretName.isEmpty) { + val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, + SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf)) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, + // to avoid creating an unnecessary secret. + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } } else { null } - } else { - null } private def needKeytabUpload: Boolean = keytab.exists(!Utils.isLocalUri(_)) - private def dtSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-delegation-tokens" + private def dtSecretName: String = s"${kubernetesConf.resourceNamePrefix}-delegation-tokens" - private def ktSecretName: String = s"${kubernetesConf.appResourceNamePrefix}-kerberos-keytab" + private def ktSecretName: String = s"${kubernetesConf.resourceNamePrefix}-kerberos-keytab" private def hasKerberosConf: Boolean = krb5CMap.isDefined | krb5File.isDefined + private def newConfigMapName: String = s"${kubernetesConf.resourceNamePrefix}-krb5-file" + override def configurePod(original: SparkPod): SparkPod = { original.transform { case pod if hasKerberosConf => val configMapVolume = if (krb5CMap.isDefined) { @@ -125,7 +130,7 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri new VolumeBuilder() .withName(KRB_FILE_VOLUME) .withNewConfigMap() - .withName(kubernetesConf.krbConfigMapName) + .withName(newConfigMapName) .withItems(new KeyToPathBuilder() .withKey(krb5Conf.getName()) .withPath(krb5Conf.getName()) @@ -174,9 +179,9 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri SparkPod(podWitKeytab, containerWithKeytab) - case pod if existingDtSecret.isDefined | delegationTokens != null => - val secretName = existingDtSecret.getOrElse(dtSecretName) - val itemKey = existingDtItemKey.getOrElse(KERBEROS_SECRET_KEY) + case pod if existingSecretName.isDefined | delegationTokens != null => + val secretName = existingSecretName.getOrElse(dtSecretName) + val itemKey = existingSecretItemKey.getOrElse(KERBEROS_SECRET_KEY) val podWithTokens = new PodBuilder(pod.pod) .editOrNewSpec() @@ -221,7 +226,7 @@ private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDri val file = new File(path) new ConfigMapBuilder() .withNewMetadata() - .withName(kubernetesConf.krbConfigMapName) + .withName(newConfigMapName) .endMetadata() .addToData( Map(file.getName() -> Files.toString(file, StandardCharsets.UTF_8)).asJava) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 74fad617b0423..7a4bf08fc3678 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -41,13 +41,12 @@ private[spark] class KubernetesDriverBuilder( new MountVolumesFeatureStep(_), provideDriverCommandStep: (KubernetesDriverConf => DriverCommandFeatureStep) = new DriverCommandFeatureStep(_), - provideHadoopConfStep: (KubernetesConf[_] => HadoopConfDriverFeatureStep) = + provideHadoopConfStep: (KubernetesConf => HadoopConfDriverFeatureStep) = new HadoopConfDriverFeatureStep(_), - provideKerberosConfStep: (KubernetesConf[_] => KerberosConfDriverFeatureStep) = + provideKerberosConfStep: (KubernetesDriverConf => KerberosConfDriverFeatureStep) = new KerberosConfDriverFeatureStep(_), - providePodTemplateConfigMapStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => PodTemplateConfigMapStep) = - new PodTemplateConfigMapStep(_), + providePodTemplateConfigMapStep: (KubernetesConf => PodTemplateConfigMapStep) = + new PodTemplateConfigMapStep(_), provideInitialPod: () => SparkPod = () => SparkPod.initialPod) { def buildFromFeatures(kubernetesConf: KubernetesDriverConf): KubernetesDriverSpec = { diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala index f58f62b0cdaa2..e1c01dbdc7358 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit.JavaMainAppResource -import org.apache.spark.util.Utils +import org.apache.spark.util.{SparkConfWithEnv, Utils} class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { @@ -36,21 +36,9 @@ class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { import SecretVolumeUtils._ test("mount hadoop config map if defined") { - val configMap = "testConfigMap" - val hadoopSpec = HadoopConfSpec(None, Some(configMap)) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some(hadoopSpec)) - + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_HADOOP_CONF_CONFIG_MAP, "testConfigMap") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val step = new HadoopConfDriverFeatureStep(conf) checkPod(step.configurePod(SparkPod.initialPod())) assert(step.getAdditionalKubernetesResources().isEmpty) @@ -64,19 +52,8 @@ class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { Files.write("some data", new File(confDir, f), UTF_8) } - val hadoopSpec = HadoopConfSpec(Some(confDir.getAbsolutePath()), None) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - hadoopConfSpec = Some(hadoopSpec)) + val sparkConf = new SparkConfWithEnv(Map(ENV_HADOOP_CONF_DIR -> confDir.getAbsolutePath())) + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val step = new HadoopConfDriverFeatureStep(conf) checkPod(step.configurePod(SparkPod.initialPod())) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala index 9169ba4842290..41ca3a94ce7a7 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -164,18 +164,7 @@ class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { } private def createStep(conf: SparkConf): KerberosConfDriverFeatureStep = { - val kconf = KubernetesConf( - conf, - KubernetesDriverSpecificConf(JavaMainAppResource(None), "class", "name", Nil), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - None) + val kconf = KubernetesTestConf.createDriverConf(sparkConf = conf) new KerberosConfDriverFeatureStep(kconf) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 50fd24067601d..07371cd3d1784 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -121,13 +121,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE, - DRIVER_CMD_STEP_TYPE, - HADOOP_GLOBAL_STEP_TYPE) + MOUNT_VOLUMES_STEP_TYPE) } test("Apply volumes step if a mount subpath is present.") { @@ -138,6 +132,15 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { false, KubernetesHostPathVolumeConf("/path")) val conf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeSpec)) + validateStepTypesApplied( + builderUnderTest.buildFromFeatures(conf), + MOUNT_VOLUMES_STEP_TYPE) + } + + test("Apply template volume step if executor template is present.") { + val sparkConf = new SparkConf(false) + .set(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, "filename") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) validateStepTypesApplied( builderUnderTest.buildFromFeatures(conf), TEMPLATE_VOLUME_STEP_TYPE) @@ -149,7 +152,7 @@ class KubernetesDriverBuilderSuite extends SparkFunSuite { val addedProperties = resolvedSpec.systemProperties .filter { case (k, _) => !k.startsWith("spark.") } .toMap - assert(resolvedSpec.systemProperties.keys === validSteps) + assert(addedProperties.keys.toSet === validSteps.toSet) validSteps.foreach { stepType => assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) assert(resolvedSpec.driverKubernetesResources.containsSlice( From cce3f1dd32c204f5c57b3db3a79d49895512f0a5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 12 Dec 2018 12:44:34 -0800 Subject: [PATCH 09/10] Post-merge fixes. --- .../scheduler/cluster/k8s/KubernetesExecutorBuilder.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 0b74966fe8685..48aa2c56d4d69 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -44,10 +44,7 @@ private[spark] class KubernetesExecutorBuilder { new MountSecretsFeatureStep(conf), new EnvSecretsFeatureStep(conf), new LocalDirsFeatureStep(conf), - new MountVolumesFeatureStep(conf), - new HadoopConfExecutorFeatureStep(conf), - new KerberosConfExecutorFeatureStep(conf), - new HadoopSparkUserExecutorFeatureStep(conf)) + new MountVolumesFeatureStep(conf)) features.foldLeft(initialPod) { case (pod, feature) => feature.configurePod(pod) } } From ccb39560298a5e54f144b8ba2a43d950289ccf34 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 13 Dec 2018 18:08:00 -0800 Subject: [PATCH 10/10] Who's a bad merge? --- .../apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index d2c0ced9fa2f4..57e4060bc85b9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -46,6 +46,7 @@ private[spark] class KubernetesDriverBuilder { new LocalDirsFeatureStep(conf), new MountVolumesFeatureStep(conf), new DriverCommandFeatureStep(conf), + new HadoopConfDriverFeatureStep(conf), new KerberosConfDriverFeatureStep(conf), new PodTemplateConfigMapStep(conf))