From ce63a9b6399176b8fa2c59c1004d796ef77c3d71 Mon Sep 17 00:00:00 2001 From: "Dr. Stefan Schimanski" Date: Wed, 10 Feb 2016 18:09:46 +0100 Subject: [PATCH 01/15] [Mesosphere SPARK-126] Move YarnSparkHadoopUtil token helpers into the generic SparkHadoopUtil class --- .../apache/spark/deploy/SparkHadoopUtil.scala | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index bae7a3f307f5..33e4a4dd84f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.{JobConf, Master} import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier @@ -138,6 +138,36 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } + def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the credentials. + */ + def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials, + renewer: Option[String] = None + ): Unit = { + if (UserGroupInformation.isSecurityEnabled()) { + val delegTokenRenewer = renewer.getOrElse(getTokenRenewer(conf)) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logInfo("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } + /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will From 75d849a494519a5af97bf22df7676b336746ac92 Mon Sep 17 00:00:00 2001 From: "Dr. Stefan Schimanski" Date: Wed, 10 Feb 2016 18:11:20 +0100 Subject: [PATCH 02/15] [Mesosphere SPARK-126] Add Mesos Kerberos support --- bin/spark-class | 13 ++ .../org/apache/spark/deploy/SparkSubmit.scala | 35 ++++ .../spark/deploy/SparkSubmitArguments.scala | 17 +- .../CoarseGrainedExecutorBackend.scala | 28 +++ .../cluster/CoarseGrainedClusterMessage.scala | 2 + .../CoarseGrainedSchedulerBackend.scala | 10 + .../cluster/mesos/MesosKerberosHandler.scala | 191 ++++++++++++++++++ .../launcher/SparkSubmitOptionParser.java | 4 + .../cluster/mesos/MesosClusterScheduler.scala | 14 ++ .../MesosCoarseGrainedSchedulerBackend.scala | 34 ++++ .../cluster/mesos/MesosSchedulerUtils.scala | 15 ++ 11 files changed, 356 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala diff --git a/bin/spark-class b/bin/spark-class index 77ea40cc3794..87aecc05e5e0 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -55,6 +55,19 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then LAUNCH_CLASSPATH="${SPARK_HOME}/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" fi +# copy krb5.conf into a temporary file and tell java.security about it +if [ -n "${SPARK_MESOS_KRB5_CONF_BASE64}" ]; then + if base64 --help | grep -q GNU; then + BASE64_D="base64 -d" # GNU + else + BASE64_D="base64 -D" # BSD + fi + KRB5_TMP_FILE=$(mktemp -t krb5-XXXXX.conf) + echo "Writing krb5.conf to $KRB5_TMP_FILE" 1>&2 + echo "${SPARK_MESOS_KRB5_CONF_BASE64}" | ${BASE64_D} > ${KRB5_TMP_FILE} + export SPARK_JAVA_OPTS="${SPARK_JAVA_OPTS} -Djava.security.krb5.conf=${KRB5_TMP_FILE}" +fi + # For tests if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR 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 77005aa9040b..445e31cb875e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -20,8 +20,10 @@ package org.apache.spark.deploy import java.io.{File, IOException} import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL +import java.nio.file.{Files, Paths} import java.security.PrivilegedExceptionAction import java.text.ParseException +import javax.xml.bind.DatatypeConverter import scala.annotation.tailrec import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -463,6 +465,10 @@ object SparkSubmit extends CommandLineUtils { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), + // Mesos cluster only + OptionAssigner(args.principal, MESOS, CLUSTER), + OptionAssigner(args.keytab, MESOS, CLUSTER), + // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.cores"), @@ -564,7 +570,36 @@ object SparkSubmit extends CommandLineUtils { // properties and then loaded by SparkConf sysProps.put("spark.yarn.keytab", args.keytab) sysProps.put("spark.yarn.principal", args.principal) + } + } + } + + if (clusterManager == MESOS && args.principal != null) { + sysProps.put("spark.yarn.principal", args.principal) + + // set principal used to renew tokens. We use the job token for now + // because we have its keytab here and in the scheduler already. + sysProps.put("spark.hadoop.yarn.resourcemanager.principal", args.principal) + + if (!args.sparkProperties.contains("spark.mesos.kerberos.keytabBase64")) { + require(args.keytab != null, "Keytab must be specified when principal is specified.") + if (args.keytab != null && !new File(args.keytab).exists()) { + throw new SparkException(s"Keytab file: ${args.keytab} does not exist") + } + + val path: String = args.keytab + val key: String = s"spark.mesos.kerberos.keytabBase64" + + // load keytab or tgt and pass to the driver via spark property + val bytes = Files.readAllBytes(Paths.get(path)) + sysProps.put(key, DatatypeConverter.printBase64Binary(bytes)) + // set auth mechanism to Kerberos and login locally if in CLIENT mode. In cluster mode + // no local Kerberos setup is necessary. + if (deployMode == CLIENT) { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(new SparkConf()) + hadoopConf.set("hadoop.security.authentication", "Kerberos") + UserGroupInformation.setConfiguration(hadoopConf) UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 0144fd1056ba..6a68382b8709 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -551,6 +551,16 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, | or all available cores on the worker in standalone mode) | + | YARN and Mesos only: + | --principal PRINCIPAL Principal to be used to login to KDC, while running on + | secure HDFS. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. For renewing the login tickets + | and the delegation tokens periodically, this keytab is copied + | - to the node running the Application Master via the Secure + | Distributed Cache on YARN, + | - or to the dispatcher and the driver on Mesos. + | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). @@ -560,13 +570,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | executors will be at least NUM. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. - | --principal PRINCIPAL Principal to be used to login to KDC, while running on - | secure HDFS. - | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. This keytab will be copied to - | the node running the Application Master via the Secure - | Distributed Cache, for renewing the login tickets and the - | delegation tokens periodically. """.stripMargin ) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b2b26ee107c0..1d1c632cc25b 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,11 +21,14 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean +import javax.xml.bind.DatatypeConverter import scala.collection.mutable import scala.util.{Failure, Success} import scala.util.control.NonFatal +import org.apache.hadoop.security.UserGroupInformation + import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil @@ -123,6 +126,10 @@ private[spark] class CoarseGrainedExecutorBackend( executor.stop() } }.start() + + case UpdateDelegationTokens(tokens) => + logInfo(s"Got UpdateDelegationTokens message with ${tokens.length} bytes") + CoarseGrainedExecutorBackend.addDelegationTokens(tokens, env.conf) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -174,6 +181,21 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { + private def addDelegationTokens(tokens: Array[Byte], sparkConf: SparkConf) { + logInfo(s"Found delegation tokens of ${tokens.length} bytes") + + // configure to use tokens for HDFS login + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + hadoopConf.set("hadoop.security.authentication", "Token") + UserGroupInformation.setConfiguration(hadoopConf) + + // decode tokens and add them to the credentials + val creds = UserGroupInformation.getCurrentUser.getCredentials + val tokensBuf = new java.io.ByteArrayInputStream(tokens) + creds.readTokenStorageStream(new java.io.DataInputStream(tokensBuf)) + UserGroupInformation.getCurrentUser.addCredentials(creds) + } + private def run( driverUrl: String, executorId: String, @@ -220,6 +242,12 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { SparkHadoopUtil.get.startCredentialUpdater(driverConf) } + if (driverConf.contains("spark.mesos.kerberos.hdfsDelegationTokens")) { + val value = driverConf.get("spark.mesos.kerberos.hdfsDelegationTokens") + val tokens = DatatypeConverter.parseBase64Binary(value) + addDelegationTokens(tokens, driverConf) + } + val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, cfg.ioEncryptionKey, isLocal = false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 6b49bd699a13..cc90132bbc88 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -53,6 +53,8 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage with RegisterExecutorResponse + case class UpdateDelegationTokens(tokens: Array[Byte]) extends CoarseGrainedClusterMessage + // Executors to driver case class RegisterExecutor( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4eedaaea6119..1f96223673eb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -146,6 +146,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.getExecutorsAliveOnHost(host).foreach { exec => killExecutors(exec.toSeq, replace = true, force = true) } + + case UpdateDelegationTokens(tokens) => + logInfo("Asking each executor to update HDFS delegation tokens") + for ((_, executorData) <- executorDataMap) { + executorData.executorEndpoint.send(UpdateDelegationTokens(tokens)) + } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -476,6 +482,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Request an additional number of executors from the cluster manager. + * * @return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { @@ -501,6 +508,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. + * * @param numExecutors The total number of executors we'd like to have. The cluster manager * shouldn't kill any running executor to reach this number, but, * if all existing executors were to die, this is the number of executors @@ -615,6 +623,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Kill the given list of executors through the cluster manager. + * * @return whether the kill request is acknowledged. */ protected def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = @@ -622,6 +631,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Request that the cluster manager kill all executors on a given host. + * * @return whether the kill request is acknowledged. */ final override def killExecutorsOnHost(host: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala new file mode 100644 index 000000000000..06e1d36b4143 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import java.io.{ByteArrayInputStream, DataInputStream} +import java.nio.file.Files +import java.nio.file.attribute.PosixFilePermissions +import java.security.PrivilegedExceptionAction +import javax.xml.bind.DatatypeConverter + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging + + +/** + * DelegationTokenBroadcaster is a callback interface to broadcast new tokens + * to the executors. + */ +trait DelegationTokenBroadcaster { + def broadcastDelegationTokens(tokens: Array[Byte]): Unit +} + +/** + * MesosKerberosHandler implements the Kerberos logic for Mesos + */ +private[spark] +class MesosKerberosHandler(conf: SparkConf, + principal: String, + broadcaster: DelegationTokenBroadcaster) + extends Object with Logging { + + @volatile private var renewalCredentials: Credentials = null + @volatile private var stopRenewal = false + var renewalThread: Thread = null + + def start(): Unit = { + logInfo("Starting delegation token renewer") + renewalThread = new Thread(new Runnable { + def run() { + renewLoop() + } + }) + renewalThread.start() + } + + def stop(): Unit = { + logWarning("Stopping delegation token renewer") + stopRenewal = true + if (renewalThread != null) { + renewalThread.interrupt() + } + } + + def createHDFSDelegationTokens: Array[Byte] = { + // get keytab or tgt, and login + val keytab64 = conf.get("spark.mesos.kerberos.keytabBase64", null) + val tgt64 = conf.get("spark.mesos.kerberos.tgtBase64", null) + require(keytab64 != null || tgt64 != null, "keytab or tgt required") + require(keytab64 == null || tgt64 == null, "keytab and tgt cannot be used at the same time") + + val mode = if (keytab64 != null) "keytab" else "tgt" + logInfo(s"Logging in as $principal with $mode to retrieve HDFS delegation tokens") + + // write keytab or tgt into a temporary file + val bytes = DatatypeConverter.parseBase64Binary(if (keytab64 != null) keytab64 else tgt64) + val kerberosSecretFile = Files.createTempFile("spark-mesos-kerberos-token", ".tmp", + PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rw-------"))) + kerberosSecretFile.toFile.deleteOnExit() // just to be sure + Files.write(kerberosSecretFile, bytes) + + // login + try { + // login with _new_ user in order to start without any token (necessary to make sure that + // new tokens are really downloaded, even when not yet expired) + val ugi = if (keytab64 != null) { + UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, kerberosSecretFile.toString) + } + else { + UserGroupInformation.getUGIFromTicketCache(kerberosSecretFile.toString, principal) + } + + // get tokens + val ugiCreds = getHDFSDelegationTokens(ugi) + logInfo(s"Got ${ugiCreds.numberOfTokens()} HDFS delegation tokens") + + // write tokens into a memory file to transfer it to the executors + val tokenBuf = new java.io.ByteArrayOutputStream(1024 * 1024) + ugiCreds.writeTokenStorageToStream(new java.io.DataOutputStream(tokenBuf)) + logDebug(s"Wrote ${tokenBuf.size()} bytes of token data") + + // store the renewal credentials, needed to get the waiting time for + // the next renewal + renewalCredentials = ugiCreds + + // make new ugi active + UserGroupInformation.setLoginUser(ugi) + + tokenBuf.toByteArray + } + finally { + kerberosSecretFile.toFile.delete() + } + } + + private def getHDFSDelegationTokens(ugi: UserGroupInformation): Credentials = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val namenodes = Option(hadoopConf.get("dfs.ha.namenodes.hdfs", null)). + map(_.split(",")).getOrElse(Array[String]()). + flatMap(id => Option(hadoopConf.get(s"dfs.namenode.rpc-address.hdfs.$id", null))). + map(hostPort => new Path(s"hdfs://$hostPort")). + toSet + logInfo(s"Found these HDFS namenodes: $namenodes") + val ugiCreds = ugi.getCredentials + ugi.doAs(new PrivilegedExceptionAction[Unit] { + override def run() = { + // use the job principal itself to renew the tokens + SparkHadoopUtil.get.obtainTokensForNamenodes( + namenodes, hadoopConf, ugiCreds, Some(principal) + ) + } + }) + ugiCreds + } + + private def renewLoop(): Unit = { + while (!stopRenewal) { + try { + val msLeft = getHDFSTokenRenewalInterval(renewalCredentials) + val msWait = Math.max(msLeft / 2, 30 * 1000) + logInfo(s"Waiting ${msWait / 1000} seconds until delegation token renewal") + Thread.sleep(msWait) + + val tokens = createHDFSDelegationTokens + broadcaster.broadcastDelegationTokens(tokens) + } + catch { + case e: SparkNoDelegationTokenException => + logError(s"Stopping delegation token renewal due to: $e") + return + case e: InterruptedException => + return + case e: Exception => + logError(s"Exception during token renewal: $e") + Thread.sleep(10000) + } + } + } + + private def getHDFSTokenRenewalInterval(creds: Credentials): Long = { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + // filter for HDFS delgation tokens. There might be others, but our renewal only + // supports HDFS for the moment. + val ts = creds.getAllTokens.asScala + .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) + if (ts.isEmpty) { + throw new SparkNoDelegationTokenException + } + val intervals = ts.map(t => { + val newExpiration = t.renew(hadoopConf) + val identifier = new DelegationTokenIdentifier() + identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) + newExpiration - identifier.getIssueDate + }) + intervals.min + } +} + +private[spark] case class SparkNoDelegationTokenException() + extends SparkException(s"No delegation token to renew") diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 6767cc507964..930d7d465135 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,6 +76,9 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; + // Mesos-only options. + protected final String TGT = "--tgt"; + /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" @@ -114,6 +117,7 @@ class SparkSubmitOptionParser { { QUEUE }, { REPOSITORIES }, { STATUS }, + { TGT }, { TOTAL_EXECUTOR_CORES }, }; diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1bc6f71860c3..d2e2bdb38b6f 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -386,6 +386,10 @@ private[spark] class MesosClusterScheduler( env.foreach { case (k, v) => envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v)) } + + // Pass the krb5.conf to the scheduler + passKerberosConf(envBuilder) + envBuilder.build() } @@ -463,6 +467,16 @@ private[spark] class MesosClusterScheduler( desc.conf.getOption("spark.executor.memory").foreach { v => options ++= Seq("--executor-memory", v) } + desc.conf.get("spark.yarn.principal").map { v => + options ++= Seq("--conf", s"spark.yarn.principal=$v") + } + desc.conf.get("spark.mesos.kerberos.keytabBase64").map { v => + options ++= Seq("--conf", s"spark.mesos.kerberos.keytabBase64=$v") + } + desc.conf.get("spark.mesos.kerberos.tgtBase64").map { v => + options ++= Seq("--conf", s"spark.mesos.kerberos.tgtBase64=$v") + } + desc.conf.getOption("spark.cores.max").foreach { v => options ++= Seq("--total-executor-cores", v) } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 2a36ec4fa811..90e05e05c847 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File import java.util.{Collections, List => JList} import java.util.concurrent.locks.ReentrantLock +import javax.xml.bind.DatatypeConverter import scala.collection.JavaConverters._ import scala.collection.mutable @@ -33,6 +34,7 @@ import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -53,6 +55,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( securityManager: SecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler + with DelegationTokenBroadcaster with MesosSchedulerUtils { // Blacklist a slave after this many failures @@ -173,6 +176,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( unsetFrameworkID(sc) startScheduler(driver) + + if (kerberosBackend != null) { + kerberosBackend.start() + } } def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { @@ -185,6 +192,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") + // Pass the krb5.conf to the scheduler + passKerberosConf(environment) + environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") @@ -238,6 +248,27 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( command.build() } + override def broadcastDelegationTokens(tokens: Array[Byte]): Unit = { + // store new tokens locally for future executors + val initialTokensBuf = DatatypeConverter.printBase64Binary(tokens) + conf.set("spark.mesos.kerberos.hdfsDelegationTokens", initialTokensBuf) + + // send token to existing executors + logInfo("Sending UpdateDelegationTokens to all executors") + driverEndpoint.send(UpdateDelegationTokens(tokens)) + } + + val principal = conf.get("spark.yarn.principal", null) + var kerberosBackend: MesosKerberosHandler = null + if (principal != null) { + kerberosBackend = new MesosKerberosHandler(conf, principal, this) + + // store tokens in spark property which is sent to the executors initially + val initialTokens = kerberosBackend.createHDFSDelegationTokens + val initialTokensBuf = DatatypeConverter.printBase64Binary(initialTokens) + conf.set("spark.mesos.kerberos.hdfsDelegationTokens", initialTokensBuf) + } + protected def driverURL: String = { if (conf.contains("spark.testing")) { "driverURL" @@ -590,6 +621,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Close the mesos external shuffle client if used mesosExternalShuffleClient.foreach(_.close()) + if (kerberosBackend != null) { + kerberosBackend.stop() + } if (schedulerDriver != null) { schedulerDriver.stop() diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 9d81025a3016..2f83b2073653 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -442,7 +442,12 @@ trait MesosSchedulerUtils extends Logging { /** * The values of the non-zero ports to be used by the executor process. + * <<<<<<< HEAD * + * ======= + * + * >>>>>>> 0a2cc42... [Mesosphere SPARK-126] Add Mesos Kerberos support + * * @param conf the spark config to use * @return the ono-zero values of the ports */ @@ -555,4 +560,14 @@ trait MesosSchedulerUtils extends Logging { driver.declineOffer(offer.getId) } } + + // Pass the krb5.conf to the scheduler + def passKerberosConf(envBuilder: Environment.Builder): Unit = { + Option(System.getenv().getOrDefault("SPARK_MESOS_KRB5_CONF_BASE64", null)).foreach(krb5conf => { + logError(s"Passing ${krb5conf.length} bytes krb5.conf to sub-task") + envBuilder.addVariables(Environment.Variable.newBuilder(). + setName("SPARK_MESOS_KRB5_CONF_BASE64").setValue(krb5conf).build() + ) + }) + } } From 35002f2bd2e906bf1c6e6800f1f346e962edca75 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 17 Apr 2017 15:31:25 -0700 Subject: [PATCH 03/15] Par down kerberos support --- bin/spark-class | 13 -- .../apache/spark/deploy/SparkHadoopUtil.scala | 30 --- .../org/apache/spark/deploy/SparkSubmit.scala | 52 ++--- .../ConfigurableCredentialManager.scala | 26 +-- .../security/HBaseCredentialProvider.scala | 2 +- .../security/HadoopFSCredentialProvider.scala | 81 ++++++++ .../security/HiveCredentialProvider.scala | 2 +- .../security/ServiceCredentialProvider.scala | 3 +- .../CoarseGrainedExecutorBackend.scala | 16 +- .../cluster/CoarseGrainedClusterMessage.scala | 2 - .../CoarseGrainedSchedulerBackend.scala | 10 - .../cluster/mesos/MesosKerberosHandler.scala | 191 ------------------ dev/.rat-excludes | 2 +- docs/running-on-yarn.md | 2 +- .../launcher/SparkSubmitOptionParser.java | 4 - ....deploy.security.ServiceCredentialProvider | 3 + .../apache/spark/deploy/mesos/config.scala | 9 + .../MesosCoarseGrainedSchedulerBackend.scala | 67 +++--- .../cluster/mesos/MesosSchedulerUtils.scala | 5 - ....deploy.security.ServiceCredentialProvider | 3 + ...oy.yarn.security.ServiceCredentialProvider | 3 - .../spark/deploy/yarn/ApplicationMaster.scala | 11 +- .../org/apache/spark/deploy/yarn/Client.scala | 7 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 10 +- .../yarn/security/AMCredentialRenewer.scala | 2 +- .../yarn/security/CredentialUpdater.scala | 3 +- ...a => YARNHadoopFSCredentialProvider.scala} | 76 ++----- ...deploy.security.ServiceCredentialProvider} | 0 .../ConfigurableCredentialManagerSuite.scala | 48 ++--- .../HadoopFSCredentialProviderSuite.scala | 2 +- 30 files changed, 231 insertions(+), 454 deletions(-) rename {resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn => core/src/main/scala/org/apache/spark/deploy}/security/ConfigurableCredentialManager.scala (83%) rename {resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn => core/src/main/scala/org/apache/spark/deploy}/security/HBaseCredentialProvider.scala (98%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala rename {resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn => core/src/main/scala/org/apache/spark/deploy}/security/HiveCredentialProvider.scala (99%) rename {resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn => core/src/main/scala/org/apache/spark/deploy}/security/ServiceCredentialProvider.scala (97%) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala create mode 100644 resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider create mode 100644 resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider delete mode 100644 resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider rename resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/{HadoopFSCredentialProvider.scala => YARNHadoopFSCredentialProvider.scala} (58%) rename resource-managers/yarn/src/test/resources/META-INF/services/{org.apache.spark.deploy.yarn.security.ServiceCredentialProvider => org.apache.spark.deploy.security.ServiceCredentialProvider} (100%) diff --git a/bin/spark-class b/bin/spark-class index 87aecc05e5e0..77ea40cc3794 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -55,19 +55,6 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then LAUNCH_CLASSPATH="${SPARK_HOME}/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" fi -# copy krb5.conf into a temporary file and tell java.security about it -if [ -n "${SPARK_MESOS_KRB5_CONF_BASE64}" ]; then - if base64 --help | grep -q GNU; then - BASE64_D="base64 -d" # GNU - else - BASE64_D="base64 -D" # BSD - fi - KRB5_TMP_FILE=$(mktemp -t krb5-XXXXX.conf) - echo "Writing krb5.conf to $KRB5_TMP_FILE" 1>&2 - echo "${SPARK_MESOS_KRB5_CONF_BASE64}" | ${BASE64_D} > ${KRB5_TMP_FILE} - export SPARK_JAVA_OPTS="${SPARK_JAVA_OPTS} -Djava.security.krb5.conf=${KRB5_TMP_FILE}" -fi - # For tests if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 33e4a4dd84f3..58558673bc2e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -138,36 +138,6 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } - def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - delegTokenRenewer - } - - /** - * Obtains tokens for the namenodes passed in and adds them to the credentials. - */ - def obtainTokensForNamenodes( - paths: Set[Path], - conf: Configuration, - creds: Credentials, - renewer: Option[String] = None - ): Unit = { - if (UserGroupInformation.isSecurityEnabled()) { - val delegTokenRenewer = renewer.getOrElse(getTokenRenewer(conf)) - paths.foreach { dst => - val dstFs = dst.getFileSystem(conf) - logInfo("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) - } - } - } - /** * Returns a function that can be called to find Hadoop FileSystem bytes read. If * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will 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 445e31cb875e..886602c04791 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -20,10 +20,8 @@ package org.apache.spark.deploy import java.io.{File, IOException} import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} import java.net.URL -import java.nio.file.{Files, Paths} import java.security.PrivilegedExceptionAction import java.text.ParseException -import javax.xml.bind.DatatypeConverter import scala.annotation.tailrec import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -32,6 +30,7 @@ import scala.util.Properties import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.Path import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.ivy.Ivy import org.apache.ivy.core.LogOptions import org.apache.ivy.core.module.descriptor._ @@ -47,6 +46,7 @@ import org.apache.ivy.plugins.resolver.{ChainResolver, FileSystemResolver, IBibl import org.apache.spark._ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ +import org.apache.spark.internal.Logging import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util._ @@ -65,7 +65,7 @@ private[deploy] object SparkSubmitAction extends Enumeration { * This program handles setting up the classpath with relevant Spark dependencies and provides * a layer over the different cluster managers and deploy modes that Spark supports. */ -object SparkSubmit extends CommandLineUtils { +object SparkSubmit extends CommandLineUtils with Logging { // Cluster managers private val YARN = 1 @@ -465,10 +465,6 @@ object SparkSubmit extends CommandLineUtils { OptionAssigner(args.principal, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.principal"), OptionAssigner(args.keytab, YARN, ALL_DEPLOY_MODES, sysProp = "spark.yarn.keytab"), - // Mesos cluster only - OptionAssigner(args.principal, MESOS, CLUSTER), - OptionAssigner(args.keytab, MESOS, CLUSTER), - // Other options OptionAssigner(args.executorCores, STANDALONE | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.cores"), @@ -574,35 +570,16 @@ object SparkSubmit extends CommandLineUtils { } } - if (clusterManager == MESOS && args.principal != null) { - sysProps.put("spark.yarn.principal", args.principal) - - // set principal used to renew tokens. We use the job token for now - // because we have its keytab here and in the scheduler already. - sysProps.put("spark.hadoop.yarn.resourcemanager.principal", args.principal) - - if (!args.sparkProperties.contains("spark.mesos.kerberos.keytabBase64")) { - require(args.keytab != null, "Keytab must be specified when principal is specified.") - if (args.keytab != null && !new File(args.keytab).exists()) { - throw new SparkException(s"Keytab file: ${args.keytab} does not exist") - } - - val path: String = args.keytab - val key: String = s"spark.mesos.kerberos.keytabBase64" - // load keytab or tgt and pass to the driver via spark property - val bytes = Files.readAllBytes(Paths.get(path)) - sysProps.put(key, DatatypeConverter.printBase64Binary(bytes)) + // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we + // must trick it into thinking we're YARN. + if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { + val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName + val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" - // set auth mechanism to Kerberos and login locally if in CLIENT mode. In cluster mode - // no local Kerberos setup is necessary. - if (deployMode == CLIENT) { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(new SparkConf()) - hadoopConf.set("hadoop.security.authentication", "Kerberos") - UserGroupInformation.setConfiguration(hadoopConf) - UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) - } - } + logDebug(s"Setting ${key} to ${shortUserName}.") + sysProps.put(key, shortUserName) } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class @@ -870,6 +847,7 @@ private[spark] object SparkSubmitUtils { /** * Represents a Maven Coordinate + * * @param groupId the groupId of the coordinate * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate @@ -881,6 +859,7 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. + * * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ @@ -911,6 +890,7 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string + * * @param defaultIvyUserDir The default user path for Ivy * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ @@ -958,6 +938,7 @@ private[spark] object SparkSubmitUtils { /** * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath * (will append to jars in SparkSubmit). + * * @param artifacts Sequence of dependencies that were resolved and retrieved * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies @@ -1010,6 +991,7 @@ private[spark] object SparkSubmitUtils { /** * Build Ivy Settings using options with default resolvers + * * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository * @return An IvySettings object @@ -1030,6 +1012,7 @@ private[spark] object SparkSubmitUtils { /** * Load Ivy settings from a given filename, using supplied resolvers + * * @param settingsFile Path to Ivy settings file * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository @@ -1095,6 +1078,7 @@ private[spark] object SparkSubmitUtils { /** * Resolves any dependencies that were supplied through maven coordinates + * * @param coordinates Comma-delimited string of maven coordinates * @param ivySettings An IvySettings containing resolvers to use * @param exclusions Exclusions to apply when resolving transitive dependencies diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala similarity index 83% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala rename to core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala index 4f4be52a0d69..24fb35de948a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn.security +package org.apache.spark.deploy.security import java.util.ServiceLoader @@ -41,7 +41,7 @@ import org.apache.spark.util.Utils * For example, Hive's credential provider [[HiveCredentialProvider]] can be enabled/disabled by * the configuration spark.yarn.security.credentials.hive.enabled. */ -private[yarn] final class ConfigurableCredentialManager( +private[spark] final class ConfigurableCredentialManager( sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled" private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled" @@ -64,6 +64,8 @@ private[yarn] final class ConfigurableCredentialManager( }.map { p => (p.serviceName, p) }.toMap } + logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") + /** * Get credential provider for the specified service. */ @@ -72,7 +74,8 @@ private[yarn] final class ConfigurableCredentialManager( } /** - * Obtain credentials from all the registered providers. + * Writes delegation tokens to creds. Delegation tokens are fetched from all registered + * providers. * @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable, * otherwise the nearest renewal time of any credentials will be returned. */ @@ -87,21 +90,4 @@ private[yarn] final class ConfigurableCredentialManager( } }.foldLeft(Long.MaxValue)(math.min) } - - /** - * Create an [[AMCredentialRenewer]] instance, caller should be responsible to stop this - * instance when it is not used. AM will use it to renew credentials periodically. - */ - def credentialRenewer(): AMCredentialRenewer = { - new AMCredentialRenewer(sparkConf, hadoopConf, this) - } - - /** - * Create an [[CredentialUpdater]] instance, caller should be resposible to stop this intance - * when it is not used. Executors and driver (client mode) will use it to update credentials. - * periodically. - */ - def credentialUpdater(): CredentialUpdater = { - new CredentialUpdater(sparkConf, hadoopConf, this) - } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala similarity index 98% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala index 5adeb8e605ff..29b0b9e9210d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn.security +package org.apache.spark.deploy.security import scala.reflect.runtime.universe import scala.util.control.NonFatal diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala new file mode 100644 index 000000000000..334fe02d23db --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala @@ -0,0 +1,81 @@ +/* + * 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.security + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging + +private[deploy] class HadoopFSCredentialProvider + extends ServiceCredentialProvider with Logging { + // Token renewal interval, this value will be set in the first call, + // if None means no token renewer specified or no token can be renewed, + // so cannot get token renewal interval. + private var tokenRenewalInterval: Option[Long] = null + + override val serviceName: String = "hadoopfs" + + override def obtainCredentials( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + // NameNode to access, used to get tokens from different FileSystems + val tmpCreds = new Credentials() + val tokenRenewer = getTokenRenewer(hadoopConf) + hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => + val dstFs = dst.getFileSystem(hadoopConf) + logInfo("getting token for: " + dst) + dstFs.addDelegationTokens(tokenRenewer, tmpCreds) + } + + // Get the token renewal interval if it is not set. It will only be called once. + if (tokenRenewalInterval == null) { + tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf) + } + + // Get the time of next renewal. + val nextRenewalDate = tokenRenewalInterval.flatMap { interval => + val nextRenewalDates = tmpCreds.getAllTokens.asScala + .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) + .map { t => + val identifier = t.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + identifier.getIssueDate + interval + } + if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) + } + + creds.addAll(tmpCreds) + nextRenewalDate + } + + def getTokenRenewalInterval(hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = None + + def getTokenRenewer(hadoopConf: Configuration): String = { + UserGroupInformation.getCurrentUser.getShortUserName + } + + def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { + Set(FileSystem.get(hadoopConf).getHomeDirectory) + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala similarity index 99% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 16d8fc32bb42..d8d4cccbd085 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn.security +package org.apache.spark.deploy.security import java.lang.reflect.UndeclaredThrowableException import java.security.PrivilegedExceptionAction diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala similarity index 97% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala index 4e3fcce8dbb1..e9a58be0dab3 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn.security +package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -44,6 +44,7 @@ trait ServiceCredentialProvider { /** * Obtain credentials for this service and get the time of the next renewal. + * * @param hadoopConf Configuration of current Hadoop Compatible system. * @param sparkConf Spark configuration. * @param creds Credentials to add tokens and security keys to. diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 1d1c632cc25b..02318c9b7d42 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -126,10 +126,6 @@ private[spark] class CoarseGrainedExecutorBackend( executor.stop() } }.start() - - case UpdateDelegationTokens(tokens) => - logInfo(s"Got UpdateDelegationTokens message with ${tokens.length} bytes") - CoarseGrainedExecutorBackend.addDelegationTokens(tokens, env.conf) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { @@ -181,11 +177,12 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { - private def addDelegationTokens(tokens: Array[Byte], sparkConf: SparkConf) { - logInfo(s"Found delegation tokens of ${tokens.length} bytes") + private def addDelegationTokens(tokens: Array[Byte], driverConf: SparkConf) { + logInfo(s"Found delegation tokens of ${tokens.length} bytes.") + // configure to use tokens for HDFS login - val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val hadoopConf = SparkHadoopUtil.get.newConfiguration(driverConf) hadoopConf.set("hadoop.security.authentication", "Token") UserGroupInformation.setConfiguration(hadoopConf) @@ -242,8 +239,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { SparkHadoopUtil.get.startCredentialUpdater(driverConf) } - if (driverConf.contains("spark.mesos.kerberos.hdfsDelegationTokens")) { - val value = driverConf.get("spark.mesos.kerberos.hdfsDelegationTokens") + if (driverConf.contains("spark.mesos.kerberos.userCredentials")) { + val value = driverConf.get("spark.mesos.kerberos.userCredentials") + logInfo(s"token value=${value}") val tokens = DatatypeConverter.parseBase64Binary(value) addDelegationTokens(tokens, driverConf) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index cc90132bbc88..6b49bd699a13 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -53,8 +53,6 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterExecutorFailed(message: String) extends CoarseGrainedClusterMessage with RegisterExecutorResponse - case class UpdateDelegationTokens(tokens: Array[Byte]) extends CoarseGrainedClusterMessage - // Executors to driver case class RegisterExecutor( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 1f96223673eb..4eedaaea6119 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -146,12 +146,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp scheduler.getExecutorsAliveOnHost(host).foreach { exec => killExecutors(exec.toSeq, replace = true, force = true) } - - case UpdateDelegationTokens(tokens) => - logInfo("Asking each executor to update HDFS delegation tokens") - for ((_, executorData) <- executorDataMap) { - executorData.executorEndpoint.send(UpdateDelegationTokens(tokens)) - } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -482,7 +476,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Request an additional number of executors from the cluster manager. - * * @return whether the request is acknowledged. */ final override def requestExecutors(numAdditionalExecutors: Int): Boolean = { @@ -508,7 +501,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. - * * @param numExecutors The total number of executors we'd like to have. The cluster manager * shouldn't kill any running executor to reach this number, but, * if all existing executors were to die, this is the number of executors @@ -623,7 +615,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Kill the given list of executors through the cluster manager. - * * @return whether the kill request is acknowledged. */ protected def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = @@ -631,7 +622,6 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp /** * Request that the cluster manager kill all executors on a given host. - * * @return whether the kill request is acknowledged. */ final override def killExecutorsOnHost(host: String): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala deleted file mode 100644 index 06e1d36b4143..000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosKerberosHandler.scala +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.io.{ByteArrayInputStream, DataInputStream} -import java.nio.file.Files -import java.nio.file.attribute.PosixFilePermissions -import java.security.PrivilegedExceptionAction -import javax.xml.bind.DatatypeConverter - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.internal.Logging - - -/** - * DelegationTokenBroadcaster is a callback interface to broadcast new tokens - * to the executors. - */ -trait DelegationTokenBroadcaster { - def broadcastDelegationTokens(tokens: Array[Byte]): Unit -} - -/** - * MesosKerberosHandler implements the Kerberos logic for Mesos - */ -private[spark] -class MesosKerberosHandler(conf: SparkConf, - principal: String, - broadcaster: DelegationTokenBroadcaster) - extends Object with Logging { - - @volatile private var renewalCredentials: Credentials = null - @volatile private var stopRenewal = false - var renewalThread: Thread = null - - def start(): Unit = { - logInfo("Starting delegation token renewer") - renewalThread = new Thread(new Runnable { - def run() { - renewLoop() - } - }) - renewalThread.start() - } - - def stop(): Unit = { - logWarning("Stopping delegation token renewer") - stopRenewal = true - if (renewalThread != null) { - renewalThread.interrupt() - } - } - - def createHDFSDelegationTokens: Array[Byte] = { - // get keytab or tgt, and login - val keytab64 = conf.get("spark.mesos.kerberos.keytabBase64", null) - val tgt64 = conf.get("spark.mesos.kerberos.tgtBase64", null) - require(keytab64 != null || tgt64 != null, "keytab or tgt required") - require(keytab64 == null || tgt64 == null, "keytab and tgt cannot be used at the same time") - - val mode = if (keytab64 != null) "keytab" else "tgt" - logInfo(s"Logging in as $principal with $mode to retrieve HDFS delegation tokens") - - // write keytab or tgt into a temporary file - val bytes = DatatypeConverter.parseBase64Binary(if (keytab64 != null) keytab64 else tgt64) - val kerberosSecretFile = Files.createTempFile("spark-mesos-kerberos-token", ".tmp", - PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rw-------"))) - kerberosSecretFile.toFile.deleteOnExit() // just to be sure - Files.write(kerberosSecretFile, bytes) - - // login - try { - // login with _new_ user in order to start without any token (necessary to make sure that - // new tokens are really downloaded, even when not yet expired) - val ugi = if (keytab64 != null) { - UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, kerberosSecretFile.toString) - } - else { - UserGroupInformation.getUGIFromTicketCache(kerberosSecretFile.toString, principal) - } - - // get tokens - val ugiCreds = getHDFSDelegationTokens(ugi) - logInfo(s"Got ${ugiCreds.numberOfTokens()} HDFS delegation tokens") - - // write tokens into a memory file to transfer it to the executors - val tokenBuf = new java.io.ByteArrayOutputStream(1024 * 1024) - ugiCreds.writeTokenStorageToStream(new java.io.DataOutputStream(tokenBuf)) - logDebug(s"Wrote ${tokenBuf.size()} bytes of token data") - - // store the renewal credentials, needed to get the waiting time for - // the next renewal - renewalCredentials = ugiCreds - - // make new ugi active - UserGroupInformation.setLoginUser(ugi) - - tokenBuf.toByteArray - } - finally { - kerberosSecretFile.toFile.delete() - } - } - - private def getHDFSDelegationTokens(ugi: UserGroupInformation): Credentials = { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val namenodes = Option(hadoopConf.get("dfs.ha.namenodes.hdfs", null)). - map(_.split(",")).getOrElse(Array[String]()). - flatMap(id => Option(hadoopConf.get(s"dfs.namenode.rpc-address.hdfs.$id", null))). - map(hostPort => new Path(s"hdfs://$hostPort")). - toSet - logInfo(s"Found these HDFS namenodes: $namenodes") - val ugiCreds = ugi.getCredentials - ugi.doAs(new PrivilegedExceptionAction[Unit] { - override def run() = { - // use the job principal itself to renew the tokens - SparkHadoopUtil.get.obtainTokensForNamenodes( - namenodes, hadoopConf, ugiCreds, Some(principal) - ) - } - }) - ugiCreds - } - - private def renewLoop(): Unit = { - while (!stopRenewal) { - try { - val msLeft = getHDFSTokenRenewalInterval(renewalCredentials) - val msWait = Math.max(msLeft / 2, 30 * 1000) - logInfo(s"Waiting ${msWait / 1000} seconds until delegation token renewal") - Thread.sleep(msWait) - - val tokens = createHDFSDelegationTokens - broadcaster.broadcastDelegationTokens(tokens) - } - catch { - case e: SparkNoDelegationTokenException => - logError(s"Stopping delegation token renewal due to: $e") - return - case e: InterruptedException => - return - case e: Exception => - logError(s"Exception during token renewal: $e") - Thread.sleep(10000) - } - } - } - - private def getHDFSTokenRenewalInterval(creds: Credentials): Long = { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - // filter for HDFS delgation tokens. There might be others, but our renewal only - // supports HDFS for the moment. - val ts = creds.getAllTokens.asScala - .filter(_.getKind == DelegationTokenIdentifier.HDFS_DELEGATION_KIND) - if (ts.isEmpty) { - throw new SparkNoDelegationTokenException - } - val intervals = ts.map(t => { - val newExpiration = t.renew(hadoopConf) - val identifier = new DelegationTokenIdentifier() - identifier.readFields(new DataInputStream(new ByteArrayInputStream(t.getIdentifier))) - newExpiration - identifier.getIssueDate - }) - intervals.min - } -} - -private[spark] case class SparkNoDelegationTokenException() - extends SparkException(s"No delegation token to renew") diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 2355d40d1e6f..4ae5b0b08324 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -102,7 +102,7 @@ spark-deps-.* org.apache.spark.scheduler.ExternalClusterManager .*\.sql .Rbuildignore -org.apache.spark.deploy.yarn.security.ServiceCredentialProvider +org.apache.spark.deploy.security.ServiceCredentialProvider spark-warehouse structured-streaming/* kafka-source-initial-offset-version-2.1.0.bin diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index e9ddaa76a797..eaa320efe4f6 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -504,7 +504,7 @@ spark.yarn.access.hadoopFileSystems hdfs://ireland.example.org:8020/,webhdfs://f ``` Spark supports integrating with other security-aware services through Java Services mechanism (see -`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` +`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.security.ServiceCredentialProvider` should be available to Spark by listing their names in the corresponding file in the jar's `META-INF/services` directory. These plug-ins can be disabled by setting `spark.yarn.security.credentials.{service}.enabled` to `false`, where `{service}` is the name of diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java index 930d7d465135..6767cc507964 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitOptionParser.java @@ -76,9 +76,6 @@ class SparkSubmitOptionParser { protected final String PRINCIPAL = "--principal"; protected final String QUEUE = "--queue"; - // Mesos-only options. - protected final String TGT = "--tgt"; - /** * This is the canonical list of spark-submit options. Each entry in the array contains the * different aliases for the same option; the first element of each entry is the "official" @@ -117,7 +114,6 @@ class SparkSubmitOptionParser { { QUEUE }, { REPOSITORIES }, { STATUS }, - { TGT }, { TOTAL_EXECUTOR_CORES }, }; diff --git a/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider new file mode 100644 index 000000000000..9ffeb4d50029 --- /dev/null +++ b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider @@ -0,0 +1,3 @@ +org.apache.spark.deploy.security.HadoopFSCredentialProvider +org.apache.spark.deploy.security.HBaseCredentialProvider +org.apache.spark.deploy.security.HiveCredentialProvider diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index 19e253394f1b..58a1c12f2de3 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -56,4 +56,13 @@ package object config { .stringConf .createOptional + private[spark] val KERBEROS_PRINCIPAL = ConfigBuilder("spark.mesos.kerberos.principal") + .doc("Kerberos principal.") + .stringConf + .createOptional + + private[spark] val USER_CREDENTIALS = ConfigBuilder("spark.mesos.kerberos.userCredentials") + .doc("Base64 encoding of UGI tokens.") + .stringConf + .createOptional } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 90e05e05c847..f166456ed897 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -26,15 +26,20 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.mesos.Protos.{Credentials => _, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.mesos.config +import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -55,7 +60,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( securityManager: SecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler - with DelegationTokenBroadcaster with MesosSchedulerUtils { // Blacklist a slave after this many failures @@ -161,7 +165,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } override def start() { + if (UserGroupInformation.isSecurityEnabled) { + setUGITokens + } + super.start() + val driver = createSchedulerDriver( master, MesosCoarseGrainedSchedulerBackend.this, @@ -176,10 +185,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( unsetFrameworkID(sc) startScheduler(driver) - - if (kerberosBackend != null) { - kerberosBackend.start() - } } def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { @@ -192,9 +197,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") - // Pass the krb5.conf to the scheduler - passKerberosConf(environment) - environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") @@ -248,25 +250,37 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( command.build() } - override def broadcastDelegationTokens(tokens: Array[Byte]): Unit = { - // store new tokens locally for future executors - val initialTokensBuf = DatatypeConverter.printBase64Binary(tokens) - conf.set("spark.mesos.kerberos.hdfsDelegationTokens", initialTokensBuf) + /** + * Returns the user's credentials, with new delegation tokens added for all configured + * services. + */ + private def getDelegationTokens: Credentials = { + logDebug(s"Retrieving delegation tokens.") + + val userCreds = UserGroupInformation.getCurrentUser.getCredentials + val numTokensBefore = userCreds.numberOfTokens + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val credentialManager = new ConfigurableCredentialManager(conf, hadoopConf) + credentialManager.obtainCredentials(hadoopConf, userCreds) + + logDebug(s"Fetched ${userCreds.numberOfTokens - numTokensBefore} delegation token(s).") - // send token to existing executors - logInfo("Sending UpdateDelegationTokens to all executors") - driverEndpoint.send(UpdateDelegationTokens(tokens)) + userCreds } - val principal = conf.get("spark.yarn.principal", null) - var kerberosBackend: MesosKerberosHandler = null - if (principal != null) { - kerberosBackend = new MesosKerberosHandler(conf, principal, this) + /** Writes delegation tokens to spark.mesos.kerberos.ugiTokens */ + private def setUGITokens: Unit = { + val userCreds = getDelegationTokens - // store tokens in spark property which is sent to the executors initially - val initialTokens = kerberosBackend.createHDFSDelegationTokens - val initialTokensBuf = DatatypeConverter.printBase64Binary(initialTokens) - conf.set("spark.mesos.kerberos.hdfsDelegationTokens", initialTokensBuf) + val byteStream = new java.io.ByteArrayOutputStream(1024 * 1024) + val dataStream = new java.io.DataOutputStream(byteStream) + userCreds.writeTokenStorageToStream(dataStream) + val credsBytes = byteStream.toByteArray + + logDebug(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") + + val creds64 = DatatypeConverter.printBase64Binary(credsBytes) + conf.set(config.USER_CREDENTIALS, creds64) } protected def driverURL: String = { @@ -621,9 +635,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // Close the mesos external shuffle client if used mesosExternalShuffleClient.foreach(_.close()) - if (kerberosBackend != null) { - kerberosBackend.stop() - } if (schedulerDriver != null) { schedulerDriver.stop() diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 2f83b2073653..693cce629858 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -442,11 +442,6 @@ trait MesosSchedulerUtils extends Logging { /** * The values of the non-zero ports to be used by the executor process. - * <<<<<<< HEAD - * - * ======= - * - * >>>>>>> 0a2cc42... [Mesosphere SPARK-126] Add Mesos Kerberos support * * @param conf the spark config to use * @return the ono-zero values of the ports diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider new file mode 100644 index 000000000000..f36407942a08 --- /dev/null +++ b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider @@ -0,0 +1,3 @@ +org.apache.spark.deploy.yarn.security.YARNHadoopFSCredentialProvider +org.apache.spark.deploy.security.HBaseCredentialProvider +org.apache.spark.deploy.security.HiveCredentialProvider diff --git a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider deleted file mode 100644 index f5a807ecac9d..000000000000 --- a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider +++ /dev/null @@ -1,3 +0,0 @@ -org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider -org.apache.spark.deploy.yarn.security.HBaseCredentialProvider -org.apache.spark.deploy.yarn.security.HiveCredentialProvider diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 864c834d110f..497cec7e1df7 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -20,25 +20,24 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import java.lang.reflect.InvocationTargetException import java.net.{Socket, URI, URL} -import java.util.concurrent.{TimeoutException, TimeUnit} +import java.util.concurrent.{TimeUnit, TimeoutException} import scala.collection.mutable.HashMap import scala.concurrent.Promise import scala.concurrent.duration.Duration import scala.util.control.NonFatal - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException import org.apache.hadoop.yarn.util.{ConverterUtils, Records} - import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer +import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, ConfigurableCredentialManager} +import org.apache.spark.deploy.yarn.security.AMCredentialRenewer import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rpc._ @@ -247,8 +246,8 @@ private[spark] class ApplicationMaster( if (sparkConf.contains(CREDENTIALS_FILE_PATH.key)) { // If a principal and keytab have been set, use that to create new credentials for executors // periodically - credentialRenewer = - new ConfigurableCredentialManager(sparkConf, yarnConf).credentialRenewer() + val credentialManager = new ConfigurableCredentialManager(sparkConf, yarnConf) + val credentialRenewer = new AMCredentialRenewer(sparkConf, yarnConf, credentialManager) credentialRenewer.scheduleLoginFromKeytab() } 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 424bbca12319..8b80077b3960 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, IOException, OutputStreamWriter} -import java.net.{InetAddress, UnknownHostException, URI} +import java.net.{InetAddress, URI, UnknownHostException} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.util.{Locale, Properties, UUID} @@ -27,7 +27,6 @@ import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.control.NonFatal - import com.google.common.base.Objects import com.google.common.io.Files import org.apache.hadoop.conf.Configuration @@ -45,11 +44,10 @@ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records - import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.ConfigurableCredentialManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} @@ -357,6 +355,7 @@ private[spark] class Client( * Upload any resources to the distributed cache if needed. If a resource is intended to be * consumed locally, set up the appropriate config for downstream code to handle it properly. * This is used for setting up a container launch context for our ApplicationMaster. + * * Exposed for testing. */ def prepareLocalResources( diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 93578855122c..adcd7b1d8ddd 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -22,7 +22,6 @@ import java.util.regex.Matcher import java.util.regex.Pattern import scala.collection.mutable.{HashMap, ListBuffer} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.JobConf @@ -32,10 +31,10 @@ import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils - import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.yarn.security.{ConfigurableCredentialManager, CredentialUpdater} +import org.apache.spark.deploy.security.ConfigurableCredentialManager +import org.apache.spark.deploy.yarn.security.CredentialUpdater import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.util.Utils @@ -87,8 +86,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = { - credentialUpdater = - new ConfigurableCredentialManager(sparkConf, newConfiguration(sparkConf)).credentialUpdater() + val hadoopConf = newConfiguration(sparkConf) + val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) credentialUpdater.start() } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala index 7e76f402db24..9d3fe003cd7c 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -22,9 +22,9 @@ import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation - import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala index 41b7b5d60b03..0b1573001ecb 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala @@ -20,13 +20,12 @@ package org.apache.spark.deploy.yarn.security import java.util.concurrent.{Executors, TimeUnit} import scala.util.control.NonFatal - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} - import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala similarity index 58% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala rename to resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala index f65c886db944..76e308372c8d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala @@ -21,60 +21,39 @@ import scala.collection.JavaConverters._ import scala.util.Try import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.Master import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.security.HadoopFSCredentialProvider import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -private[security] class HadoopFSCredentialProvider - extends ServiceCredentialProvider with Logging { - // Token renewal interval, this value will be set in the first call, - // if None means no token renewer specified or no token can be renewed, - // so cannot get token renewal interval. - private var tokenRenewalInterval: Option[Long] = null +class YARNHadoopFSCredentialProvider extends HadoopFSCredentialProvider { - override val serviceName: String = "hadoopfs" - - override def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] = { - // NameNode to access, used to get tokens from different FileSystems - val tmpCreds = new Credentials() - val tokenRenewer = getTokenRenewer(hadoopConf) - hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => - val dstFs = dst.getFileSystem(hadoopConf) - logInfo("getting token for: " + dst) - dstFs.addDelegationTokens(tokenRenewer, tmpCreds) - } - - // Get the token renewal interval if it is not set. It will only be called once. - if (tokenRenewalInterval == null) { - tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf) + override def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) } - // Get the time of next renewal. - val nextRenewalDate = tokenRenewalInterval.flatMap { interval => - val nextRenewalDates = tmpCreds.getAllTokens.asScala - .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) - .map { t => - val identifier = t.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] - identifier.getIssueDate + interval - } - if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) - } + delegTokenRenewer + } - creds.addAll(tmpCreds) - nextRenewalDate + override def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { + sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet + + sparkConf.get(STAGING_DIR).map(new Path(_)) + .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory) } - private def getTokenRenewalInterval( - hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = { + override def getTokenRenewalInterval( + hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = { // We cannot use the tokens generated with renewer yarn. Trying to renew // those will fail with an access control issue. So create new tokens with the logged in // user as renewer. @@ -100,21 +79,4 @@ private[security] class HadoopFSCredentialProvider } } - private def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - - delegTokenRenewer - } - - private def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { - sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet + - sparkConf.get(STAGING_DIR).map(new Path(_)) - .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory) - } } diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider similarity index 100% rename from resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider rename to resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala index b0067aa4517c..c4c3e061fadf 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala @@ -21,8 +21,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.Token +import org.apache.spark.deploy.security.{ConfigurableCredentialManager, HBaseCredentialProvider, HiveCredentialProvider, ServiceCredentialProvider} import org.scalatest.{BeforeAndAfter, Matchers} - import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.config._ @@ -96,29 +96,29 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal) } - test("obtain tokens For HiveMetastore") { - val hadoopConf = new Configuration() - hadoopConf.set("hive.metastore.kerberos.principal", "bob") - // thrift picks up on port 0 and bails out, without trying to talk to endpoint - hadoopConf.set("hive.metastore.uris", "http://localhost:0") - - val hiveCredentialProvider = new HiveCredentialProvider() - val credentials = new Credentials() - hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials) - - credentials.getAllTokens.size() should be (0) - } - - test("Obtain tokens For HBase") { - val hadoopConf = new Configuration() - hadoopConf.set("hbase.security.authentication", "kerberos") - - val hbaseTokenProvider = new HBaseCredentialProvider() - val creds = new Credentials() - hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds) - - creds.getAllTokens.size should be (0) - } +// test("obtain tokens For HiveMetastore") { +// val hadoopConf = new Configuration() +// hadoopConf.set("hive.metastore.kerberos.principal", "bob") +// // thrift picks up on port 0 and bails out, without trying to talk to endpoint +// hadoopConf.set("hive.metastore.uris", "http://localhost:0") +// +// val hiveCredentialProvider = new HiveCredentialProvider() +// val credentials = new Credentials() +// hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials) +// +// credentials.getAllTokens.size() should be (0) +// } +// +// test("Obtain tokens For HBase") { +// val hadoopConf = new Configuration() +// hadoopConf.set("hbase.security.authentication", "kerberos") +// +// val hbaseTokenProvider = new HBaseCredentialProvider() +// val creds = new Credentials() +// hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds) +// +// creds.getAllTokens.size should be (0) +// } } class TestCredentialProvider extends ServiceCredentialProvider { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala index f50ee193c258..770ec6e0434f 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala @@ -18,8 +18,8 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.security.HadoopFSCredentialProvider import org.scalatest.{Matchers, PrivateMethodTester} - import org.apache.spark.{SparkException, SparkFunSuite} class HadoopFSCredentialProviderSuite From 13981c8fe7934a8cee53be4cfd59fb14c8d9b07c Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 17 Apr 2017 15:57:51 -0700 Subject: [PATCH 04/15] cleanup --- .../apache/spark/deploy/SparkHadoopUtil.scala | 3 +- .../org/apache/spark/deploy/SparkSubmit.scala | 7 --- .../spark/deploy/SparkSubmitArguments.scala | 17 +++---- .../security/ServiceCredentialProvider.scala | 1 - .../CoarseGrainedExecutorBackend.scala | 15 +++--- .../ConfigurableCredentialManagerSuite.scala | 51 +++++++++---------- .../apache/spark/deploy/mesos/config.scala | 5 -- .../cluster/mesos/MesosClusterScheduler.scala | 12 ----- .../MesosCoarseGrainedSchedulerBackend.scala | 6 +-- .../cluster/mesos/MesosSchedulerUtils.scala | 10 ---- .../org/apache/spark/deploy/yarn/Client.scala | 3 +- ....deploy.security.ServiceCredentialProvider | 2 +- 12 files changed, 45 insertions(+), 87 deletions(-) rename {resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn => core/src/test/scala/org/apache/spark/deploy}/security/ConfigurableCredentialManagerSuite.scala (80%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 58558673bc2e..2a765e128620 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -28,7 +28,7 @@ import scala.util.control.NonFatal import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} -import org.apache.hadoop.mapred.{JobConf, Master} +import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier @@ -72,7 +72,6 @@ class SparkHadoopUtil extends Logging { } } - /** * Appends S3-specific, spark.hadoop.*, and spark.buffer.size configurations to a Hadoop * configuration. 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 886602c04791..e6e59fdf127e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -847,7 +847,6 @@ private[spark] object SparkSubmitUtils { /** * Represents a Maven Coordinate - * * @param groupId the groupId of the coordinate * @param artifactId the artifactId of the coordinate * @param version the version of the coordinate @@ -859,7 +858,6 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string. Coordinates should be provided * in the format `groupId:artifactId:version` or `groupId/artifactId:version`. - * * @param coordinates Comma-delimited string of maven coordinates * @return Sequence of Maven coordinates */ @@ -890,7 +888,6 @@ private[spark] object SparkSubmitUtils { /** * Extracts maven coordinates from a comma-delimited string - * * @param defaultIvyUserDir The default user path for Ivy * @return A ChainResolver used by Ivy to search for and resolve dependencies. */ @@ -938,7 +935,6 @@ private[spark] object SparkSubmitUtils { /** * Output a comma-delimited list of paths for the downloaded jars to be added to the classpath * (will append to jars in SparkSubmit). - * * @param artifacts Sequence of dependencies that were resolved and retrieved * @param cacheDirectory directory where jars are cached * @return a comma-delimited list of paths for the dependencies @@ -991,7 +987,6 @@ private[spark] object SparkSubmitUtils { /** * Build Ivy Settings using options with default resolvers - * * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository * @return An IvySettings object @@ -1012,7 +1007,6 @@ private[spark] object SparkSubmitUtils { /** * Load Ivy settings from a given filename, using supplied resolvers - * * @param settingsFile Path to Ivy settings file * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository @@ -1078,7 +1072,6 @@ private[spark] object SparkSubmitUtils { /** * Resolves any dependencies that were supplied through maven coordinates - * * @param coordinates Comma-delimited string of maven coordinates * @param ivySettings An IvySettings containing resolvers to use * @param exclusions Exclusions to apply when resolving transitive dependencies diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 6a68382b8709..0144fd1056ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -551,16 +551,6 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, | or all available cores on the worker in standalone mode) | - | YARN and Mesos only: - | --principal PRINCIPAL Principal to be used to login to KDC, while running on - | secure HDFS. - | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. For renewing the login tickets - | and the delegation tokens periodically, this keytab is copied - | - to the node running the Application Master via the Secure - | Distributed Cache on YARN, - | - or to the dispatcher and the driver on Mesos. - | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). @@ -570,6 +560,13 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | executors will be at least NUM. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. + | --principal PRINCIPAL Principal to be used to login to KDC, while running on + | secure HDFS. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. This keytab will be copied to + | the node running the Application Master via the Secure + | Distributed Cache, for renewing the login tickets and the + | delegation tokens periodically. """.stripMargin ) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala index e9a58be0dab3..667960c0f43b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala @@ -44,7 +44,6 @@ trait ServiceCredentialProvider { /** * Obtain credentials for this service and get the time of the next renewal. - * * @param hadoopConf Configuration of current Hadoop Compatible system. * @param sparkConf Spark configuration. * @param creds Credentials to add tokens and security keys to. diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 02318c9b7d42..f77795abe0a6 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -177,16 +177,18 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { - private def addDelegationTokens(tokens: Array[Byte], driverConf: SparkConf) { - logInfo(s"Found delegation tokens of ${tokens.length} bytes.") + private def addMesosDelegationTokens(driverConf: SparkConf) { + val value = driverConf.get("spark.mesos.kerberos.userCredentials") + val tokens = DatatypeConverter.parseBase64Binary(value) + logDebug(s"Found delegation tokens of ${tokens.length} bytes.") - // configure to use tokens for HDFS login + // Use tokens for HDFS login. val hadoopConf = SparkHadoopUtil.get.newConfiguration(driverConf) hadoopConf.set("hadoop.security.authentication", "Token") UserGroupInformation.setConfiguration(hadoopConf) - // decode tokens and add them to the credentials + // Decode tokens and add them to the current user's credentials. val creds = UserGroupInformation.getCurrentUser.getCredentials val tokensBuf = new java.io.ByteArrayInputStream(tokens) creds.readTokenStorageStream(new java.io.DataInputStream(tokensBuf)) @@ -240,10 +242,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { } if (driverConf.contains("spark.mesos.kerberos.userCredentials")) { - val value = driverConf.get("spark.mesos.kerberos.userCredentials") - logInfo(s"token value=${value}") - val tokens = DatatypeConverter.parseBase64Binary(value) - addDelegationTokens(tokens, driverConf) + addMesosDelegationTokens(driverConf) } val env = SparkEnv.createExecutorEnv( diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala similarity index 80% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala rename to core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala index c4c3e061fadf..428a30c77a39 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.deploy.yarn.security +package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.Token -import org.apache.spark.deploy.security.{ConfigurableCredentialManager, HBaseCredentialProvider, HiveCredentialProvider, ServiceCredentialProvider} import org.scalatest.{BeforeAndAfter, Matchers} + import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.yarn.config._ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { private var credentialManager: ConfigurableCredentialManager = null @@ -96,29 +95,29 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal) } -// test("obtain tokens For HiveMetastore") { -// val hadoopConf = new Configuration() -// hadoopConf.set("hive.metastore.kerberos.principal", "bob") -// // thrift picks up on port 0 and bails out, without trying to talk to endpoint -// hadoopConf.set("hive.metastore.uris", "http://localhost:0") -// -// val hiveCredentialProvider = new HiveCredentialProvider() -// val credentials = new Credentials() -// hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials) -// -// credentials.getAllTokens.size() should be (0) -// } -// -// test("Obtain tokens For HBase") { -// val hadoopConf = new Configuration() -// hadoopConf.set("hbase.security.authentication", "kerberos") -// -// val hbaseTokenProvider = new HBaseCredentialProvider() -// val creds = new Credentials() -// hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds) -// -// creds.getAllTokens.size should be (0) -// } + test("obtain tokens For HiveMetastore") { + val hadoopConf = new Configuration() + hadoopConf.set("hive.metastore.kerberos.principal", "bob") + // thrift picks up on port 0 and bails out, without trying to talk to endpoint + hadoopConf.set("hive.metastore.uris", "http://localhost:0") + + val hiveCredentialProvider = new HiveCredentialProvider() + val credentials = new Credentials() + hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials) + + credentials.getAllTokens.size() should be (0) + } + + test("Obtain tokens For HBase") { + val hadoopConf = new Configuration() + hadoopConf.set("hbase.security.authentication", "kerberos") + + val hbaseTokenProvider = new HBaseCredentialProvider() + val creds = new Credentials() + hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds) + + creds.getAllTokens.size should be (0) + } } class TestCredentialProvider extends ServiceCredentialProvider { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index 58a1c12f2de3..78d2de6ba537 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -56,11 +56,6 @@ package object config { .stringConf .createOptional - private[spark] val KERBEROS_PRINCIPAL = ConfigBuilder("spark.mesos.kerberos.principal") - .doc("Kerberos principal.") - .stringConf - .createOptional - private[spark] val USER_CREDENTIALS = ConfigBuilder("spark.mesos.kerberos.userCredentials") .doc("Base64 encoding of UGI tokens.") .stringConf diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index d2e2bdb38b6f..af1430e5914e 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -387,9 +387,6 @@ private[spark] class MesosClusterScheduler( envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v)) } - // Pass the krb5.conf to the scheduler - passKerberosConf(envBuilder) - envBuilder.build() } @@ -467,15 +464,6 @@ private[spark] class MesosClusterScheduler( desc.conf.getOption("spark.executor.memory").foreach { v => options ++= Seq("--executor-memory", v) } - desc.conf.get("spark.yarn.principal").map { v => - options ++= Seq("--conf", s"spark.yarn.principal=$v") - } - desc.conf.get("spark.mesos.kerberos.keytabBase64").map { v => - options ++= Seq("--conf", s"spark.mesos.kerberos.keytabBase64=$v") - } - desc.conf.get("spark.mesos.kerberos.tgtBase64").map { v => - options ++= Seq("--conf", s"spark.mesos.kerberos.tgtBase64=$v") - } desc.conf.getOption("spark.cores.max").foreach { v => options ++= Seq("--total-executor-cores", v) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index f166456ed897..68650400e084 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -26,9 +26,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future -import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.mesos.Protos.{Credentials => _, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver @@ -166,7 +164,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( override def start() { if (UserGroupInformation.isSecurityEnabled) { - setUGITokens + setUGITokens() } super.start() @@ -269,7 +267,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } /** Writes delegation tokens to spark.mesos.kerberos.ugiTokens */ - private def setUGITokens: Unit = { + private def setUGITokens(): Unit = { val userCreds = getDelegationTokens val byteStream = new java.io.ByteArrayOutputStream(1024 * 1024) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 693cce629858..514491d76222 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -555,14 +555,4 @@ trait MesosSchedulerUtils extends Logging { driver.declineOffer(offer.getId) } } - - // Pass the krb5.conf to the scheduler - def passKerberosConf(envBuilder: Environment.Builder): Unit = { - Option(System.getenv().getOrDefault("SPARK_MESOS_KRB5_CONF_BASE64", null)).foreach(krb5conf => { - logError(s"Passing ${krb5conf.length} bytes krb5.conf to sub-task") - envBuilder.addVariables(Environment.Variable.newBuilder(). - setName("SPARK_MESOS_KRB5_CONF_BASE64").setValue(krb5conf).build() - ) - }) - } } 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 8b80077b3960..45945134c3fc 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 @@ -27,6 +27,7 @@ import java.util.zip.{ZipEntry, ZipOutputStream} import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, ListBuffer, Map} import scala.util.control.NonFatal + import com.google.common.base.Objects import com.google.common.io.Files import org.apache.hadoop.conf.Configuration @@ -44,6 +45,7 @@ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records + import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.ConfigurableCredentialManager @@ -355,7 +357,6 @@ private[spark] class Client( * Upload any resources to the distributed cache if needed. If a resource is intended to be * consumed locally, set up the appropriate config for downstream code to handle it properly. * This is used for setting up a container launch context for our ApplicationMaster. - * * Exposed for testing. */ def prepareLocalResources( diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider index d0ef5efa36e8..2676a0ad589f 100644 --- a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider +++ b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider @@ -1 +1 @@ -org.apache.spark.deploy.yarn.security.TestCredentialProvider +org.apache.spark.deploy.security.TestCredentialProvider From af4a3e4f53509ee1bee714d0846518d2696e0800 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 17 Apr 2017 16:14:05 -0700 Subject: [PATCH 05/15] style --- .../main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala | 1 + .../org.apache.spark.deploy.security.ServiceCredentialProvider | 0 .../src/main/scala/org/apache/spark/deploy/mesos/config.scala | 2 +- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 2 -- 4 files changed, 2 insertions(+), 3 deletions(-) rename {resource-managers/yarn => core}/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider (100%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 2a765e128620..bae7a3f307f5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -72,6 +72,7 @@ class SparkHadoopUtil extends Logging { } } + /** * Appends S3-specific, spark.hadoop.*, and spark.buffer.size configurations to a Hadoop * configuration. diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider similarity index 100% rename from resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider rename to core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index 78d2de6ba537..55ef6ea38281 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -57,7 +57,7 @@ package object config { .createOptional private[spark] val USER_CREDENTIALS = ConfigBuilder("spark.mesos.kerberos.userCredentials") - .doc("Base64 encoding of UGI tokens.") + .doc("Base64 encoding of UGI credentials.") .stringConf .createOptional } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index af1430e5914e..1bc6f71860c3 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -386,7 +386,6 @@ private[spark] class MesosClusterScheduler( env.foreach { case (k, v) => envBuilder.addVariables(Variable.newBuilder().setName(k).setValue(v)) } - envBuilder.build() } @@ -464,7 +463,6 @@ private[spark] class MesosClusterScheduler( desc.conf.getOption("spark.executor.memory").foreach { v => options ++= Seq("--executor-memory", v) } - desc.conf.getOption("spark.cores.max").foreach { v => options ++= Seq("--total-executor-cores", v) } From 5cc66dc91e7684c582b08a84b4901541dd60e38b Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 17 Apr 2017 17:27:28 -0700 Subject: [PATCH 06/15] Add MesosSecurityManager --- .../cluster/mesos/MesosClusterManager.scala | 3 +- .../MesosCoarseGrainedSchedulerBackend.scala | 39 +---- .../cluster/mesos/MesosSecurityManager.scala | 68 +++++++++ ...osCoarseGrainedSchedulerBackendSuite.scala | 144 ++++++++++-------- 4 files changed, 153 insertions(+), 101 deletions(-) create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala index 911a0857917e..60cd324bbe26 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala @@ -48,7 +48,8 @@ private[spark] class MesosClusterManager extends ExternalClusterManager { scheduler.asInstanceOf[TaskSchedulerImpl], sc, mesosUrl, - sc.env.securityManager) + sc.env.securityManager, + new MesosSecurityManager) } else { new MesosFineGrainedSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 68650400e084..8ed50898f85f 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -55,7 +55,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, - securityManager: SecurityManager) + securityManager: SecurityManager, + mesosSecurityManager: MesosSecurityManager) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler with MesosSchedulerUtils { @@ -163,8 +164,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } override def start() { - if (UserGroupInformation.isSecurityEnabled) { - setUGITokens() + if (mesosSecurityManager.isSecurityEnabled()) { + mesosSecurityManager.setUGITokens(conf) } super.start() @@ -248,38 +249,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( command.build() } - /** - * Returns the user's credentials, with new delegation tokens added for all configured - * services. - */ - private def getDelegationTokens: Credentials = { - logDebug(s"Retrieving delegation tokens.") - - val userCreds = UserGroupInformation.getCurrentUser.getCredentials - val numTokensBefore = userCreds.numberOfTokens - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val credentialManager = new ConfigurableCredentialManager(conf, hadoopConf) - credentialManager.obtainCredentials(hadoopConf, userCreds) - - logDebug(s"Fetched ${userCreds.numberOfTokens - numTokensBefore} delegation token(s).") - - userCreds - } - - /** Writes delegation tokens to spark.mesos.kerberos.ugiTokens */ - private def setUGITokens(): Unit = { - val userCreds = getDelegationTokens - - val byteStream = new java.io.ByteArrayOutputStream(1024 * 1024) - val dataStream = new java.io.DataOutputStream(byteStream) - userCreds.writeTokenStorageToStream(dataStream) - val credsBytes = byteStream.toByteArray - - logDebug(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") - - val creds64 = DatatypeConverter.printBase64Binary(credsBytes) - conf.set(config.USER_CREDENTIALS, creds64) - } protected def driverURL: String = { if (conf.contains("spark.testing")) { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala new file mode 100644 index 000000000000..7b3d58440bfb --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.cluster.mesos + +import javax.xml.bind.DatatypeConverter + +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.mesos.config +import org.apache.spark.deploy.security.ConfigurableCredentialManager +import org.apache.spark.internal.Logging + + +private[mesos] class MesosSecurityManager extends Logging { + def isSecurityEnabled(): Boolean = { + UserGroupInformation.isSecurityEnabled + } + + /** Writes delegation tokens to spark.mesos.kerberos.ugiTokens */ + def setUGITokens(conf: SparkConf): Unit = { + val userCreds = getDelegationTokens(conf) + + val byteStream = new java.io.ByteArrayOutputStream(1024 * 1024) + val dataStream = new java.io.DataOutputStream(byteStream) + userCreds.writeTokenStorageToStream(dataStream) + val credsBytes = byteStream.toByteArray + + logDebug(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") + + val creds64 = DatatypeConverter.printBase64Binary(credsBytes) + conf.set(config.USER_CREDENTIALS, creds64) + } + + /** + * Returns the user's credentials, with new delegation tokens added for all configured + * services. + */ + private def getDelegationTokens(conf: SparkConf): Credentials = { + logDebug(s"Retrieving delegation tokens.") + + val userCreds = UserGroupInformation.getCurrentUser.getCredentials + val numTokensBefore = userCreds.numberOfTokens + val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + val credentialManager = new ConfigurableCredentialManager(conf, hadoopConf) + credentialManager.obtainCredentials(hadoopConf, userCreds) + + logDebug(s"Fetched ${userCreds.numberOfTokens - numTokensBefore} delegation token(s).") + + userCreds + } +} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index c040f05d93b3..71063ac0dfea 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -33,10 +33,11 @@ import org.scalatest.mock.MockitoSugar import org.scalatest.BeforeAndAfter import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.deploy.mesos.config import org.apache.spark.internal.config._ import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor, RetrieveSparkAppConfig, SparkAppConfig} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.mesos.Utils._ @@ -52,6 +53,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private var backend: MesosCoarseGrainedSchedulerBackend = _ private var externalShuffleClient: MesosExternalShuffleClient = _ private var driverEndpoint: RpcEndpointRef = _ + private var security: MesosSecurityManager = _ @volatile private var stopCalled = false // All 'requests' to the scheduler run immediately on the same thread, so @@ -59,7 +61,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS)) test("mesos supports killing and limiting executors") { - setBackend() + init() sparkConf.set("spark.driver.host", "driverHost") sparkConf.set("spark.driver.port", "1234") @@ -88,8 +90,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos supports killing and relaunching tasks with executors") { - setBackend() - + init() + // launches a task on a valid offer val minMem = backend.executorMemory(sc) + 1024 val minCpu = 4 @@ -110,7 +112,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports spark.executor.cores") { val executorCores = 4 - setBackend(Map("spark.executor.cores" -> executorCores.toString)) + init(Map("spark.executor.cores" -> executorCores.toString)) val executorMemory = backend.executorMemory(sc) val offers = List(Resources(executorMemory * 2, executorCores + 1)) @@ -124,7 +126,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos supports unset spark.executor.cores") { - setBackend() + init() val executorMemory = backend.executorMemory(sc) val offerCores = 10 @@ -139,7 +141,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos does not acquire more than spark.cores.max") { val maxCores = 10 - setBackend(Map("spark.cores.max" -> maxCores.toString)) + init(Map("spark.cores.max" -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List(Resources(executorMemory, maxCores + 1))) @@ -152,7 +154,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos does not acquire gpus if not specified") { - setBackend() + init() val executorMemory = backend.executorMemory(sc) offerResources(List(Resources(executorMemory, 1, 1))) @@ -167,7 +169,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos does not acquire more than spark.mesos.gpus.max") { val maxGpus = 5 - setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString)) + init(Map("spark.mesos.gpus.max" -> maxGpus.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) @@ -181,14 +183,14 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos declines offers that violate attribute constraints") { - setBackend(Map("spark.mesos.constraints" -> "x:true")) + init(Map("spark.mesos.constraints" -> "x:true")) offerResources(List(Resources(backend.executorMemory(sc), 4))) verifyDeclinedOffer(driver, createOfferId("o1"), true) } test("mesos declines offers with a filter when reached spark.cores.max") { val maxCores = 3 - setBackend(Map("spark.cores.max" -> maxCores.toString)) + init(Map("spark.cores.max" -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List( @@ -202,7 +204,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos assigns tasks round-robin on offers") { val executorCores = 4 val maxCores = executorCores * 2 - setBackend(Map("spark.executor.cores" -> executorCores.toString, + init(Map("spark.executor.cores" -> executorCores.toString, "spark.cores.max" -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) @@ -216,7 +218,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos creates multiple executors on a single slave") { val executorCores = 4 - setBackend(Map("spark.executor.cores" -> executorCores.toString)) + init(Map("spark.executor.cores" -> executorCores.toString)) // offer with room for two executors val executorMemory = backend.executorMemory(sc) @@ -228,7 +230,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos doesn't register twice with the same shuffle service") { - setBackend(Map("spark.shuffle.service.enabled" -> "true")) + init(Map("spark.shuffle.service.enabled" -> "true")) val (mem, cpu) = (backend.executorMemory(sc), 4) val offer1 = createOffer("o1", "s1", mem, cpu) @@ -249,7 +251,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("Port offer decline when there is no appropriate range") { - setBackend(Map(BLOCK_MANAGER_PORT.key -> "30100")) + init(Map(BLOCK_MANAGER_PORT.key -> "30100")) val offeredPorts = (31100L, 31200L) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -259,7 +261,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("Port offer accepted when ephemeral ports are used") { - setBackend() + init() val offeredPorts = (31100L, 31200L) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -270,7 +272,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("Port offer accepted with user defined port numbers") { val port = 30100 - setBackend(Map(BLOCK_MANAGER_PORT.key -> s"$port")) + init(Map(BLOCK_MANAGER_PORT.key -> s"$port")) val offeredPorts = (30000L, 31000L) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -289,7 +291,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos kills an executor when told") { - setBackend() + init() val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -302,7 +304,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("weburi is set in created scheduler driver") { - initializeSparkConf() + getSparkConf() sc = new SparkContext(sparkConf) val taskScheduler = mock[TaskSchedulerImpl] @@ -312,9 +314,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) val securityManager = mock[SecurityManager] + val security = new MesosSecurityManager val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager) { + taskScheduler, sc, "master", securityManager, security) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -336,7 +339,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("honors unset spark.mesos.containerizer") { - setBackend(Map("spark.mesos.executor.docker.image" -> "test")) + init(Map("spark.mesos.executor.docker.image" -> "test")) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -348,7 +351,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("honors spark.mesos.containerizer=\"mesos\"") { - setBackend(Map( + init(Map( "spark.mesos.executor.docker.image" -> "test", "spark.mesos.containerizer" -> "mesos")) @@ -362,7 +365,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("docker settings are reflected in created tasks") { - setBackend(Map( + init(Map( "spark.mesos.executor.docker.image" -> "some_image", "spark.mesos.executor.docker.forcePullImage" -> "true", "spark.mesos.executor.docker.volumes" -> "/host_vol:/container_vol:ro", @@ -400,7 +403,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("force-pull-image option is disabled by default") { - setBackend(Map( + init(Map( "spark.mesos.executor.docker.image" -> "some_image" )) @@ -423,7 +426,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports spark.executor.uri") { val url = "spark.spark.spark.com" - setBackend(Map( + init(Map( "spark.executor.uri" -> url ), null) @@ -438,7 +441,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports setting fetcher cache") { val url = "spark.spark.spark.com" - setBackend(Map( + init(Map( "spark.mesos.fetcherCache.enable" -> "true", "spark.executor.uri" -> url ), null) @@ -452,7 +455,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports disabling fetcher cache") { val url = "spark.spark.spark.com" - setBackend(Map( + init(Map( "spark.mesos.fetcherCache.enable" -> "false", "spark.executor.uri" -> url ), null) @@ -522,7 +525,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos supports spark.mesos.network.name") { - setBackend(Map( + init(Map( "spark.mesos.network.name" -> "test-network-name" )) @@ -539,7 +542,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("supports spark.scheduler.minRegisteredResourcesRatio") { val expectedCores = 1 - setBackend(Map( + init(Map( "spark.cores.max" -> expectedCores.toString, "spark.scheduler.minRegisteredResourcesRatio" -> "1.0")) @@ -552,6 +555,17 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite assert(backend.isReady) } + test("start() sets spark.mesos.kerberos.userCredentials") { + init() + + assert(backend + .driverEndpoint + .askSync[SparkAppConfig](RetrieveSparkAppConfig) + .sparkProperties + .toMap + .contains(config.USER_CREDENTIALS.key)) + } + private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) private def registerMockExecutor(executorId: String, slaveId: String, cores: Integer) = { @@ -587,27 +601,41 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite .build } - private def createSchedulerBackend( - taskScheduler: TaskSchedulerImpl, - driver: SchedulerDriver, - shuffleClient: MesosExternalShuffleClient) = { - val securityManager = mock[SecurityManager] + private def init( + properties: Map[String, String] = null, + home: String = "/path"): Unit = { - val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager) { - override protected def createSchedulerDriver( - masterUrl: String, - scheduler: Scheduler, - sparkUser: String, - appName: String, - conf: SparkConf, - webuiUrl: Option[String] = None, - checkpoint: Option[Boolean] = None, - failoverTimeout: Option[Double] = None, - frameworkId: Option[String] = None): SchedulerDriver = driver + sparkConf = getSparkConf(properties, home) + sc = new SparkContext(sparkConf) + + driver = mock[SchedulerDriver] + when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - override protected def getShuffleClient(): MesosExternalShuffleClient = shuffleClient + taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.sc).thenReturn(sc) + + externalShuffleClient = mock[MesosExternalShuffleClient] + + security = spy(new MesosSecurityManager) + when(security.isSecurityEnabled).thenReturn(true) + + val securityManager = mock[SecurityManager] + backend = new MesosCoarseGrainedSchedulerBackend( + taskScheduler, sc, "master", securityManager, security) { + override protected def createSchedulerDriver( + masterUrl: String, + scheduler: Scheduler, + sparkUser: String, + appName: String, + conf: SparkConf, + webuiUrl: Option[String] = None, + checkpoint: Option[Boolean] = None, + failoverTimeout: Option[Double] = None, + frameworkId: Option[String] = None): SchedulerDriver = driver + + override protected def getShuffleClient(): MesosExternalShuffleClient = externalShuffleClient + // override to avoid race condition with the driver thread on `mesosDriver` override def startScheduler(newDriver: SchedulerDriver): Unit = {} @@ -617,13 +645,12 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } backend.start() backend.registered(driver, Utils.TEST_FRAMEWORK_ID, Utils.TEST_MASTER_INFO) - backend } - private def initializeSparkConf( + private def getSparkConf( sparkConfVars: Map[String, String] = null, - home: String = "/path"): Unit = { - sparkConf = (new SparkConf) + home: String = "/path"): SparkConf = { + val sparkConf = (new SparkConf) .setMaster("local[*]") .setAppName("test-mesos-dynamic-alloc") .set("spark.mesos.driver.webui.url", "http://webui") @@ -635,20 +662,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite if (sparkConfVars != null) { sparkConf.setAll(sparkConfVars) } - } - - private def setBackend(sparkConfVars: Map[String, String] = null, home: String = "/path") { - initializeSparkConf(sparkConfVars, home) - sc = new SparkContext(sparkConf) - - driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - externalShuffleClient = mock[MesosExternalShuffleClient] - backend = createSchedulerBackend(taskScheduler, driver, externalShuffleClient) + sparkConf } } From a47c9c04f61dce38f64e291c66793742239761b7 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 17 Apr 2017 17:43:18 -0700 Subject: [PATCH 07/15] info logs --- .../spark/scheduler/cluster/mesos/MesosSecurityManager.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala index 7b3d58440bfb..3d33d2e02672 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala @@ -42,7 +42,7 @@ private[mesos] class MesosSecurityManager extends Logging { userCreds.writeTokenStorageToStream(dataStream) val credsBytes = byteStream.toByteArray - logDebug(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") + logInfo(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") val creds64 = DatatypeConverter.printBase64Binary(credsBytes) conf.set(config.USER_CREDENTIALS, creds64) @@ -53,8 +53,6 @@ private[mesos] class MesosSecurityManager extends Logging { * services. */ private def getDelegationTokens(conf: SparkConf): Credentials = { - logDebug(s"Retrieving delegation tokens.") - val userCreds = UserGroupInformation.getCurrentUser.getCredentials val numTokensBefore = userCreds.numberOfTokens val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) From c8ec0496ca1c12e5eb43c530f08cb033a7c862fa Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Tue, 18 Apr 2017 13:24:11 -0700 Subject: [PATCH 08/15] style --- .../MesosCoarseGrainedSchedulerBackendSuite.scala | 12 ++++++------ .../apache/spark/deploy/yarn/ApplicationMaster.scala | 4 +++- .../scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 2 ++ .../deploy/yarn/security/AMCredentialRenewer.scala | 1 + .../deploy/yarn/security/CredentialUpdater.scala | 2 ++ .../security/HadoopFSCredentialProviderSuite.scala | 3 ++- 7 files changed, 17 insertions(+), 9 deletions(-) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index 71063ac0dfea..d16e0e3a5411 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.deploy.mesos.config import org.apache.spark.internal.config._ import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RemoveExecutor, RetrieveSparkAppConfig, SparkAppConfig} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.{RegisterExecutor, RetrieveSparkAppConfig, SparkAppConfig} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.mesos.Utils._ @@ -91,7 +91,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports killing and relaunching tasks with executors") { init() - + // launches a task on a valid offer val minMem = backend.executorMemory(sc) + 1024 val minCpu = 4 @@ -468,7 +468,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("mesos sets task name to spark.app.name") { - setBackend() + init() val offers = List(Resources(backend.executorMemory(sc), 1)) offerResources(offers) @@ -480,7 +480,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos sets configurable labels on tasks") { val taskLabelsString = "mesos:test,label:test" - setBackend(Map( + init(Map( "spark.mesos.task.labels" -> taskLabelsString )) @@ -503,7 +503,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos ignored invalid labels and sets configurable labels on tasks") { val taskLabelsString = "mesos:test,label:test,incorrect:label:here" - setBackend(Map( + init(Map( "spark.mesos.task.labels" -> taskLabelsString )) @@ -635,7 +635,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite frameworkId: Option[String] = None): SchedulerDriver = driver override protected def getShuffleClient(): MesosExternalShuffleClient = externalShuffleClient - + // override to avoid race condition with the driver thread on `mesosDriver` override def startScheduler(newDriver: SchedulerDriver): Unit = {} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 497cec7e1df7..425c0f390217 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -20,18 +20,20 @@ package org.apache.spark.deploy.yarn import java.io.{File, IOException} import java.lang.reflect.InvocationTargetException import java.net.{Socket, URI, URL} -import java.util.concurrent.{TimeUnit, TimeoutException} +import java.util.concurrent.{TimeoutException, TimeUnit} import scala.collection.mutable.HashMap import scala.concurrent.Promise import scala.concurrent.duration.Duration import scala.util.control.NonFatal + import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException import org.apache.hadoop.yarn.util.{ConverterUtils, Records} + import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer 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 45945134c3fc..5c60a8bc31bc 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 @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.{File, FileOutputStream, IOException, OutputStreamWriter} -import java.net.{InetAddress, URI, UnknownHostException} +import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets import java.util.{Locale, Properties, UUID} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index adcd7b1d8ddd..8812365d4321 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -22,6 +22,7 @@ import java.util.regex.Matcher import java.util.regex.Pattern import scala.collection.mutable.{HashMap, ListBuffer} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.JobConf @@ -31,6 +32,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils + import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.ConfigurableCredentialManager diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala index 9d3fe003cd7c..9a3bcb4f4d4f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{Executors, TimeUnit} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.UserGroupInformation + import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.ConfigurableCredentialManager diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala index 0b1573001ecb..57fdd0d2a282 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala @@ -20,9 +20,11 @@ package org.apache.spark.deploy.yarn.security import java.util.concurrent.{Executors, TimeUnit} import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.security.{Credentials, UserGroupInformation} + import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.ConfigurableCredentialManager diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala index 770ec6e0434f..eee01b563002 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala @@ -18,9 +18,10 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration -import org.apache.spark.deploy.security.HadoopFSCredentialProvider import org.scalatest.{Matchers, PrivateMethodTester} + import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.deploy.security.HadoopFSCredentialProvider class HadoopFSCredentialProviderSuite extends SparkFunSuite From 954eeffda336bbbf6d5a588a38c95f092ecf1679 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Tue, 18 Apr 2017 14:34:14 -0700 Subject: [PATCH 09/15] Re-add org.apache.spark.deploy.yarn.security.ServiceCredentialProvider for backwards compatibility --- .../ConfigurableCredentialManager.scala | 18 ++++--- .../spark/deploy/yarn/ApplicationMaster.scala | 5 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 4 +- .../yarn/security/AMCredentialRenewer.scala | 3 +- .../yarn/security/CredentialUpdater.scala | 3 +- .../security/ServiceCredentialProvider.scala | 22 ++++++++ .../YARNConfigurableCredentialManager.scala | 51 +++++++++++++++++++ 8 files changed, 93 insertions(+), 17 deletions(-) create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala create mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala index 24fb35de948a..dfcbfc9c08fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala @@ -41,15 +41,17 @@ import org.apache.spark.util.Utils * For example, Hive's credential provider [[HiveCredentialProvider]] can be enabled/disabled by * the configuration spark.yarn.security.credentials.hive.enabled. */ -private[spark] final class ConfigurableCredentialManager( +private[spark] class ConfigurableCredentialManager( sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled" private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled" // Maintain all the registered credential providers - private val credentialProviders = { - val providers = ServiceLoader.load(classOf[ServiceCredentialProvider], - Utils.getContextOrSparkClassLoader).asScala + private val credentialProviders = getCredentialProviders() + logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") + + private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { + val providers = loadCredentialProviders // Filter out credentials in which spark.yarn.security.credentials.{service}.enabled is false. providers.filter { p => @@ -64,9 +66,12 @@ private[spark] final class ConfigurableCredentialManager( }.map { p => (p.serviceName, p) }.toMap } - logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") + protected def loadCredentialProviders: List[ServiceCredentialProvider] = { + ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) + .asScala.toList + } - /** + /** * Get credential provider for the specified service. */ def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = { @@ -76,6 +81,7 @@ private[spark] final class ConfigurableCredentialManager( /** * Writes delegation tokens to creds. Delegation tokens are fetched from all registered * providers. + * * @return nearest time of next renewal, Long.MaxValue if all the credentials aren't renewable, * otherwise the nearest renewal time of any credentials will be returned. */ diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 425c0f390217..a2248d91d8b8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -37,9 +37,8 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer -import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.AMCredentialRenewer +import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, YARNConfigurableCredentialManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rpc._ @@ -248,7 +247,7 @@ private[spark] class ApplicationMaster( if (sparkConf.contains(CREDENTIALS_FILE_PATH.key)) { // If a principal and keytab have been set, use that to create new credentials for executors // periodically - val credentialManager = new ConfigurableCredentialManager(sparkConf, yarnConf) + val credentialManager = new YARNConfigurableCredentialManager(sparkConf, yarnConf) val credentialRenewer = new AMCredentialRenewer(sparkConf, yarnConf, credentialManager) credentialRenewer.scheduleLoginFromKeytab() } 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 5c60a8bc31bc..537a43c355cd 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 @@ -48,8 +48,8 @@ import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.deploy.yarn.security.YARNConfigurableCredentialManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} @@ -121,7 +121,7 @@ private[spark] class Client( private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) } .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) - private val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + private val credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 8812365d4321..da4f5edad001 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -35,8 +35,8 @@ import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.security.CredentialUpdater +import org.apache.spark.deploy.yarn.security.YARNConfigurableCredentialManager import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.util.Utils @@ -89,7 +89,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = { val hadoopConf = newConfiguration(sparkConf) - val credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + val credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) credentialUpdater.start() } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala index 9a3bcb4f4d4f..e81d072c5ff7 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -25,7 +25,6 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging @@ -55,7 +54,7 @@ import org.apache.spark.util.ThreadUtils private[yarn] class AMCredentialRenewer( sparkConf: SparkConf, hadoopConf: Configuration, - credentialManager: ConfigurableCredentialManager) extends Logging { + credentialManager: YARNConfigurableCredentialManager) extends Logging { private var lastCredentialsFileSuffix = 0 diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala index 57fdd0d2a282..11c046052c2b 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/CredentialUpdater.scala @@ -27,7 +27,6 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.util.{ThreadUtils, Utils} @@ -35,7 +34,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class CredentialUpdater( sparkConf: SparkConf, hadoopConf: Configuration, - credentialManager: ConfigurableCredentialManager) extends Logging { + credentialManager: YARNConfigurableCredentialManager) extends Logging { @volatile private var lastCredentialsFileSuffix = 0 diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala new file mode 100644 index 000000000000..ce33f834272c --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn.security + +@deprecated("Use org.apache.spark.deploy.security.ServiceCredentialProvider", "2.3.0") +trait ServiceCredentialProvider + extends org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala new file mode 100644 index 000000000000..9c5836f5205a --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala @@ -0,0 +1,51 @@ +/* + * 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.yarn.security + +import java.util.ServiceLoader + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.security.ConfigurableCredentialManager +import org.apache.spark.util.Utils + +/** + * This class exists for backwards compatibility. It loads services registered under the + * deprecated [[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider]]. + */ +private[yarn] class YARNConfigurableCredentialManager( + sparkConf: SparkConf, + hadoopConf: Configuration) + extends ConfigurableCredentialManager(sparkConf, hadoopConf) { + + override def loadCredentialProviders: + List[org.apache.spark.deploy.security.ServiceCredentialProvider] = { + val superProviders = super.loadCredentialProviders + val yarnProviders = ServiceLoader.load( + classOf[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider], + Utils.getContextOrSparkClassLoader) + .asScala + .toList + + superProviders ++ yarnProviders + } + +} From 2d769287edd2ac6867e9696798c116fdf9165411 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Tue, 18 Apr 2017 14:43:56 -0700 Subject: [PATCH 10/15] move YARNHadoopFSCredentialProviderSuite --- .../deploy/security/HadoopFSCredentialProvider.scala | 8 +++++--- ...scala => YARNHadoopFSCredentialProviderSuite.scala} | 10 +++++----- 2 files changed, 10 insertions(+), 8 deletions(-) rename resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/{HadoopFSCredentialProviderSuite.scala => YARNHadoopFSCredentialProviderSuite.scala} (85%) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala index 334fe02d23db..489554a925c5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala @@ -69,13 +69,15 @@ private[deploy] class HadoopFSCredentialProvider nextRenewalDate } - def getTokenRenewalInterval(hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = None + protected def getTokenRenewalInterval( + hadoopConf: Configuration, + sparkConf: SparkConf): Option[Long] = None - def getTokenRenewer(hadoopConf: Configuration): String = { + protected def getTokenRenewer(hadoopConf: Configuration): String = { UserGroupInformation.getCurrentUser.getShortUserName } - def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { + protected def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { Set(FileSystem.get(hadoopConf).getHomeDirectory) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala similarity index 85% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala index eee01b563002..31c8b847a2ba 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala @@ -21,26 +21,26 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.{Matchers, PrivateMethodTester} import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.deploy.security.HadoopFSCredentialProvider +import org.apache.spark.deploy.yarn.security.YARNHadoopFSCredentialProvider -class HadoopFSCredentialProviderSuite +class YARNHadoopFSCredentialProviderSuite extends SparkFunSuite with PrivateMethodTester with Matchers { private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer) private def getTokenRenewer( - fsCredentialProvider: HadoopFSCredentialProvider, conf: Configuration): String = { + fsCredentialProvider: YARNHadoopFSCredentialProvider, conf: Configuration): String = { fsCredentialProvider invokePrivate _getTokenRenewer(conf) } - private var hadoopFsCredentialProvider: HadoopFSCredentialProvider = null + private var hadoopFsCredentialProvider: YARNHadoopFSCredentialProvider = null override def beforeAll() { super.beforeAll() if (hadoopFsCredentialProvider == null) { - hadoopFsCredentialProvider = new HadoopFSCredentialProvider() + hadoopFsCredentialProvider = new YARNHadoopFSCredentialProvider() } } From d8a968d66c577cc702d00e980c968a57c3f12565 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 19 Apr 2017 10:35:03 -0700 Subject: [PATCH 11/15] Move hive test deps to the core module --- core/pom.xml | 25 +++++++++++++++++++ ....deploy.security.ServiceCredentialProvider | 0 resource-managers/yarn/pom.xml | 25 +------------------ 3 files changed, 26 insertions(+), 24 deletions(-) rename {resource-managers/mesos => core}/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider (100%) diff --git a/core/pom.xml b/core/pom.xml index 24ce36deeb16..828d7727a969 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -357,6 +357,31 @@ org.apache.commons commons-crypto + + + + ${hive.group} + hive-exec + test + + + ${hive.group} + hive-metastore + test + + + org.apache.thrift + libthrift + test + + + org.apache.thrift + libfb303 + test + target/scala-${scala.binary.version}/classes diff --git a/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider similarity index 100% rename from resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider rename to core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index a1b641c8eeb8..6d2af5e02b43 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -167,30 +167,7 @@ ${jersey-1.version} - - - ${hive.group} - hive-exec - test - - - ${hive.group} - hive-metastore - test - - - org.apache.thrift - libthrift - test - - - org.apache.thrift - libfb303 - test - + From b8093c863ce9af3eadc3fd2b371e1bafe4cf4a47 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 19 Apr 2017 15:10:25 -0700 Subject: [PATCH 12/15] remove test scope --- core/pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 828d7727a969..7c8f3de75ffc 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -365,12 +365,11 @@ ${hive.group} hive-exec - test ${hive.group} hive-metastore - test + org.apache.thrift From 25d508823d238d905b102196962f39900b5c526a Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 19 Apr 2017 15:50:10 -0700 Subject: [PATCH 13/15] remove test scope --- core/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 7c8f3de75ffc..2b47d9aada2b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -369,7 +369,6 @@ ${hive.group} hive-metastore - org.apache.thrift From 4c387ebcb584732d0d67e83c0b9d5f4cfd1db247 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 20 Apr 2017 15:15:51 -0700 Subject: [PATCH 14/15] Removed MesosSecurityManager, added RPC call, removed META-INF ServiceCredentialProvider from core --- core/pom.xml | 5 ++ .../ConfigurableCredentialManager.scala | 65 +++++++++++++----- .../security/CredentialsSerializer.scala | 39 +++++++++++ .../CoarseGrainedExecutorBackend.scala | 23 ++----- .../cluster/CoarseGrainedClusterMessage.scala | 5 +- .../CoarseGrainedSchedulerBackend.scala | 25 +++++-- .../cluster/StandaloneSchedulerBackend.scala | 2 +- .../apache/spark/HeartbeatReceiverSuite.scala | 2 +- .../ConfigurableCredentialManagerSuite.scala | 7 -- docs/configuration.md | 6 +- docs/running-on-yarn.md | 12 ++-- ....deploy.security.ServiceCredentialProvider | 0 .../cluster/mesos/MesosClusterManager.scala | 3 +- .../MesosCoarseGrainedSchedulerBackend.scala | 14 ++-- .../cluster/mesos/MesosSecurityManager.scala | 66 ------------------- ...osCoarseGrainedSchedulerBackendSuite.scala | 9 +-- .../cluster/YarnSchedulerBackend.scala | 2 +- .../ConfigurableCredentialManagerSuite.scala | 54 +++++++++++++++ 18 files changed, 200 insertions(+), 139 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala rename {core => resource-managers/mesos}/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider (100%) delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala create mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 2b47d9aada2b..d96841008ae3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -380,6 +380,11 @@ libfb303 test + + + org.apache.hadoop + hadoop-yarn-api + target/scala-${scala.binary.version}/classes diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala index dfcbfc9c08fe..2d62fb0339de 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala @@ -22,7 +22,7 @@ import java.util.ServiceLoader import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -37,14 +37,19 @@ import org.apache.spark.util.Utils * interface and put into resources/META-INF/services to be loaded by ServiceLoader. * * Also each credential provider is controlled by - * spark.yarn.security.credentials.{service}.enabled, it will not be loaded in if set to false. + * spark.security.credentials.{service}.enabled, it will not be loaded in if set to false. * For example, Hive's credential provider [[HiveCredentialProvider]] can be enabled/disabled by - * the configuration spark.yarn.security.credentials.hive.enabled. + * the configuration spark.security.credentials.hive.enabled. */ private[spark] class ConfigurableCredentialManager( - sparkConf: SparkConf, hadoopConf: Configuration) extends Logging { - private val deprecatedProviderEnabledConfig = "spark.yarn.security.tokens.%s.enabled" - private val providerEnabledConfig = "spark.yarn.security.credentials.%s.enabled" + sparkConf: SparkConf, + hadoopConf: Configuration) + extends Logging { + + private val deprecatedProviderEnabledConfigs = List( + "spark.yarn.security.tokens.%s.enabled", + "spark.yarn.security.credentials.%s.enabled") + private val providerEnabledConfig = "spark.security.credentials.%s.enabled" // Maintain all the registered credential providers private val credentialProviders = getCredentialProviders() @@ -53,16 +58,30 @@ private[spark] class ConfigurableCredentialManager( private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { val providers = loadCredentialProviders - // Filter out credentials in which spark.yarn.security.credentials.{service}.enabled is false. + // Filter out credentials in which spark.security.credentials.{service}.enabled is false. providers.filter { p => - sparkConf.getOption(providerEnabledConfig.format(p.serviceName)) - .orElse { - sparkConf.getOption(deprecatedProviderEnabledConfig.format(p.serviceName)).map { c => - logWarning(s"${deprecatedProviderEnabledConfig.format(p.serviceName)} is deprecated, " + - s"using ${providerEnabledConfig.format(p.serviceName)} instead") - c - } - }.map(_.toBoolean).getOrElse(true) + + val key = providerEnabledConfig.format(p) + + deprecatedProviderEnabledConfigs.foreach { pattern => + val deprecatedKey = pattern.format(p.serviceName) + if (sparkConf.contains(deprecatedKey)) { + logWarning(s"${deprecatedKey} is deprecated, using ${key} instead") + } + } + + val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => + sparkConf + .getOption(pattern.format(p.serviceName)) + .map(_.toBoolean) + .getOrElse(true) + } + + sparkConf + .getOption(key) + .map(_.toBoolean) + .getOrElse(isEnabledDeprecated) + }.map { p => (p.serviceName, p) }.toMap } @@ -71,7 +90,7 @@ private[spark] class ConfigurableCredentialManager( .asScala.toList } - /** + /** * Get credential provider for the specified service. */ def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = { @@ -96,4 +115,18 @@ private[spark] class ConfigurableCredentialManager( } }.foldLeft(Long.MaxValue)(math.min) } + + /** + * Returns a copy of the current user's credentials, augmented with new delegation tokens. + */ + def obtainUserCredentials: Credentials = { + val userCreds = UserGroupInformation.getCurrentUser.getCredentials + val numTokensBefore = userCreds.numberOfTokens + obtainCredentials(hadoopConf, userCreds) + + logDebug(s"Fetched ${userCreds.numberOfTokens - numTokensBefore} delegation token(s).") + + userCreds + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala b/core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala new file mode 100644 index 000000000000..e74a4820b5ee --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala @@ -0,0 +1,39 @@ +/* + * 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.security + +import java.io.{ByteArrayOutputStream, DataOutputStream} + +import org.apache.hadoop.security.Credentials + +class CredentialsSerializer { + def serializeTokens(creds: Credentials): Array[Byte] = { + val byteStream = new ByteArrayOutputStream + val dataStream = new DataOutputStream(byteStream) + creds.writeTokenStorageToStream(dataStream) + byteStream.toByteArray + } + + def deserializeTokens(tokenBytes: Array[Byte]): Credentials = { + val tokensBuf = new java.io.ByteArrayInputStream(tokenBytes) + + val creds = new Credentials() + creds.readTokenStorageStream(new java.io.DataInputStream(tokensBuf)) + creds + } +} diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index f77795abe0a6..3fc27dbd1352 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,6 @@ import java.net.URL import java.nio.ByteBuffer import java.util.Locale import java.util.concurrent.atomic.AtomicBoolean -import javax.xml.bind.DatatypeConverter import scala.collection.mutable import scala.util.{Failure, Success} @@ -32,6 +31,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.CredentialsSerializer import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.internal.Logging import org.apache.spark.rpc._ @@ -177,21 +177,12 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { - private def addMesosDelegationTokens(driverConf: SparkConf) { - val value = driverConf.get("spark.mesos.kerberos.userCredentials") - val tokens = DatatypeConverter.parseBase64Binary(value) + private def addDelegationTokens(tokenBytes: Array[Byte], driverConf: SparkConf) { + val creds = new CredentialsSerializer().deserializeTokens(tokenBytes) - logDebug(s"Found delegation tokens of ${tokens.length} bytes.") + logInfo(s"Adding ${creds.numberOfTokens()} tokens and ${creds.numberOfSecretKeys()} secret" + + s"keys to the current user's credentials.") - // Use tokens for HDFS login. - val hadoopConf = SparkHadoopUtil.get.newConfiguration(driverConf) - hadoopConf.set("hadoop.security.authentication", "Token") - UserGroupInformation.setConfiguration(hadoopConf) - - // Decode tokens and add them to the current user's credentials. - val creds = UserGroupInformation.getCurrentUser.getCredentials - val tokensBuf = new java.io.ByteArrayInputStream(tokens) - creds.readTokenStorageStream(new java.io.DataInputStream(tokensBuf)) UserGroupInformation.getCurrentUser.addCredentials(creds) } @@ -241,9 +232,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { SparkHadoopUtil.get.startCredentialUpdater(driverConf) } - if (driverConf.contains("spark.mesos.kerberos.userCredentials")) { - addMesosDelegationTokens(driverConf) - } + cfg.ugiTokens.foreach(addDelegationTokens(_, driverConf)) val env = SparkEnv.createExecutorEnv( driverConf, executorId, hostname, port, cores, cfg.ioEncryptionKey, isLocal = false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 6b49bd699a13..02dd8dd24869 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer +import org.apache.hadoop.security.Credentials + import org.apache.spark.TaskState.TaskState import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.ExecutorLossReason @@ -32,7 +34,8 @@ private[spark] object CoarseGrainedClusterMessages { case class SparkAppConfig( sparkProperties: Seq[(String, String)], - ioEncryptionKey: Option[Array[Byte]]) + ioEncryptionKey: Option[Array[Byte]], + ugiTokens: Option[Array[Byte]]) extends CoarseGrainedClusterMessage case object RetrieveLastAllocatedExecutorId extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 4eedaaea6119..9c4c46de177f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -23,9 +23,11 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future -import scala.concurrent.duration.Duration + +import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} +import org.apache.spark.deploy.security.{ConfigurableCredentialManager, CredentialsSerializer} import org.apache.spark.internal.Logging import org.apache.spark.rpc._ import org.apache.spark.scheduler._ @@ -42,7 +44,10 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils, Utils} * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) +class CoarseGrainedSchedulerBackend( + scheduler: TaskSchedulerImpl, + val rpcEnv: RpcEnv, + credentialManager: Option[ConfigurableCredentialManager]) extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -92,6 +97,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 + // Hadoop delegation tokens to send executors. + private val userTokens = if (UserGroupInformation.isSecurityEnabled) { + credentialManager.map { manager => + new CredentialsSerializer().serializeTokens(manager.obtainUserCredentials) + } + } else { + None + } + class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) extends ThreadSafeRpcEndpoint with Logging { @@ -216,8 +230,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp context.reply(true) case RetrieveSparkAppConfig => - val reply = SparkAppConfig(sparkProperties, - SparkEnv.get.securityManager.getIOEncryptionKey()) + val reply = SparkAppConfig( + sparkProperties, + SparkEnv.get.securityManager.getIOEncryptionKey(), + userTokens + ) context.reply(reply) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 0529fe9eed4d..96db24ef5af1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -38,7 +38,7 @@ private[spark] class StandaloneSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, masters: Array[String]) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv, None) with StandaloneAppClientListener with Logging { diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 88916488c0de..14d0575a13cc 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -268,7 +268,7 @@ private class FakeSchedulerBackend( scheduler: TaskSchedulerImpl, rpcEnv: RpcEnv, clusterManagerEndpoint: RpcEndpointRef) - extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv, None) { protected override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = { clusterManagerEndpoint.ask[Boolean]( diff --git a/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala index 428a30c77a39..0838aadb1021 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala @@ -35,13 +35,6 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit sparkConf = new SparkConf() hadoopConf = new Configuration() - System.setProperty("SPARK_YARN_MODE", "true") - } - - override def afterAll(): Unit = { - System.clearProperty("SPARK_YARN_MODE") - - super.afterAll() } test("Correctly load default credential providers") { diff --git a/docs/configuration.md b/docs/configuration.md index 2687f542b8bd..0f8a4254b796 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -685,7 +685,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedJobs 1000 - How many jobs the Spark UI and status APIs remember before garbage collecting. + How many jobs the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. @@ -693,7 +693,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedStages 1000 - How many stages the Spark UI and status APIs remember before garbage collecting. + How many stages the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. @@ -701,7 +701,7 @@ Apart from these, the following properties are also available, and may be useful spark.ui.retainedTasks 100000 - How many tasks the Spark UI and status APIs remember before garbage collecting. + How many tasks the Spark UI and status APIs remember before garbage collecting. This is a target maximum, and fewer elements may be retained in some circumstances. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index eaa320efe4f6..07415952fec5 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -426,7 +426,7 @@ To use a custom metrics.properties for the application master and executors, upd - spark.yarn.security.credentials.${service}.enabled + spark.security.credentials.${service}.enabled true Controls whether to obtain credentials for services when security is enabled. @@ -489,11 +489,11 @@ token for the cluster's default Hadoop filesystem, and potentially for HBase and An HBase token will be obtained if HBase is in on classpath, the HBase configuration declares the application is secure (i.e. `hbase-site.xml` sets `hbase.security.authentication` to `kerberos`), -and `spark.yarn.security.credentials.hbase.enabled` is not set to `false`. +and `spark.security.credentials.hbase.enabled` is not set to `false`. Similarly, a Hive token will be obtained if Hive is on the classpath, its configuration includes a URI of the metadata store in `"hive.metastore.uris`, and -`spark.yarn.security.credentials.hive.enabled` is not set to `false`. +`spark.security.credentials.hive.enabled` is not set to `false`. If an application needs to interact with other secure Hadoop filesystems, then the tokens needed to access these clusters must be explicitly requested at @@ -507,7 +507,7 @@ Spark supports integrating with other security-aware services through Java Servi `java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.security.ServiceCredentialProvider` should be available to Spark by listing their names in the corresponding file in the jar's `META-INF/services` directory. These plug-ins can be disabled by setting -`spark.yarn.security.credentials.{service}.enabled` to `false`, where `{service}` is the name of +`spark.security.credentials.{service}.enabled` to `false`, where `{service}` is the name of credential provider. ## Configuring the External Shuffle Service @@ -571,8 +571,8 @@ the Spark configuration must be set to disable token collection for the services The Spark configuration must include the lines: ``` -spark.yarn.security.credentials.hive.enabled false -spark.yarn.security.credentials.hbase.enabled false +spark.security.credentials.hive.enabled false +spark.security.credentials.hbase.enabled false ``` The configuration option `spark.yarn.access.hadoopFileSystems` must be unset. diff --git a/core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider b/resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider similarity index 100% rename from core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider rename to resource-managers/mesos/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala index 60cd324bbe26..911a0857917e 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala @@ -48,8 +48,7 @@ private[spark] class MesosClusterManager extends ExternalClusterManager { scheduler.asInstanceOf[TaskSchedulerImpl], sc, mesosUrl, - sc.env.securityManager, - new MesosSecurityManager) + sc.env.securityManager) } else { new MesosFineGrainedSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index 8ed50898f85f..b4da4285bdf9 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -55,9 +55,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, master: String, - securityManager: SecurityManager, - mesosSecurityManager: MesosSecurityManager) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) + securityManager: SecurityManager) + extends CoarseGrainedSchedulerBackend( + scheduler, + sc.env.rpcEnv, + Option(new ConfigurableCredentialManager( + sc.conf, + SparkHadoopUtil.get.newConfiguration(sc.conf)))) with org.apache.mesos.Scheduler with MesosSchedulerUtils { @@ -164,10 +168,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } override def start() { - if (mesosSecurityManager.isSecurityEnabled()) { - mesosSecurityManager.setUGITokens(conf) - } - super.start() val driver = createSchedulerDriver( diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala deleted file mode 100644 index 3d33d2e02672..000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSecurityManager.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import javax.xml.bind.DatatypeConverter - -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.mesos.config -import org.apache.spark.deploy.security.ConfigurableCredentialManager -import org.apache.spark.internal.Logging - - -private[mesos] class MesosSecurityManager extends Logging { - def isSecurityEnabled(): Boolean = { - UserGroupInformation.isSecurityEnabled - } - - /** Writes delegation tokens to spark.mesos.kerberos.ugiTokens */ - def setUGITokens(conf: SparkConf): Unit = { - val userCreds = getDelegationTokens(conf) - - val byteStream = new java.io.ByteArrayOutputStream(1024 * 1024) - val dataStream = new java.io.DataOutputStream(byteStream) - userCreds.writeTokenStorageToStream(dataStream) - val credsBytes = byteStream.toByteArray - - logInfo(s"Writing ${credsBytes.length} bytes to ${config.USER_CREDENTIALS.key}.") - - val creds64 = DatatypeConverter.printBase64Binary(credsBytes) - conf.set(config.USER_CREDENTIALS, creds64) - } - - /** - * Returns the user's credentials, with new delegation tokens added for all configured - * services. - */ - private def getDelegationTokens(conf: SparkConf): Credentials = { - val userCreds = UserGroupInformation.getCurrentUser.getCredentials - val numTokensBefore = userCreds.numberOfTokens - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val credentialManager = new ConfigurableCredentialManager(conf, hadoopConf) - credentialManager.obtainCredentials(hadoopConf, userCreds) - - logDebug(s"Fetched ${userCreds.numberOfTokens - numTokensBefore} delegation token(s).") - - userCreds - } -} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index d16e0e3a5411..1ba05a1aa140 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -53,7 +53,6 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private var backend: MesosCoarseGrainedSchedulerBackend = _ private var externalShuffleClient: MesosExternalShuffleClient = _ private var driverEndpoint: RpcEndpointRef = _ - private var security: MesosSecurityManager = _ @volatile private var stopCalled = false // All 'requests' to the scheduler run immediately on the same thread, so @@ -314,10 +313,9 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) val securityManager = mock[SecurityManager] - val security = new MesosSecurityManager val backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager, security) { + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, @@ -616,13 +614,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite externalShuffleClient = mock[MesosExternalShuffleClient] - security = spy(new MesosSecurityManager) - when(security.isSecurityEnabled).thenReturn(true) - val securityManager = mock[SecurityManager] backend = new MesosCoarseGrainedSchedulerBackend( - taskScheduler, sc, "master", securityManager, security) { + taskScheduler, sc, "master", securityManager) { override protected def createSchedulerDriver( masterUrl: String, scheduler: Scheduler, diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index cbc6e60e839c..12158d0f2d86 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -38,7 +38,7 @@ import org.apache.spark.util.{RpcUtils, ThreadUtils} private[spark] abstract class YarnSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv, None) { override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala new file mode 100644 index 000000000000..f46dd7d2e0f5 --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn.security + +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfter, Matchers} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.security.ConfigurableCredentialManager + +class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { + private var credentialManager: YARNConfigurableCredentialManager = null + private var sparkConf: SparkConf = null + private var hadoopConf: Configuration = null + + override def beforeAll(): Unit = { + super.beforeAll() + + sparkConf = new SparkConf() + hadoopConf = new Configuration() + System.setProperty("SPARK_YARN_MODE", "true") + } + + override def afterAll(): Unit = { + System.clearProperty("SPARK_YARN_MODE") + + super.afterAll() + } + + test("Correctly load YARNHadoopFSCredentialProvider") { + credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) + + assert(credentialManager + .getServiceCredentialProvider("hadoopfs") + .get + .isInstanceOf[YARNHadoopFSCredentialProvider]) + } + +} From e32afeeac95883138751c060a3ebfaf309e3d22f Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 20 Apr 2017 15:17:37 -0700 Subject: [PATCH 15/15] add InterfaceStability annotation to ServiceCredentialProvider --- .../spark/deploy/security/ServiceCredentialProvider.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala index 667960c0f43b..9016c256be64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala @@ -19,13 +19,14 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} - import org.apache.spark.SparkConf +import org.apache.spark.annotation.InterfaceStability /** * A credential provider for a service. User must implement this if they need to access a * secure service from Spark. */ +@InterfaceStability.Unstable trait ServiceCredentialProvider { /**