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/40] [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 bae7a3f307f52..33e4a4dd84f31 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/40] [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 77ea40cc37946..87aecc05e5e08 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 77005aa9040b5..445e31cb875ea 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 0144fd1056bac..6a68382b8709b 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 b2b26ee107c00..1d1c632cc25b6 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 6b49bd699a13a..cc90132bbc880 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 4eedaaea61195..1f96223673eb3 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 0000000000000..06e1d36b41430 --- /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 6767cc5079649..930d7d465135d 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 1bc6f71860c3f..d2e2bdb38b6fd 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 2a36ec4fa8112..90e05e05c847b 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 9d81025a3016b..2f83b20736534 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/40] 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 87aecc05e5e08..77ea40cc37946 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 33e4a4dd84f31..58558673bc2ec 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 445e31cb875ea..886602c047919 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 4f4be52a0d691..24fb35de948a1 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 5adeb8e605ff4..29b0b9e9210dd 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 0000000000000..334fe02d23dba --- /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 16d8fc32bb42d..d8d4cccbd0857 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 4e3fcce8dbb1d..e9a58be0dab3d 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 1d1c632cc25b6..02318c9b7d424 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 cc90132bbc880..6b49bd699a13a 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 1f96223673eb3..4eedaaea61195 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 06e1d36b41430..0000000000000 --- 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 2355d40d1e6fe..4ae5b0b083241 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 e9ddaa76a797f..eaa320efe4f64 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 930d7d465135d..6767cc5079649 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 0000000000000..9ffeb4d500296 --- /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 19e253394f1b2..58a1c12f2de31 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 90e05e05c847b..f166456ed8974 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 2f83b20736534..693cce629858d 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 0000000000000..f36407942a08d --- /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 f5a807ecac9d7..0000000000000 --- 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 864c834d110fd..497cec7e1df7e 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 424bbca123190..8b80077b39600 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 93578855122cd..adcd7b1d8ddd3 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 7e76f402db249..9d3fe003cd7c1 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 41b7b5d60b038..0b1573001ecbd 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 f65c886db944e..76e308372c8d1 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 b0067aa4517c7..c4c3e061fadf7 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 f50ee193c258f..770ec6e0434f7 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/40] 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 58558673bc2ec..2a765e1286203 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 886602c047919..e6e59fdf127e8 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 6a68382b8709b..0144fd1056bac 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 e9a58be0dab3d..667960c0f43b5 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 02318c9b7d424..f77795abe0a64 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 c4c3e061fadf7..428a30c77a39c 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 58a1c12f2de31..78d2de6ba5378 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 d2e2bdb38b6fd..af1430e5914ef 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 f166456ed8974..68650400e084b 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 693cce629858d..514491d762224 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 8b80077b39600..45945134c3fc3 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 d0ef5efa36e86..2676a0ad589fa 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/40] 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 2a765e1286203..bae7a3f307f52 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 78d2de6ba5378..55ef6ea382815 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 af1430e5914ef..1bc6f71860c3f 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/40] 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 911a0857917ef..60cd324bbe267 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 68650400e084b..8ed50898f85f2 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 0000000000000..7b3d58440bfbe --- /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 c040f05d93b3a..71063ac0dfea6 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/40] 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 7b3d58440bfbe..3d33d2e026729 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/40] 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 71063ac0dfea6..d16e0e3a54110 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 497cec7e1df7e..425c0f3902173 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 45945134c3fc3..5c60a8bc31bc3 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 adcd7b1d8ddd3..8812365d43217 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 9d3fe003cd7c1..9a3bcb4f4d4fb 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 0b1573001ecbd..57fdd0d2a282f 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 770ec6e0434f7..eee01b563002f 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/40] 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 24fb35de948a1..dfcbfc9c08fef 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 425c0f3902173..a2248d91d8b8a 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 5c60a8bc31bc3..537a43c355cd8 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 8812365d43217..da4f5edad0018 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 9a3bcb4f4d4fb..e81d072c5ff7a 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 57fdd0d2a282f..11c046052c2b7 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 0000000000000..ce33f834272c8 --- /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 0000000000000..9c5836f5205ab --- /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/40] 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 334fe02d23dba..489554a925c57 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 eee01b563002f..31c8b847a2ba0 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/40] 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 24ce36deeb169..828d7727a9699 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 a1b641c8eeb84..6d2af5e02b43d 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/40] 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 828d7727a9699..7c8f3de75ffc2 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/40] remove test scope --- core/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 7c8f3de75ffc2..2b47d9aada2be 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/40] 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 2b47d9aada2be..d96841008ae34 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 dfcbfc9c08fef..2d62fb0339ded 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 0000000000000..e74a4820b5ee8 --- /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 f77795abe0a64..3fc27dbd1352c 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 6b49bd699a13a..02dd8dd24869d 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 4eedaaea61195..9c4c46de177fb 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 0529fe9eed4da..96db24ef5af1b 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 88916488c0def..14d0575a13cc7 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 428a30c77a39c..0838aadb1021c 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 2687f542b8bd3..0f8a4254b7961 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 eaa320efe4f64..07415952fec52 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 60cd324bbe267..911a0857917ef 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 8ed50898f85f2..b4da4285bdf9e 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 3d33d2e026729..0000000000000 --- 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 d16e0e3a54110..1ba05a1aa1400 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 cbc6e60e839c1..12158d0f2d867 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 0000000000000..f46dd7d2e0f57 --- /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/40] 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 667960c0f43b5..9016c256be643 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 { /** From be69f5a639caad0abadafcae471e71847fc9f935 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 20 Apr 2017 18:00:52 -0700 Subject: [PATCH 16/40] Add HadoopAccessManager --- ....deploy.security.ServiceCredentialProvider | 2 +- .../ConfigurableCredentialManager.scala | 60 ++++++++++++------- .../security/DefaultHadoopAccessManager.scala | 38 ++++++++++++ .../security/HBaseCredentialProvider.scala | 2 +- .../deploy/security/HadoopAccessManager.scala | 37 ++++++++++++ .../security/HadoopFSCredentialProvider.scala | 25 ++------ .../security/HiveCredentialProvider.scala | 2 +- .../security/ServiceCredentialProvider.scala | 9 +-- .../ConfigurableCredentialManagerSuite.scala | 12 +++- .../MesosCoarseGrainedSchedulerBackend.scala | 4 +- ....deploy.security.ServiceCredentialProvider | 3 - .../security/ServiceCredentialProvider.scala | 43 ++++++++++++- .../YARNConfigurableCredentialManager.scala | 46 ++++++++++---- ...er.scala => YARNHadoopAccessManager.scala} | 18 +++--- .../ConfigurableCredentialManagerSuite.scala | 5 +- .../YARNHadoopFSCredentialProviderSuite.scala | 8 +-- 16 files changed, 229 insertions(+), 85 deletions(-) rename {resource-managers/mesos/src/main/resources/META-INF/services => core/src/main/resources/META-INF.services}/org.apache.spark.deploy.security.ServiceCredentialProvider (63%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala delete mode 100644 resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider rename resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/{YARNHadoopFSCredentialProvider.scala => YARNHadoopAccessManager.scala} (84%) 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 63% 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 index 9ffeb4d500296..791a0c24ed4cf 100644 --- 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 @@ -1,3 +1,3 @@ -org.apache.spark.deploy.security.HadoopFSCredentialProvider +org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider org.apache.spark.deploy.security.HBaseCredentialProvider org.apache.spark.deploy.security.HiveCredentialProvider 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 2d62fb0339ded..17a17ea4d710f 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -43,7 +44,8 @@ import org.apache.spark.util.Utils */ private[spark] class ConfigurableCredentialManager( sparkConf: SparkConf, - hadoopConf: Configuration) + hadoopConf: Configuration, + hadoopAccessManager: HadoopAccessManager) extends Logging { private val deprecatedProviderEnabledConfigs = List( @@ -52,37 +54,48 @@ private[spark] class ConfigurableCredentialManager( private val providerEnabledConfig = "spark.security.credentials.%s.enabled" // Maintain all the registered credential providers - private val credentialProviders = getCredentialProviders() + private val credentialProviders = getCredentialProviders logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") + def this(sparkConf: SparkConf, hadoopConf: Configuration) { + this(sparkConf, hadoopConf, new DefaultHadoopAccessManager(hadoopConf)) + } + + def this(sparkConf: SparkConf) { + this(sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + } + private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { val providers = loadCredentialProviders // Filter out credentials in which spark.security.credentials.{service}.enabled is false. - providers.filter { p => - - val key = providerEnabledConfig.format(p) + providers + .filter(p => isServiceEnabled(p.serviceName)) + .map(p => (p.serviceName, p)) + .toMap + } - deprecatedProviderEnabledConfigs.foreach { pattern => - val deprecatedKey = pattern.format(p.serviceName) - if (sparkConf.contains(deprecatedKey)) { - logWarning(s"${deprecatedKey} is deprecated, using ${key} instead") - } - } + protected def isServiceEnabled(serviceName: String): Boolean = { + val key = providerEnabledConfig.format(serviceName) - val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => - sparkConf - .getOption(pattern.format(p.serviceName)) - .map(_.toBoolean) - .getOrElse(true) + deprecatedProviderEnabledConfigs.foreach { pattern => + val deprecatedKey = pattern.format(serviceName) + if (sparkConf.contains(deprecatedKey)) { + logWarning(s"${deprecatedKey} is deprecated, using ${key} instead") } + } + val isEnabledDeprecated = deprecatedProviderEnabledConfigs.forall { pattern => sparkConf - .getOption(key) + .getOption(pattern.format(serviceName)) .map(_.toBoolean) - .getOrElse(isEnabledDeprecated) + .getOrElse(true) + } - }.map { p => (p.serviceName, p) }.toMap + sparkConf + .getOption(key) + .map(_.toBoolean) + .getOrElse(isEnabledDeprecated) } protected def loadCredentialProviders: List[ServiceCredentialProvider] = { @@ -104,10 +117,12 @@ private[spark] class ConfigurableCredentialManager( * @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. */ - def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = { + def obtainCredentials( + hadoopConf: Configuration, + creds: Credentials): Long = { credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { - provider.obtainCredentials(hadoopConf, sparkConf, creds) + provider.obtainCredentials(hadoopConf, hadoopAccessManager, creds) } else { logDebug(s"Service ${provider.serviceName} does not require a token." + s" Check your configuration to see if security is disabled or not.") @@ -115,7 +130,7 @@ private[spark] class ConfigurableCredentialManager( } }.foldLeft(Long.MaxValue)(math.min) } - + /** * Returns a copy of the current user's credentials, augmented with new delegation tokens. */ @@ -128,5 +143,4 @@ private[spark] class ConfigurableCredentialManager( userCreds } - } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala new file mode 100644 index 0000000000000..7dd37c50ebfb4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path +import org.apache.hadoop.security.UserGroupInformation + +class DefaultHadoopAccessManager(hadoopConf: Configuration) extends HadoopAccessManager { + + def getTokenRenewer: String = { + UserGroupInformation.getCurrentUser.getShortUserName + } + + def hadoopFSsToAccess: Set[Path] = { + Set(FileSystem.get(hadoopConf).getHomeDirectory) + } + + def getTokenRenewalInterval: Option[Long] = { + None + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala index 29b0b9e9210dd..d1486773b96c5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala @@ -34,7 +34,7 @@ private[security] class HBaseCredentialProvider extends ServiceCredentialProvide override def obtainCredentials( hadoopConf: Configuration, - sparkConf: SparkConf, + hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { try { val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala new file mode 100644 index 0000000000000..8d6b4849fd510 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +/** [[HadoopAccessManager]] returns information related to how Hadoop delegation tokens should be + * fetched. + */ +private[spark] trait HadoopAccessManager { + + /** The user allowed to renew delegation tokens */ + def getTokenRenewer: String + + /** The renewal interval, or [[None]] if the token shouldn't be renewed */ + def getTokenRenewalInterval: Option[Long] + + /** The set of hadoop file systems to fetch delegation tokens for */ + def hadoopFSsToAccess: Set[Path] +} + 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 489554a925c57..fe027c9eb5f46 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 @@ -20,11 +20,9 @@ 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.Credentials import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier -import org.apache.spark.SparkConf import org.apache.spark.internal.Logging private[deploy] class HadoopFSCredentialProvider @@ -38,12 +36,13 @@ private[deploy] class HadoopFSCredentialProvider override def obtainCredentials( hadoopConf: Configuration, - sparkConf: SparkConf, + hadoopAccessManager: HadoopAccessManager, 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 tokenRenewer = hadoopAccessManager.getTokenRenewer + hadoopAccessManager.hadoopFSsToAccess.foreach { dst => val dstFs = dst.getFileSystem(hadoopConf) logInfo("getting token for: " + dst) dstFs.addDelegationTokens(tokenRenewer, tmpCreds) @@ -51,7 +50,7 @@ private[deploy] class HadoopFSCredentialProvider // Get the token renewal interval if it is not set. It will only be called once. if (tokenRenewalInterval == null) { - tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf) + tokenRenewalInterval = hadoopAccessManager.getTokenRenewalInterval } // Get the time of next renewal. @@ -68,16 +67,4 @@ private[deploy] class HadoopFSCredentialProvider creds.addAll(tmpCreds) nextRenewalDate } - - protected def getTokenRenewalInterval( - hadoopConf: Configuration, - sparkConf: SparkConf): Option[Long] = None - - protected def getTokenRenewer(hadoopConf: Configuration): String = { - UserGroupInformation.getCurrentUser.getShortUserName - } - - protected def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { - Set(FileSystem.get(hadoopConf).getHomeDirectory) - } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index d8d4cccbd0857..0048fb329e520 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -62,7 +62,7 @@ private[security] class HiveCredentialProvider extends ServiceCredentialProvider override def obtainCredentials( hadoopConf: Configuration, - sparkConf: SparkConf, + hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { val conf = hiveConf(hadoopConf) 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 9016c256be643..46b52f618a466 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,6 +19,7 @@ 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 @@ -46,13 +47,13 @@ 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 hadoopAccessManager HadoopAccessManager * @param creds Credentials to add tokens and security keys to. * @return If this Credential is renewable and can be renewed, return the time of the next * renewal, otherwise None should be returned. */ def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] + hadoopConf: Configuration, + hadoopAccessManager: HadoopAccessManager, + creds: Credentials): Option[Long] } 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 0838aadb1021c..414e65a6fad19 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 @@ -96,7 +96,10 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit val hiveCredentialProvider = new HiveCredentialProvider() val credentials = new Credentials() - hiveCredentialProvider.obtainCredentials(hadoopConf, sparkConf, credentials) + hiveCredentialProvider.obtainCredentials( + hadoopConf, + new DefaultHadoopAccessManager(hadoopConf), + credentials) credentials.getAllTokens.size() should be (0) } @@ -107,7 +110,10 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit val hbaseTokenProvider = new HBaseCredentialProvider() val creds = new Credentials() - hbaseTokenProvider.obtainCredentials(hadoopConf, sparkConf, creds) + hbaseTokenProvider.obtainCredentials( + hadoopConf, + new DefaultHadoopAccessManager(hadoopConf), + creds) creds.getAllTokens.size should be (0) } @@ -123,7 +129,7 @@ class TestCredentialProvider extends ServiceCredentialProvider { override def obtainCredentials( hadoopConf: Configuration, - sparkConf: SparkConf, + hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { if (creds == null) { // Guard out other unit test failures. 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 b4da4285bdf9e..791ddc895ffef 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 @@ -59,9 +59,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( extends CoarseGrainedSchedulerBackend( scheduler, sc.env.rpcEnv, - Option(new ConfigurableCredentialManager( - sc.conf, - SparkHadoopUtil.get.newConfiguration(sc.conf)))) + Option(new ConfigurableCredentialManager(sc.conf))) with org.apache.mesos.Scheduler with MesosSchedulerUtils { 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 deleted file mode 100644 index f36407942a08d..0000000000000 --- a/resource-managers/yarn/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider +++ /dev/null @@ -1,3 +0,0 @@ -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/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala index ce33f834272c8..6cb77adffcb44 100644 --- 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 @@ -17,6 +17,45 @@ package org.apache.spark.deploy.yarn.security +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.SparkConf + +/** + * A credential provider for a service. User must implement this if they need to access a + * secure service from Spark. + */ @deprecated("Use org.apache.spark.deploy.security.ServiceCredentialProvider", "2.3.0") -trait ServiceCredentialProvider - extends org.apache.spark.deploy.security.ServiceCredentialProvider +trait ServiceCredentialProvider { + + /** + * Name of the service to provide credentials. This name should unique, Spark internally will + * use this name to differentiate credential provider. + */ + def serviceName: String + + /** + * To decide whether credential is required for this service. By default it based on whether + * Hadoop security is enabled. + */ + def credentialsRequired(hadoopConf: Configuration): Boolean = { + UserGroupInformation.isSecurityEnabled + } + + /** + * Obtain credentials for this service and get the time of the next renewal. + * + * @param hadoopConf Configuration of current Hadoop Compatible system. + * @param sparkConf SparkConf + * @param creds Credentials to add tokens and security keys to. + * @return If this Credential is renewable and can be renewed, return the time of the next + * renewal, otherwise None should be returned. + */ + def obtainCredentials( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] +} + + 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 index 9c5836f5205ab..443b0d152db99 100644 --- 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 @@ -22,6 +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.spark.SparkConf import org.apache.spark.deploy.security.ConfigurableCredentialManager @@ -32,20 +33,45 @@ import org.apache.spark.util.Utils * 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( + sparkConf: SparkConf, + hadoopConf: Configuration) + extends ConfigurableCredentialManager( + sparkConf, + hadoopConf, + new YARNHadoopAccessManager(hadoopConf, sparkConf)) { + + private val deprecatedCredentialProviders = getDeprecatedCredentialProviders + + def getDeprecatedCredentialProviders: + Map[String, org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = { + val deprecatedProviders = loadDeprecatedCredentialProviders + + deprecatedProviders. + filter(p => isServiceEnabled(p.serviceName)) + .map(p => (p.serviceName, p)) + .toMap + } + + def loadDeprecatedCredentialProviders: + List[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = { + ServiceLoader.load( classOf[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) .asScala .toList - - superProviders ++ yarnProviders } + override def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = { + val superInterval = super.obtainCredentials(hadoopConf, creds) + + deprecatedCredentialProviders.values.flatMap { provider => + if (provider.credentialsRequired(hadoopConf)) { + provider.obtainCredentials(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") + None + } + }.foldLeft(superInterval)(math.min) + } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala similarity index 84% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala rename to resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala index 76e308372c8d1..3a2087764b56d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProvider.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala @@ -28,14 +28,17 @@ 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.security.{HadoopAccessManager} import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -class YARNHadoopFSCredentialProvider extends HadoopFSCredentialProvider { +private[yarn] class YARNHadoopAccessManager( + hadoopConf: Configuration, + sparkConf: SparkConf) extends HadoopAccessManager with Logging { - override def getTokenRenewer(conf: Configuration): String = { - val delegTokenRenewer = Master.getMasterPrincipal(conf) + def getTokenRenewer: String = { + val delegTokenRenewer = Master.getMasterPrincipal(hadoopConf) logDebug("delegation token renewer is: " + delegTokenRenewer) if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { val errorMessage = "Can't get Master Kerberos principal for use as renewer" @@ -46,20 +49,19 @@ class YARNHadoopFSCredentialProvider extends HadoopFSCredentialProvider { delegTokenRenewer } - override def hadoopFSsToAccess(hadoopConf: Configuration, sparkConf: SparkConf): Set[Path] = { + def hadoopFSsToAccess: Set[Path] = { sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet + sparkConf.get(STAGING_DIR).map(new Path(_)) .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory) } - override def getTokenRenewalInterval( - hadoopConf: Configuration, sparkConf: SparkConf): Option[Long] = { + def getTokenRenewalInterval: 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. sparkConf.get(PRINCIPAL).flatMap { renewer => val creds = new Credentials() - hadoopFSsToAccess(hadoopConf, sparkConf).foreach { dst => + hadoopFSsToAccess.foreach { dst => val dstFs = dst.getFileSystem(hadoopConf) dstFs.addDelegationTokens(renewer, creds) } 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 f46dd7d2e0f57..0f65265af8b2b 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 @@ -42,13 +42,12 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit super.afterAll() } - test("Correctly load YARNHadoopFSCredentialProvider") { + test("Correctly load ") { credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) assert(credentialManager .getServiceCredentialProvider("hadoopfs") .get - .isInstanceOf[YARNHadoopFSCredentialProvider]) + .isInstanceOf[YARNHadoopAccessManager]) } - } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala index 31c8b847a2ba0..0e3a2afab909c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.conf.Configuration import org.scalatest.{Matchers, PrivateMethodTester} import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.deploy.yarn.security.YARNHadoopFSCredentialProvider +import org.apache.spark.deploy.yarn.security.YARNHadoopAccessManager class YARNHadoopFSCredentialProviderSuite extends SparkFunSuite @@ -30,17 +30,17 @@ class YARNHadoopFSCredentialProviderSuite private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer) private def getTokenRenewer( - fsCredentialProvider: YARNHadoopFSCredentialProvider, conf: Configuration): String = { + fsCredentialProvider: YARNHadoopAccessManager, conf: Configuration): String = { fsCredentialProvider invokePrivate _getTokenRenewer(conf) } - private var hadoopFsCredentialProvider: YARNHadoopFSCredentialProvider = null + private var hadoopFsCredentialProvider: YARNHadoopAccessManager = null override def beforeAll() { super.beforeAll() if (hadoopFsCredentialProvider == null) { - hadoopFsCredentialProvider = new YARNHadoopFSCredentialProvider() + hadoopFsCredentialProvider = new YARNHadoopAccessManager() } } From 55616da9f0fd15f1594233b5fe43b04ef1c901c8 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 12:28:43 -0700 Subject: [PATCH 17/40] Remove mesos code --- ....deploy.security.ServiceCredentialProvider | 3 - .../org/apache/spark/deploy/SparkSubmit.scala | 18 +- .../ConfigurableCredentialManager.scala | 2 +- .../security/CredentialsSerializer.scala | 39 - .../CoarseGrainedExecutorBackend.scala | 14 - .../cluster/CoarseGrainedClusterMessage.scala | 5 +- .../CoarseGrainedSchedulerBackend.scala | 25 +- .../cluster/StandaloneSchedulerBackend.scala | 2 +- .../apache/spark/deploy/mesos/config.scala | 63 -- .../MesosCoarseGrainedSchedulerBackend.scala | 689 ------------------ .../cluster/mesos/MesosSchedulerUtils.scala | 558 -------------- ...osCoarseGrainedSchedulerBackendSuite.scala | 663 ----------------- 12 files changed, 10 insertions(+), 2071 deletions(-) delete mode 100644 core/src/main/resources/META-INF.services/org.apache.spark.deploy.security.ServiceCredentialProvider delete mode 100644 core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala delete mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala delete mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala diff --git a/core/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 deleted file mode 100644 index 791a0c24ed4cf..0000000000000 --- a/core/src/main/resources/META-INF.services/org.apache.spark.deploy.security.ServiceCredentialProvider +++ /dev/null @@ -1,3 +0,0 @@ -org.apache.spark.deploy.yarn.security.HadoopFSCredentialProvider -org.apache.spark.deploy.security.HBaseCredentialProvider -org.apache.spark.deploy.security.HiveCredentialProvider 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 e6e59fdf127e8..77005aa9040b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -30,7 +30,6 @@ 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._ @@ -46,7 +45,6 @@ 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 +63,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 with Logging { +object SparkSubmit extends CommandLineUtils { // Cluster managers private val YARN = 1 @@ -566,22 +564,12 @@ object SparkSubmit extends CommandLineUtils with Logging { // properties and then loaded by SparkConf sysProps.put("spark.yarn.keytab", args.keytab) sysProps.put("spark.yarn.principal", args.principal) + + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) } } } - - // [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}" - - logDebug(s"Setting ${key} to ${shortUserName}.") - sysProps.put(key, shortUserName) - } - // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" 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 17a17ea4d710f..e8bf551b7383a 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 @@ -130,7 +130,7 @@ private[spark] class ConfigurableCredentialManager( } }.foldLeft(Long.MaxValue)(math.min) } - + /** * Returns a copy of the current user's credentials, augmented with new delegation tokens. */ 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 deleted file mode 100644 index e74a4820b5ee8..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/security/CredentialsSerializer.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.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 3fc27dbd1352c..b2b26ee107c00 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -26,12 +26,9 @@ 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 -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,15 +174,6 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { - private def addDelegationTokens(tokenBytes: Array[Byte], driverConf: SparkConf) { - val creds = new CredentialsSerializer().deserializeTokens(tokenBytes) - - logInfo(s"Adding ${creds.numberOfTokens()} tokens and ${creds.numberOfSecretKeys()} secret" + - s"keys to the current user's credentials.") - - UserGroupInformation.getCurrentUser.addCredentials(creds) - } - private def run( driverUrl: String, executorId: String, @@ -232,8 +220,6 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { SparkHadoopUtil.get.startCredentialUpdater(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 02dd8dd24869d..6b49bd699a13a 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,8 +19,6 @@ 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 @@ -34,8 +32,7 @@ private[spark] object CoarseGrainedClusterMessages { case class SparkAppConfig( sparkProperties: Seq[(String, String)], - ioEncryptionKey: Option[Array[Byte]], - ugiTokens: Option[Array[Byte]]) + ioEncryptionKey: 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 9c4c46de177fb..4eedaaea61195 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,11 +23,9 @@ import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future - -import org.apache.hadoop.security.UserGroupInformation +import scala.concurrent.duration.Duration 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._ @@ -44,10 +42,7 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer, ThreadUtils, Utils} * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend( - scheduler: TaskSchedulerImpl, - val rpcEnv: RpcEnv, - credentialManager: Option[ConfigurableCredentialManager]) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: RpcEnv) extends ExecutorAllocationClient with SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -97,15 +92,6 @@ class CoarseGrainedSchedulerBackend( // 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 { @@ -230,11 +216,8 @@ class CoarseGrainedSchedulerBackend( context.reply(true) case RetrieveSparkAppConfig => - val reply = SparkAppConfig( - sparkProperties, - SparkEnv.get.securityManager.getIOEncryptionKey(), - userTokens - ) + val reply = SparkAppConfig(sparkProperties, + SparkEnv.get.securityManager.getIOEncryptionKey()) 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 96db24ef5af1b..0529fe9eed4da 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, None) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with StandaloneAppClientListener with Logging { 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 deleted file mode 100644 index 55ef6ea382815..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.mesos - -import java.util.concurrent.TimeUnit - -import org.apache.spark.internal.config.ConfigBuilder - -package object config { - - /* Common app configuration. */ - - private[spark] val SHUFFLE_CLEANER_INTERVAL_S = - ConfigBuilder("spark.shuffle.cleaner.interval") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("30s") - - private[spark] val RECOVERY_MODE = - ConfigBuilder("spark.deploy.recoveryMode") - .stringConf - .createWithDefault("NONE") - - private[spark] val DISPATCHER_WEBUI_URL = - ConfigBuilder("spark.mesos.dispatcher.webui.url") - .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + - "framework. If unset it will point to Spark's internal web UI.") - .stringConf - .createOptional - - private[spark] val ZOOKEEPER_URL = - ConfigBuilder("spark.deploy.zookeeper.url") - .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + - "configuration is used to set the zookeeper URL to connect to.") - .stringConf - .createOptional - - private[spark] val HISTORY_SERVER_URL = - ConfigBuilder("spark.mesos.dispatcher.historyServer.url") - .doc("Set the URL of the history server. The dispatcher will then " + - "link each driver to its entry in the history server.") - .stringConf - .createOptional - - private[spark] val USER_CREDENTIALS = ConfigBuilder("spark.mesos.kerberos.userCredentials") - .doc("Base64 encoding of UGI credentials.") - .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 deleted file mode 100644 index 791ddc895ffef..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ /dev/null @@ -1,689 +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.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 -import scala.concurrent.Future - -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -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.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils - -/** - * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds - * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever - * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the - * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable - * latency. - * - * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]], - * but it seems hard to remove this. - */ -private[spark] class MesosCoarseGrainedSchedulerBackend( - scheduler: TaskSchedulerImpl, - sc: SparkContext, - master: String, - securityManager: SecurityManager) - extends CoarseGrainedSchedulerBackend( - scheduler, - sc.env.rpcEnv, - Option(new ConfigurableCredentialManager(sc.conf))) - with org.apache.mesos.Scheduler - with MesosSchedulerUtils { - - // Blacklist a slave after this many failures - private val MAX_SLAVE_FAILURES = 2 - - private val maxCoresOption = conf.getOption("spark.cores.max").map(_.toInt) - - // Maximum number of cores to acquire - private val maxCores = maxCoresOption.getOrElse(Int.MaxValue) - - private val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false) - - private val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) - - private val taskLabels = conf.get("spark.mesos.task.labels", "") - - private[this] val shutdownTimeoutMS = - conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s") - .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0") - - // Synchronization protected by stateLock - private[this] var stopCalled: Boolean = false - - // If shuffle service is enabled, the Spark driver will register with the shuffle service. - // This is for cleaning up shuffle files reliably. - private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) - - // Cores we have acquired with each Mesos task ID - private val coresByTaskId = new mutable.HashMap[String, Int] - private val gpusByTaskId = new mutable.HashMap[String, Int] - private var totalCoresAcquired = 0 - private var totalGpusAcquired = 0 - - // SlaveID -> Slave - // This map accumulates entries for the duration of the job. Slaves are never deleted, because - // we need to maintain e.g. failure state and connection state. - private val slaves = new mutable.HashMap[String, Slave] - - /** - * The total number of executors we aim to have. Undefined when not using dynamic allocation. - * Initially set to 0 when using dynamic allocation, the executor allocation manager will send - * the real initial limit later. - */ - private var executorLimitOption: Option[Int] = { - if (Utils.isDynamicAllocationEnabled(conf)) { - Some(0) - } else { - None - } - } - - /** - * Return the current executor limit, which may be [[Int.MaxValue]] - * before properly initialized. - */ - private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue) - - // private lock object protecting mutable state above. Using the intrinsic lock - // may lead to deadlocks since the superclass might also try to lock - private val stateLock = new ReentrantLock - - private val extraCoresPerExecutor = conf.getInt("spark.mesos.extra.cores", 0) - - // Offer constraints - private val slaveOfferConstraints = - parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) - - // Reject offers with mismatched constraints in seconds - private val rejectOfferDurationForUnmetConstraints = - getRejectOfferDurationForUnmetConstraints(sc.conf) - - // Reject offers when we reached the maximum number of cores for this framework - private val rejectOfferDurationForReachedMaxCores = - getRejectOfferDurationForReachedMaxCores(sc.conf) - - // A client for talking to the external shuffle service - private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { - if (shuffleServiceEnabled) { - Some(getShuffleClient()) - } else { - None - } - } - - // This method is factored out for testability - protected def getShuffleClient(): MesosExternalShuffleClient = { - new MesosExternalShuffleClient( - SparkTransportConf.fromSparkConf(conf, "shuffle"), - securityManager, - securityManager.isAuthenticationEnabled()) - } - - private var nextMesosTaskId = 0 - - @volatile var appId: String = _ - - private var schedulerDriver: SchedulerDriver = _ - - def newMesosTaskId(): String = { - val id = nextMesosTaskId - nextMesosTaskId += 1 - id.toString - } - - override def start() { - super.start() - - val driver = createSchedulerDriver( - master, - MesosCoarseGrainedSchedulerBackend.this, - sc.sparkUser, - sc.appName, - sc.conf, - sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), - None, - None, - sc.conf.getOption("spark.mesos.driver.frameworkId") - ) - - unsetFrameworkID(sc) - startScheduler(driver) - } - - def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { - val environment = Environment.newBuilder() - val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") - - // Set the environment variable through a command prefix - // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => - Utils.libraryPathEnvPrefix(Seq(p)) - }.getOrElse("") - - environment.addVariables( - Environment.Variable.newBuilder() - .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraJavaOpts) - .build()) - - sc.executorEnvs.foreach { case (key, value) => - environment.addVariables(Environment.Variable.newBuilder() - .setName(key) - .setValue(value) - .build()) - } - val command = CommandInfo.newBuilder() - .setEnvironment(environment) - - val uri = conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) - - if (uri.isEmpty) { - val executorSparkHome = conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } - val runScript = new File(executorSparkHome, "./bin/spark-class").getPath - command.setValue( - "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" - .format(prefixEnv, runScript) + - s" --driver-url $driverURL" + - s" --executor-id $taskId" + - s" --hostname ${executorHostname(offer)}" + - s" --cores $numCores" + - s" --app-id $appId") - } else { - // Grab everything to the first '.'. We'll use that and '*' to - // glob the directory "correctly". - val basename = uri.get.split('/').last.split('.').head - command.setValue( - s"cd $basename*; $prefixEnv " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + - s" --driver-url $driverURL" + - s" --executor-id $taskId" + - s" --hostname ${executorHostname(offer)}" + - s" --cores $numCores" + - s" --app-id $appId") - command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache)) - } - - conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache)) - - command.build() - } - - - protected def driverURL: String = { - if (conf.contains("spark.testing")) { - "driverURL" - } else { - RpcEndpointAddress( - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, - CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString - } - } - - override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} - - override def registered( - driver: org.apache.mesos.SchedulerDriver, - frameworkId: FrameworkID, - masterInfo: MasterInfo) { - this.appId = frameworkId.getValue - this.mesosExternalShuffleClient.foreach(_.init(appId)) - this.schedulerDriver = driver - markRegistered() - } - - override def sufficientResourcesRegistered(): Boolean = { - totalCoreCount.get >= maxCoresOption.getOrElse(0) * minRegisteredRatio - } - - override def disconnected(d: org.apache.mesos.SchedulerDriver) {} - - override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} - - /** - * Method called by Mesos to offer resources on slaves. We respond by launching an executor, - * unless we've already launched more than we wanted to. - */ - override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { - stateLock.synchronized { - if (stopCalled) { - logDebug("Ignoring offers during shutdown") - // Driver should simply return a stopped status on race - // condition between this.stop() and completing here - offers.asScala.map(_.getId).foreach(d.declineOffer) - return - } - - logDebug(s"Received ${offers.size} resource offers.") - - val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer => - val offerAttributes = toAttributeMap(offer.getAttributesList) - matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) - } - - declineUnmatchedOffers(d, unmatchedOffers) - handleMatchedOffers(d, matchedOffers) - } - } - - private def declineUnmatchedOffers( - driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { - offers.foreach { offer => - declineOffer( - driver, - offer, - Some("unmet constraints"), - Some(rejectOfferDurationForUnmetConstraints)) - } - } - - /** - * Launches executors on accepted offers, and declines unused offers. Executors are launched - * round-robin on offers. - * - * @param driver SchedulerDriver - * @param offers Mesos offers that match attribute constraints - */ - private def handleMatchedOffers( - driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { - val tasks = buildMesosTasks(offers) - for (offer <- offers) { - val offerAttributes = toAttributeMap(offer.getAttributesList) - val offerMem = getResource(offer.getResourcesList, "mem") - val offerCpus = getResource(offer.getResourcesList, "cpus") - val offerPorts = getRangeResource(offer.getResourcesList, "ports") - val id = offer.getId.getValue - - if (tasks.contains(offer.getId)) { // accept - val offerTasks = tasks(offer.getId) - - logDebug(s"Accepting offer: $id with attributes: $offerAttributes " + - s"mem: $offerMem cpu: $offerCpus ports: $offerPorts." + - s" Launching ${offerTasks.size} Mesos tasks.") - - for (task <- offerTasks) { - val taskId = task.getTaskId - val mem = getResource(task.getResourcesList, "mem") - val cpus = getResource(task.getResourcesList, "cpus") - val ports = getRangeResource(task.getResourcesList, "ports").mkString(",") - - logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" + - s" ports: $ports") - } - - driver.launchTasks( - Collections.singleton(offer.getId), - offerTasks.asJava) - } else if (totalCoresAcquired >= maxCores) { - // Reject an offer for a configurable amount of time to avoid starving other frameworks - declineOffer(driver, - offer, - Some("reached spark.cores.max"), - Some(rejectOfferDurationForReachedMaxCores)) - } else { - declineOffer( - driver, - offer) - } - } - } - - /** - * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize - * per-task memory and IO, tasks are round-robin assigned to offers. - * - * @param offers Mesos offers that match attribute constraints - * @return A map from OfferID to a list of Mesos tasks to launch on that offer - */ - private def buildMesosTasks(offers: mutable.Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = { - // offerID -> tasks - val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) - - // offerID -> resources - val remainingResources = mutable.Map(offers.map(offer => - (offer.getId.getValue, offer.getResourcesList)): _*) - - var launchTasks = true - - // TODO(mgummelt): combine offers for a single slave - // - // round-robin create executors on the available offers - while (launchTasks) { - launchTasks = false - - for (offer <- offers) { - val slaveId = offer.getSlaveId.getValue - val offerId = offer.getId.getValue - val resources = remainingResources(offerId) - - if (canLaunchTask(slaveId, resources)) { - // Create a task - launchTasks = true - val taskId = newMesosTaskId() - val offerCPUs = getResource(resources, "cpus").toInt - val taskGPUs = Math.min( - Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt) - - val taskCPUs = executorCores(offerCPUs) - val taskMemory = executorMemory(sc) - - slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId) - - val (resourcesLeft, resourcesToUse) = - partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs) - - val taskBuilder = MesosTaskInfo.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) - .setSlaveId(offer.getSlaveId) - .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId)) - .setName(s"${sc.appName} $taskId") - - taskBuilder.addAllResources(resourcesToUse.asJava) - taskBuilder.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf)) - - val labelsBuilder = taskBuilder.getLabelsBuilder - val labels = buildMesosLabels().asJava - - labelsBuilder.addAllLabels(labels) - - taskBuilder.setLabels(labelsBuilder) - - tasks(offer.getId) ::= taskBuilder.build() - remainingResources(offerId) = resourcesLeft.asJava - totalCoresAcquired += taskCPUs - coresByTaskId(taskId) = taskCPUs - if (taskGPUs > 0) { - totalGpusAcquired += taskGPUs - gpusByTaskId(taskId) = taskGPUs - } - } - } - } - tasks.toMap - } - - private def buildMesosLabels(): List[Label] = { - taskLabels.split(",").flatMap(label => - label.split(":") match { - case Array(key, value) => - Some(Label.newBuilder() - .setKey(key) - .setValue(value) - .build()) - case _ => - logWarning(s"Unable to parse $label into a key:value label for the task.") - None - } - ).toList - } - - /** Extracts task needed resources from a list of available resources. */ - private def partitionTaskResources( - resources: JList[Resource], - taskCPUs: Int, - taskMemory: Int, - taskGPUs: Int) - : (List[Resource], List[Resource]) = { - - // partition cpus & mem - val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs) - val (afterMemResources, memResourcesToUse) = - partitionResources(afterCPUResources.asJava, "mem", taskMemory) - val (afterGPUResources, gpuResourcesToUse) = - partitionResources(afterMemResources.asJava, "gpus", taskGPUs) - - // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched - // on the same host. This essentially means one executor per host. - // TODO: handle network isolator case - val (nonPortResources, portResourcesToUse) = - partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources) - - (nonPortResources, - cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse) - } - - private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = { - val offerMem = getResource(resources, "mem") - val offerCPUs = getResource(resources, "cpus").toInt - val cpus = executorCores(offerCPUs) - val mem = executorMemory(sc) - val ports = getRangeResource(resources, "ports") - val meetsPortRequirements = checkPorts(sc.conf, ports) - - cpus > 0 && - cpus <= offerCPUs && - cpus + totalCoresAcquired <= maxCores && - mem <= offerMem && - numExecutors() < executorLimit && - slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES && - meetsPortRequirements - } - - private def executorCores(offerCPUs: Int): Int = { - sc.conf.getInt("spark.executor.cores", - math.min(offerCPUs, maxCores - totalCoresAcquired)) - } - - override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { - val taskId = status.getTaskId.getValue - val slaveId = status.getSlaveId.getValue - val state = mesosToTaskState(status.getState) - - logInfo(s"Mesos task $taskId is now ${status.getState}") - - stateLock.synchronized { - val slave = slaves(slaveId) - - // If the shuffle service is enabled, have the driver register with each one of the - // shuffle services. This allows the shuffle services to clean up state associated with - // this application when the driver exits. There is currently not a great way to detect - // this through Mesos, since the shuffle services are set up independently. - if (state.equals(TaskState.RUNNING) && - shuffleServiceEnabled && - !slave.shuffleRegistered) { - assume(mesosExternalShuffleClient.isDefined, - "External shuffle client was not instantiated even though shuffle service is enabled.") - // TODO: Remove this and allow the MesosExternalShuffleService to detect - // framework termination when new Mesos Framework HTTP API is available. - val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337) - - logDebug(s"Connecting to shuffle service on slave $slaveId, " + - s"host ${slave.hostname}, port $externalShufflePort for app ${conf.getAppId}") - - mesosExternalShuffleClient.get - .registerDriverWithShuffleService( - slave.hostname, - externalShufflePort, - sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", - s"${sc.conf.getTimeAsMs("spark.network.timeout", "120s")}ms"), - sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) - slave.shuffleRegistered = true - } - - if (TaskState.isFinished(state)) { - // Remove the cores we have remembered for this task, if it's in the hashmap - for (cores <- coresByTaskId.get(taskId)) { - totalCoresAcquired -= cores - coresByTaskId -= taskId - } - // Also remove the gpus we have remembered for this task, if it's in the hashmap - for (gpus <- gpusByTaskId.get(taskId)) { - totalGpusAcquired -= gpus - gpusByTaskId -= taskId - } - // If it was a failure, mark the slave as failed for blacklisting purposes - if (TaskState.isFailed(state)) { - slave.taskFailures += 1 - - if (slave.taskFailures >= MAX_SLAVE_FAILURES) { - logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " + - "is Spark installed on it?") - } - } - executorTerminated(d, slaveId, taskId, s"Executor finished with state $state") - // In case we'd rejected everything before but have now lost a node - d.reviveOffers() - } - } - } - - override def error(d: org.apache.mesos.SchedulerDriver, message: String) { - logError(s"Mesos error: $message") - scheduler.error(message) - } - - override def stop() { - // Make sure we're not launching tasks during shutdown - stateLock.synchronized { - if (stopCalled) { - logWarning("Stop called multiple times, ignoring") - return - } - stopCalled = true - super.stop() - } - - // Wait for executors to report done, or else mesosDriver.stop() will forcefully kill them. - // See SPARK-12330 - val startTime = System.nanoTime() - - // slaveIdsWithExecutors has no memory barrier, so this is eventually consistent - while (numExecutors() > 0 && - System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) { - Thread.sleep(100) - } - - if (numExecutors() > 0) { - logWarning(s"Timed out waiting for ${numExecutors()} remaining executors " - + s"to terminate within $shutdownTimeoutMS ms. This may leave temporary files " - + "on the mesos nodes.") - } - - // Close the mesos external shuffle client if used - mesosExternalShuffleClient.foreach(_.close()) - - if (schedulerDriver != null) { - schedulerDriver.stop() - } - } - - override def frameworkMessage( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} - - /** - * Called when a slave is lost or a Mesos task finished. Updates local view on - * what tasks are running. It also notifies the driver that an executor was removed. - */ - private def executorTerminated( - d: org.apache.mesos.SchedulerDriver, - slaveId: String, - taskId: String, - reason: String): Unit = { - stateLock.synchronized { - // Do not call removeExecutor() after this scheduler backend was stopped because - // removeExecutor() internally will send a message to the driver endpoint but - // the driver endpoint is not available now, otherwise an exception will be thrown. - if (!stopCalled) { - removeExecutor(taskId, SlaveLost(reason)) - } - slaves(slaveId).taskIDs.remove(taskId) - } - } - - override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { - logInfo(s"Mesos slave lost: ${slaveId.getValue}") - } - - override def executorLost( - d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { - logInfo("Mesos executor lost: %s".format(e.getValue)) - } - - override def applicationId(): String = - Option(appId).getOrElse { - logWarning("Application ID is not initialized yet.") - super.applicationId - } - - override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful { - // We don't truly know if we can fulfill the full amount of executors - // since at coarse grain it depends on the amount of slaves available. - logInfo("Capping the total amount of executors to " + requestedTotal) - executorLimitOption = Some(requestedTotal) - true - } - - override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful { - if (schedulerDriver == null) { - logWarning("Asked to kill executors before the Mesos driver was started.") - false - } else { - for (executorId <- executorIds) { - val taskId = TaskID.newBuilder().setValue(executorId).build() - schedulerDriver.killTask(taskId) - } - // no need to adjust `executorLimitOption` since the AllocationManager already communicated - // the desired limit through a call to `doRequestTotalExecutors`. - // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]] - true - } - } - - private def numExecutors(): Int = { - slaves.values.map(_.taskIDs.size).sum - } - - private def executorHostname(offer: Offer): String = { - if (sc.conf.getOption("spark.mesos.network.name").isDefined) { - // The agent's IP is not visible in a CNI container, so we bind to 0.0.0.0 - "0.0.0.0" - } else { - offer.getHostname - } - } -} - -private class Slave(val hostname: String) { - val taskIDs = new mutable.HashSet[String]() - var taskFailures = 0 - var shuffleRegistered = false -} 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 deleted file mode 100644 index 514491d762224..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ /dev/null @@ -1,558 +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.util.{List => JList} -import java.util.concurrent.CountDownLatch - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.util.control.NonFatal - -import com.google.common.base.Splitter -import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} -import org.apache.mesos.Protos.FrameworkInfo.Capability -import org.apache.mesos.protobuf.{ByteString, GeneratedMessage} - -import org.apache.spark.{SparkConf, SparkContext, SparkException} -import org.apache.spark.TaskState -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.util.Utils - - - -/** - * Shared trait for implementing a Mesos Scheduler. This holds common state and helper - * methods and Mesos scheduler will use. - */ -trait MesosSchedulerUtils extends Logging { - // Lock used to wait for scheduler to be registered - private final val registerLatch = new CountDownLatch(1) - - /** - * Creates a new MesosSchedulerDriver that communicates to the Mesos master. - * - * @param masterUrl The url to connect to Mesos master - * @param scheduler the scheduler class to receive scheduler callbacks - * @param sparkUser User to impersonate with when running tasks - * @param appName The framework name to display on the Mesos UI - * @param conf Spark configuration - * @param webuiUrl The WebUI url to link from Mesos UI - * @param checkpoint Option to checkpoint tasks for failover - * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect - * @param frameworkId The id of the new framework - */ - 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 = { - val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName) - val credBuilder = Credential.newBuilder() - webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) } - checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) } - failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) } - frameworkId.foreach { id => - fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build()) - } - fwInfoBuilder.setHostname(Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse( - conf.get(DRIVER_HOST_ADDRESS))) - conf.getOption("spark.mesos.principal").foreach { principal => - fwInfoBuilder.setPrincipal(principal) - credBuilder.setPrincipal(principal) - } - conf.getOption("spark.mesos.secret").foreach { secret => - credBuilder.setSecret(secret) - } - if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) { - throw new SparkException( - "spark.mesos.principal must be configured when spark.mesos.secret is set") - } - conf.getOption("spark.mesos.role").foreach { role => - fwInfoBuilder.setRole(role) - } - val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) - if (maxGpus > 0) { - fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES)) - } - if (credBuilder.hasPrincipal) { - new MesosSchedulerDriver( - scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build()) - } else { - new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl) - } - } - - /** - * Starts the MesosSchedulerDriver and stores the current running driver to this new instance. - * This driver is expected to not be running. - * This method returns only after the scheduler has registered with Mesos. - */ - def startScheduler(newDriver: SchedulerDriver): Unit = { - synchronized { - @volatile - var error: Option[Exception] = None - - // We create a new thread that will block inside `mesosDriver.run` - // until the scheduler exists - new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { - setDaemon(true) - override def run() { - try { - val ret = newDriver.run() - logInfo("driver.run() returned with code " + ret) - if (ret != null && ret.equals(Status.DRIVER_ABORTED)) { - error = Some(new SparkException("Error starting driver, DRIVER_ABORTED")) - markErr() - } - } catch { - case e: Exception => - logError("driver.run() failed", e) - error = Some(e) - markErr() - } - } - }.start() - - registerLatch.await() - - // propagate any error to the calling thread. This ensures that SparkContext creation fails - // without leaving a broken context that won't be able to schedule any tasks - error.foreach(throw _) - } - } - - def getResource(res: JList[Resource], name: String): Double = { - // A resource can have multiple values in the offer since it can either be from - // a specific role or wildcard. - res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum - } - - /** - * Transforms a range resource to a list of ranges - * - * @param res the mesos resource list - * @param name the name of the resource - * @return the list of ranges returned - */ - protected def getRangeResource(res: JList[Resource], name: String): List[(Long, Long)] = { - // A resource can have multiple values in the offer since it can either be from - // a specific role or wildcard. - res.asScala.filter(_.getName == name).flatMap(_.getRanges.getRangeList.asScala - .map(r => (r.getBegin, r.getEnd)).toList).toList - } - - /** - * Signal that the scheduler has registered with Mesos. - */ - protected def markRegistered(): Unit = { - registerLatch.countDown() - } - - protected def markErr(): Unit = { - registerLatch.countDown() - } - - def createResource(name: String, amount: Double, role: Option[String] = None): Resource = { - val builder = Resource.newBuilder() - .setName(name) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(amount).build()) - - role.foreach { r => builder.setRole(r) } - - builder.build() - } - - /** - * Partition the existing set of resources into two groups, those remaining to be - * scheduled and those requested to be used for a new task. - * - * @param resources The full list of available resources - * @param resourceName The name of the resource to take from the available resources - * @param amountToUse The amount of resources to take from the available resources - * @return The remaining resources list and the used resources list. - */ - def partitionResources( - resources: JList[Resource], - resourceName: String, - amountToUse: Double): (List[Resource], List[Resource]) = { - var remain = amountToUse - var requestedResources = new ArrayBuffer[Resource] - val remainingResources = resources.asScala.map { - case r => - if (remain > 0 && - r.getType == Value.Type.SCALAR && - r.getScalar.getValue > 0.0 && - r.getName == resourceName) { - val usage = Math.min(remain, r.getScalar.getValue) - requestedResources += createResource(resourceName, usage, Some(r.getRole)) - remain -= usage - createResource(resourceName, r.getScalar.getValue - usage, Some(r.getRole)) - } else { - r - } - } - - // Filter any resource that has depleted. - val filteredResources = - remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0) - - (filteredResources.toList, requestedResources.toList) - } - - /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ - protected def getAttribute(attr: Attribute): (String, Set[String]) = { - (attr.getName, attr.getText.getValue.split(',').toSet) - } - - - /** Build a Mesos resource protobuf object */ - protected def createResource(resourceName: String, quantity: Double): Protos.Resource = { - Resource.newBuilder() - .setName(resourceName) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) - .build() - } - - /** - * Converts the attributes from the resource offer into a Map of name to Attribute Value - * The attribute values are the mesos attribute types and they are - * - * @param offerAttributes the attributes offered - * @return - */ - protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { - offerAttributes.asScala.map { attr => - val attrValue = attr.getType match { - case Value.Type.SCALAR => attr.getScalar - case Value.Type.RANGES => attr.getRanges - case Value.Type.SET => attr.getSet - case Value.Type.TEXT => attr.getText - } - (attr.getName, attrValue) - }.toMap - } - - - /** - * Match the requirements (if any) to the offer attributes. - * if attribute requirements are not specified - return true - * else if attribute is defined and no values are given, simple attribute presence is performed - * else if attribute name and value is specified, subset match is performed on slave attributes - */ - def matchesAttributeRequirements( - slaveOfferConstraints: Map[String, Set[String]], - offerAttributes: Map[String, GeneratedMessage]): Boolean = { - slaveOfferConstraints.forall { - // offer has the required attribute and subsumes the required values for that attribute - case (name, requiredValues) => - offerAttributes.get(name) match { - case None => false - case Some(_) if requiredValues.isEmpty => true // empty value matches presence - case Some(scalarValue: Value.Scalar) => - // check if provided values is less than equal to the offered values - requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) - case Some(rangeValue: Value.Range) => - val offerRange = rangeValue.getBegin to rangeValue.getEnd - // Check if there is some required value that is between the ranges specified - // Note: We only support the ability to specify discrete values, in the future - // we may expand it to subsume ranges specified with a XX..YY value or something - // similar to that. - requiredValues.map(_.toLong).exists(offerRange.contains(_)) - case Some(offeredValue: Value.Set) => - // check if the specified required values is a subset of offered set - requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet) - case Some(textValue: Value.Text) => - // check if the specified value is equal, if multiple values are specified - // we succeed if any of them match. - requiredValues.contains(textValue.getValue) - } - } - } - - /** - * Parses the attributes constraints provided to spark and build a matching data struct: - * {@literal Map[, Set[values-to-match]} - * The constraints are specified as ';' separated key-value pairs where keys and values - * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for - * multiple values (comma separated). For example: - * {{{ - * parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") - * // would result in - * - * Map( - * "os" -> Set("centos7"), - * "zone": -> Set("us-east-1a", "us-east-1b") - * ) - * }}} - * - * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ - * https://github.com/apache/mesos/blob/master/src/common/values.cpp - * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp - * - * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated - * by ':') - * @return Map of constraints to match resources offers. - */ - def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { - /* - Based on mesos docs: - attributes : attribute ( ";" attribute )* - attribute : labelString ":" ( labelString | "," )+ - labelString : [a-zA-Z0-9_/.-] - */ - val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') - // kv splitter - if (constraintsVal.isEmpty) { - Map() - } else { - try { - splitter.split(constraintsVal).asScala.toMap.mapValues(v => - if (v == null || v.isEmpty) { - Set[String]() - } else { - v.split(',').toSet - } - ) - } catch { - case NonFatal(e) => - throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) - } - } - } - - // These defaults copied from YARN - private val MEMORY_OVERHEAD_FRACTION = 0.10 - private val MEMORY_OVERHEAD_MINIMUM = 384 - - /** - * Return the amount of memory to allocate to each executor, taking into account - * container overheads. - * - * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value - * @return memory requirement as (0.1 * memoryOverhead) or MEMORY_OVERHEAD_MINIMUM - * (whichever is larger) - */ - def executorMemory(sc: SparkContext): Int = { - sc.conf.getInt("spark.mesos.executor.memoryOverhead", - math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + - sc.executorMemory - } - - def setupUris(uris: String, - builder: CommandInfo.Builder, - useFetcherCache: Boolean = false): Unit = { - uris.split(",").foreach { uri => - builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache)) - } - } - - private def getRejectOfferDurationStr(conf: SparkConf): String = { - conf.get("spark.mesos.rejectOfferDuration", "120s") - } - - protected def getRejectOfferDuration(conf: SparkConf): Long = { - Utils.timeStringAsSeconds(getRejectOfferDurationStr(conf)) - } - - protected def getRejectOfferDurationForUnmetConstraints(conf: SparkConf): Long = { - conf.getTimeAsSeconds( - "spark.mesos.rejectOfferDurationForUnmetConstraints", - getRejectOfferDurationStr(conf)) - } - - protected def getRejectOfferDurationForReachedMaxCores(conf: SparkConf): Long = { - conf.getTimeAsSeconds( - "spark.mesos.rejectOfferDurationForReachedMaxCores", - getRejectOfferDurationStr(conf)) - } - - /** - * Checks executor ports if they are within some range of the offered list of ports ranges, - * - * @param conf the Spark Config - * @param ports the list of ports to check - * @return true if ports are within range false otherwise - */ - protected def checkPorts(conf: SparkConf, ports: List[(Long, Long)]): Boolean = { - - def checkIfInRange(port: Long, ps: List[(Long, Long)]): Boolean = { - ps.exists{case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port } - } - - val portsToCheck = nonZeroPortValuesFromConfig(conf) - val withinRange = portsToCheck.forall(p => checkIfInRange(p, ports)) - // make sure we have enough ports to allocate per offer - val enoughPorts = - ports.map{case (rangeStart, rangeEnd) => rangeEnd - rangeStart + 1}.sum >= portsToCheck.size - enoughPorts && withinRange - } - - /** - * Partitions port resources. - * - * @param requestedPorts non-zero ports to assign - * @param offeredResources the resources offered - * @return resources left, port resources to be used. - */ - def partitionPortResources(requestedPorts: List[Long], offeredResources: List[Resource]) - : (List[Resource], List[Resource]) = { - if (requestedPorts.isEmpty) { - (offeredResources, List[Resource]()) - } else { - // partition port offers - val (resourcesWithoutPorts, portResources) = filterPortResources(offeredResources) - - val portsAndRoles = requestedPorts. - map(x => (x, findPortAndGetAssignedRangeRole(x, portResources))) - - val assignedPortResources = createResourcesFromPorts(portsAndRoles) - - // ignore non-assigned port resources, they will be declined implicitly by mesos - // no need for splitting port resources. - (resourcesWithoutPorts, assignedPortResources) - } - } - - val managedPortNames = List("spark.executor.port", BLOCK_MANAGER_PORT.key) - - /** - * The values of the non-zero ports to be used by the executor process. - * - * @param conf the spark config to use - * @return the ono-zero values of the ports - */ - def nonZeroPortValuesFromConfig(conf: SparkConf): List[Long] = { - managedPortNames.map(conf.getLong(_, 0)).filter( _ != 0) - } - - /** Creates a mesos resource for a specific port number. */ - private def createResourcesFromPorts(portsAndRoles: List[(Long, String)]) : List[Resource] = { - portsAndRoles.flatMap{ case (port, role) => - createMesosPortResource(List((port, port)), Some(role))} - } - - /** Helper to create mesos resources for specific port ranges. */ - private def createMesosPortResource( - ranges: List[(Long, Long)], - role: Option[String] = None): List[Resource] = { - ranges.map { case (rangeStart, rangeEnd) => - val rangeValue = Value.Range.newBuilder() - .setBegin(rangeStart) - .setEnd(rangeEnd) - val builder = Resource.newBuilder() - .setName("ports") - .setType(Value.Type.RANGES) - .setRanges(Value.Ranges.newBuilder().addRange(rangeValue)) - role.foreach(r => builder.setRole(r)) - builder.build() - } - } - - /** - * Helper to assign a port to an offered range and get the latter's role - * info to use it later on. - */ - private def findPortAndGetAssignedRangeRole(port: Long, portResources: List[Resource]) - : String = { - - val ranges = portResources. - map(resource => - (resource.getRole, resource.getRanges.getRangeList.asScala - .map(r => (r.getBegin, r.getEnd)).toList)) - - val rangePortRole = ranges - .find { case (role, rangeList) => rangeList - .exists{ case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port}} - // this is safe since we have previously checked about the ranges (see checkPorts method) - rangePortRole.map{ case (role, rangeList) => role}.get - } - - /** Retrieves the port resources from a list of mesos offered resources */ - private def filterPortResources(resources: List[Resource]): (List[Resource], List[Resource]) = { - resources.partition { r => !(r.getType == Value.Type.RANGES && r.getName == "ports") } - } - - /** - * spark.mesos.driver.frameworkId is set by the cluster dispatcher to correlate driver - * submissions with frameworkIDs. However, this causes issues when a driver process launches - * more than one framework (more than one SparkContext(, because they all try to register with - * the same frameworkID. To enforce that only the first driver registers with the configured - * framework ID, the driver calls this method after the first registration. - */ - def unsetFrameworkID(sc: SparkContext) { - sc.conf.remove("spark.mesos.driver.frameworkId") - System.clearProperty("spark.mesos.driver.frameworkId") - } - - def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match { - case MesosTaskState.TASK_STAGING | MesosTaskState.TASK_STARTING => TaskState.LAUNCHING - case MesosTaskState.TASK_RUNNING | MesosTaskState.TASK_KILLING => TaskState.RUNNING - case MesosTaskState.TASK_FINISHED => TaskState.FINISHED - case MesosTaskState.TASK_FAILED => TaskState.FAILED - case MesosTaskState.TASK_KILLED => TaskState.KILLED - case MesosTaskState.TASK_LOST | MesosTaskState.TASK_ERROR => TaskState.LOST - } - - def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match { - case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING - case TaskState.RUNNING => MesosTaskState.TASK_RUNNING - case TaskState.FINISHED => MesosTaskState.TASK_FINISHED - case TaskState.FAILED => MesosTaskState.TASK_FAILED - case TaskState.KILLED => MesosTaskState.TASK_KILLED - case TaskState.LOST => MesosTaskState.TASK_LOST - } - - protected def declineOffer( - driver: org.apache.mesos.SchedulerDriver, - offer: Offer, - reason: Option[String] = None, - refuseSeconds: Option[Long] = None): Unit = { - - val id = offer.getId.getValue - val offerAttributes = toAttributeMap(offer.getAttributesList) - val mem = getResource(offer.getResourcesList, "mem") - val cpus = getResource(offer.getResourcesList, "cpus") - val ports = getRangeResource(offer.getResourcesList, "ports") - - logDebug(s"Declining offer: $id with " + - s"attributes: $offerAttributes " + - s"mem: $mem " + - s"cpu: $cpus " + - s"port: $ports " + - refuseSeconds.map(s => s"for ${s} seconds ").getOrElse("") + - reason.map(r => s" (reason: $r)").getOrElse("")) - - refuseSeconds match { - case Some(seconds) => - val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() - driver.declineOffer(offer.getId, filters) - case _ => - driver.declineOffer(offer.getId) - } - } -} 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 deleted file mode 100644 index 1ba05a1aa1400..0000000000000 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ /dev/null @@ -1,663 +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.util.concurrent.TimeUnit - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.reflect.ClassTag - -import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} -import org.apache.mesos.Protos._ -import org.mockito.Matchers -import org.mockito.Matchers._ -import org.mockito.Mockito._ -import org.scalatest.concurrent.ScalaFutures -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, RetrieveSparkAppConfig, SparkAppConfig} -import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.scheduler.cluster.mesos.Utils._ - -class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite - with LocalSparkContext - with MockitoSugar - with BeforeAndAfter - with ScalaFutures { - - private var sparkConf: SparkConf = _ - private var driver: SchedulerDriver = _ - private var taskScheduler: TaskSchedulerImpl = _ - private var backend: MesosCoarseGrainedSchedulerBackend = _ - private var externalShuffleClient: MesosExternalShuffleClient = _ - private var driverEndpoint: RpcEndpointRef = _ - @volatile private var stopCalled = false - - // All 'requests' to the scheduler run immediately on the same thread, so - // demand that all futures have their value available immediately. - implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS)) - - test("mesos supports killing and limiting executors") { - init() - sparkConf.set("spark.driver.host", "driverHost") - sparkConf.set("spark.driver.port", "1234") - - val minMem = backend.executorMemory(sc) - val minCpu = 4 - val offers = List(Resources(minMem, minCpu)) - - // launches a task on a valid offer - offerResources(offers) - verifyTaskLaunched(driver, "o1") - - // kills executors - assert(backend.doRequestTotalExecutors(0).futureValue) - assert(backend.doKillExecutors(Seq("0")).futureValue) - val taskID0 = createTaskId("0") - verify(driver, times(1)).killTask(taskID0) - - // doesn't launch a new task when requested executors == 0 - offerResources(offers, 2) - verifyDeclinedOffer(driver, createOfferId("o2")) - - // Launches a new task when requested executors is positive - backend.doRequestTotalExecutors(2) - offerResources(offers, 2) - verifyTaskLaunched(driver, "o2") - } - - 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 - val offer1 = Resources(minMem, minCpu) - val offer2 = Resources(minMem, 1) - offerResources(List(offer1, offer2)) - verifyTaskLaunched(driver, "o1") - - // accounts for a killed task - val status = createTaskStatus("0", "s1", TaskState.TASK_KILLED) - backend.statusUpdate(driver, status) - verify(driver, times(1)).reviveOffers() - - // Launches a new task on a valid offer from the same slave - offerResources(List(offer2)) - verifyTaskLaunched(driver, "o2") - } - - test("mesos supports spark.executor.cores") { - val executorCores = 4 - init(Map("spark.executor.cores" -> executorCores.toString)) - - val executorMemory = backend.executorMemory(sc) - val offers = List(Resources(executorMemory * 2, executorCores + 1)) - offerResources(offers) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == executorCores) - } - - test("mesos supports unset spark.executor.cores") { - init() - - val executorMemory = backend.executorMemory(sc) - val offerCores = 10 - offerResources(List(Resources(executorMemory * 2, offerCores))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == offerCores) - } - - test("mesos does not acquire more than spark.cores.max") { - val maxCores = 10 - init(Map("spark.cores.max" -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, maxCores + 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") - assert(cpus == maxCores) - } - - test("mesos does not acquire gpus if not specified") { - init() - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, 1, 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") - assert(gpus == 0.0) - } - - - test("mesos does not acquire more than spark.mesos.gpus.max") { - val maxGpus = 5 - init(Map("spark.mesos.gpus.max" -> maxGpus.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 1) - - val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") - assert(gpus == maxGpus) - } - - - test("mesos declines offers that violate attribute constraints") { - 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 - init(Map("spark.cores.max" -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory, maxCores + 1), - Resources(executorMemory, maxCores + 1))) - - verifyTaskLaunched(driver, "o1") - verifyDeclinedOffer(driver, createOfferId("o2"), true) - } - - test("mesos assigns tasks round-robin on offers") { - val executorCores = 4 - val maxCores = executorCores * 2 - init(Map("spark.executor.cores" -> executorCores.toString, - "spark.cores.max" -> maxCores.toString)) - - val executorMemory = backend.executorMemory(sc) - offerResources(List( - Resources(executorMemory * 2, executorCores * 2), - Resources(executorMemory * 2, executorCores * 2))) - - verifyTaskLaunched(driver, "o1") - verifyTaskLaunched(driver, "o2") - } - - test("mesos creates multiple executors on a single slave") { - val executorCores = 4 - init(Map("spark.executor.cores" -> executorCores.toString)) - - // offer with room for two executors - val executorMemory = backend.executorMemory(sc) - offerResources(List(Resources(executorMemory * 2, executorCores * 2))) - - // verify two executors were started on a single offer - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.length == 2) - } - - test("mesos doesn't register twice with the same shuffle service") { - init(Map("spark.shuffle.service.enabled" -> "true")) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - - val offer2 = createOffer("o2", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer2).asJava) - verifyTaskLaunched(driver, "o2") - - val status1 = createTaskStatus("0", "s1", TaskState.TASK_RUNNING) - backend.statusUpdate(driver, status1) - - val status2 = createTaskStatus("1", "s1", TaskState.TASK_RUNNING) - backend.statusUpdate(driver, status2) - verify(externalShuffleClient, times(1)) - .registerDriverWithShuffleService(anyString, anyInt, anyLong, anyLong) - } - - test("Port offer decline when there is no appropriate range") { - init(Map(BLOCK_MANAGER_PORT.key -> "30100")) - val offeredPorts = (31100L, 31200L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - verify(driver, times(1)).declineOffer(offer1.getId) - } - - test("Port offer accepted when ephemeral ports are used") { - init() - val offeredPorts = (31100L, 31200L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - } - - test("Port offer accepted with user defined port numbers") { - val port = 30100 - init(Map(BLOCK_MANAGER_PORT.key -> s"$port")) - val offeredPorts = (30000L, 31000L) - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) - backend.resourceOffers(driver, List(offer1).asJava) - val taskInfo = verifyTaskLaunched(driver, "o1") - - val taskPortResources = taskInfo.head.getResourcesList.asScala. - find(r => r.getType == Value.Type.RANGES && r.getName == "ports") - - val isPortInOffer = (r: Resource) => { - r.getRanges().getRangeList - .asScala.exists(range => range.getBegin == port && range.getEnd == port) - } - assert(taskPortResources.exists(isPortInOffer)) - } - - test("mesos kills an executor when told") { - init() - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - verifyTaskLaunched(driver, "o1") - - backend.doKillExecutors(List("0")) - verify(driver, times(1)).killTask(createTaskId("0")) - } - - test("weburi is set in created scheduler driver") { - getSparkConf() - sc = new SparkContext(sparkConf) - - val taskScheduler = mock[TaskSchedulerImpl] - when(taskScheduler.sc).thenReturn(sc) - - val driver = mock[SchedulerDriver] - when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) - - val securityManager = mock[SecurityManager] - - 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 = { - markRegistered() - assert(webuiUrl.isDefined) - assert(webuiUrl.get.equals("http://webui")) - driver - } - } - - backend.start() - } - - test("honors unset spark.mesos.containerizer") { - init(Map("spark.mesos.executor.docker.image" -> "test")) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.DOCKER) - } - - test("honors spark.mesos.containerizer=\"mesos\"") { - init(Map( - "spark.mesos.executor.docker.image" -> "test", - "spark.mesos.containerizer" -> "mesos")) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val taskInfos = verifyTaskLaunched(driver, "o1") - assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.MESOS) - } - - test("docker settings are reflected in created tasks") { - 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", - "spark.mesos.executor.docker.portmaps" -> "8080:80:tcp" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.size == 1) - - val containerInfo = launchedTasks.head.getContainer - assert(containerInfo.getType == ContainerInfo.Type.DOCKER) - - val volumes = containerInfo.getVolumesList.asScala - assert(volumes.size == 1) - - val volume = volumes.head - assert(volume.getHostPath == "/host_vol") - assert(volume.getContainerPath == "/container_vol") - assert(volume.getMode == Volume.Mode.RO) - - val dockerInfo = containerInfo.getDocker - - val portMappings = dockerInfo.getPortMappingsList.asScala - assert(portMappings.size == 1) - - val portMapping = portMappings.head - assert(portMapping.getHostPort == 8080) - assert(portMapping.getContainerPort == 80) - assert(portMapping.getProtocol == "tcp") - } - - test("force-pull-image option is disabled by default") { - init(Map( - "spark.mesos.executor.docker.image" -> "some_image" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.size == 1) - - val containerInfo = launchedTasks.head.getContainer - assert(containerInfo.getType == ContainerInfo.Type.DOCKER) - - val dockerInfo = containerInfo.getDocker - - assert(dockerInfo.getImage == "some_image") - assert(!dockerInfo.getForcePullImage) - } - - test("mesos supports spark.executor.uri") { - val url = "spark.spark.spark.com" - init(Map( - "spark.executor.uri" -> url - ), null) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url) - } - - test("mesos supports setting fetcher cache") { - val url = "spark.spark.spark.com" - init(Map( - "spark.mesos.fetcherCache.enable" -> "true", - "spark.executor.uri" -> url - ), null) - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.size() == 1) - assert(uris.asScala.head.getCache) - } - - test("mesos supports disabling fetcher cache") { - val url = "spark.spark.spark.com" - init(Map( - "spark.mesos.fetcherCache.enable" -> "false", - "spark.executor.uri" -> url - ), null) - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.size() == 1) - assert(!uris.asScala.head.getCache) - } - - test("mesos sets task name to spark.app.name") { - init() - - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - - // Add " 0" to the taskName to match the executor number that is appended - assert(launchedTasks.head.getName == "test-mesos-dynamic-alloc 0") - } - - test("mesos sets configurable labels on tasks") { - val taskLabelsString = "mesos:test,label:test" - init(Map( - "spark.mesos.task.labels" -> taskLabelsString - )) - - // Build up the labels - val taskLabels = Protos.Labels.newBuilder() - .addLabels(Protos.Label.newBuilder() - .setKey("mesos").setValue("test").build()) - .addLabels(Protos.Label.newBuilder() - .setKey("label").setValue("test").build()) - .build() - - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - - val labels = launchedTasks.head.getLabels - - assert(launchedTasks.head.getLabels.equals(taskLabels)) - } - - test("mesos ignored invalid labels and sets configurable labels on tasks") { - val taskLabelsString = "mesos:test,label:test,incorrect:label:here" - init(Map( - "spark.mesos.task.labels" -> taskLabelsString - )) - - // Build up the labels - val taskLabels = Protos.Labels.newBuilder() - .addLabels(Protos.Label.newBuilder() - .setKey("mesos").setValue("test").build()) - .addLabels(Protos.Label.newBuilder() - .setKey("label").setValue("test").build()) - .build() - - val offers = List(Resources(backend.executorMemory(sc), 1)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - - val labels = launchedTasks.head.getLabels - - assert(launchedTasks.head.getLabels.equals(taskLabels)) - } - - test("mesos supports spark.mesos.network.name") { - init(Map( - "spark.mesos.network.name" -> "test-network-name" - )) - - val (mem, cpu) = (backend.executorMemory(sc), 4) - - val offer1 = createOffer("o1", "s1", mem, cpu) - backend.resourceOffers(driver, List(offer1).asJava) - - val launchedTasks = verifyTaskLaunched(driver, "o1") - val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList - assert(networkInfos.size == 1) - assert(networkInfos.get(0).getName == "test-network-name") - } - - test("supports spark.scheduler.minRegisteredResourcesRatio") { - val expectedCores = 1 - init(Map( - "spark.cores.max" -> expectedCores.toString, - "spark.scheduler.minRegisteredResourcesRatio" -> "1.0")) - - val offers = List(Resources(backend.executorMemory(sc), expectedCores)) - offerResources(offers) - val launchedTasks = verifyTaskLaunched(driver, "o1") - assert(!backend.isReady) - - registerMockExecutor(launchedTasks(0).getTaskId.getValue, "s1", expectedCores) - 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) = { - val mockEndpointRef = mock[RpcEndpointRef] - val mockAddress = mock[RpcAddress] - val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty) - - backend.driverEndpoint.askSync[Boolean](message) - } - - private def verifyDeclinedOffer(driver: SchedulerDriver, - offerId: OfferID, - filter: Boolean = false): Unit = { - if (filter) { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters]) - } else { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId)) - } - } - - private def offerResources(offers: List[Resources], startId: Int = 1): Unit = { - val mesosOffers = offers.zipWithIndex.map {case (offer, i) => - createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)} - - backend.resourceOffers(driver, mesosOffers.asJava) - } - - private def createTaskStatus(taskId: String, slaveId: String, state: TaskState): TaskStatus = { - TaskStatus.newBuilder() - .setTaskId(TaskID.newBuilder().setValue(taskId).build()) - .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) - .setState(state) - .build - } - - private def init( - properties: Map[String, String] = null, - home: String = "/path"): Unit = { - - sparkConf = getSparkConf(properties, 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] - - val securityManager = mock[SecurityManager] - - 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 - - override protected def getShuffleClient(): MesosExternalShuffleClient = externalShuffleClient - - // override to avoid race condition with the driver thread on `mesosDriver` - override def startScheduler(newDriver: SchedulerDriver): Unit = {} - - override def stopExecutors(): Unit = { - stopCalled = true - } - } - backend.start() - backend.registered(driver, Utils.TEST_FRAMEWORK_ID, Utils.TEST_MASTER_INFO) - } - - private def getSparkConf( - sparkConfVars: Map[String, String] = null, - home: String = "/path"): SparkConf = { - val sparkConf = (new SparkConf) - .setMaster("local[*]") - .setAppName("test-mesos-dynamic-alloc") - .set("spark.mesos.driver.webui.url", "http://webui") - - if (home != null) { - sparkConf.setSparkHome(home) - } - - if (sparkConfVars != null) { - sparkConf.setAll(sparkConfVars) - } - - sparkConf - } -} From 240df317dd42584349a3c4a0bf6f7d78a4fbe0e6 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 12:38:07 -0700 Subject: [PATCH 18/40] re-add mistakenly removed files --- .../apache/spark/deploy/mesos/config.scala | 59 ++ .../MesosCoarseGrainedSchedulerBackend.scala | 679 ++++++++++++++++++ .../cluster/mesos/MesosSchedulerUtils.scala | 558 ++++++++++++++ ...osCoarseGrainedSchedulerBackendSuite.scala | 654 +++++++++++++++++ .../cluster/YarnSchedulerBackend.scala | 2 +- 5 files changed, 1951 insertions(+), 1 deletion(-) create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala create mode 100644 resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala create mode 100644 resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala 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 new file mode 100644 index 0000000000000..19e253394f1b2 --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -0,0 +1,59 @@ +/* + * 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.mesos + +import java.util.concurrent.TimeUnit + +import org.apache.spark.internal.config.ConfigBuilder + +package object config { + + /* Common app configuration. */ + + private[spark] val SHUFFLE_CLEANER_INTERVAL_S = + ConfigBuilder("spark.shuffle.cleaner.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("30s") + + private[spark] val RECOVERY_MODE = + ConfigBuilder("spark.deploy.recoveryMode") + .stringConf + .createWithDefault("NONE") + + private[spark] val DISPATCHER_WEBUI_URL = + ConfigBuilder("spark.mesos.dispatcher.webui.url") + .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + + "framework. If unset it will point to Spark's internal web UI.") + .stringConf + .createOptional + + private[spark] val ZOOKEEPER_URL = + ConfigBuilder("spark.deploy.zookeeper.url") + .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + + "configuration is used to set the zookeeper URL to connect to.") + .stringConf + .createOptional + + private[spark] val HISTORY_SERVER_URL = + ConfigBuilder("spark.mesos.dispatcher.historyServer.url") + .doc("Set the URL of the history server. The dispatcher will then " + + "link each driver to its entry in the history server.") + .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 new file mode 100644 index 0000000000000..2a36ec4fa8112 --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -0,0 +1,679 @@ +/* + * 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.File +import java.util.{Collections, List => JList} +import java.util.concurrent.locks.ReentrantLock + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.concurrent.Future + +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.mesos.SchedulerDriver + +import org.apache.spark.{SecurityManager, SparkContext, SparkException, TaskState} +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.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils + +/** + * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds + * onto each Mesos node for the duration of the Spark job instead of relinquishing cores whenever + * a task is done. It launches Spark tasks within the coarse-grained Mesos tasks using the + * CoarseGrainedSchedulerBackend mechanism. This class is useful for lower and more predictable + * latency. + * + * Unfortunately this has a bit of duplication from [[MesosFineGrainedSchedulerBackend]], + * but it seems hard to remove this. + */ +private[spark] class MesosCoarseGrainedSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext, + master: String, + securityManager: SecurityManager) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) + with org.apache.mesos.Scheduler + with MesosSchedulerUtils { + + // Blacklist a slave after this many failures + private val MAX_SLAVE_FAILURES = 2 + + private val maxCoresOption = conf.getOption("spark.cores.max").map(_.toInt) + + // Maximum number of cores to acquire + private val maxCores = maxCoresOption.getOrElse(Int.MaxValue) + + private val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false) + + private val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + + private val taskLabels = conf.get("spark.mesos.task.labels", "") + + private[this] val shutdownTimeoutMS = + conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s") + .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0") + + // Synchronization protected by stateLock + private[this] var stopCalled: Boolean = false + + // If shuffle service is enabled, the Spark driver will register with the shuffle service. + // This is for cleaning up shuffle files reliably. + private val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) + + // Cores we have acquired with each Mesos task ID + private val coresByTaskId = new mutable.HashMap[String, Int] + private val gpusByTaskId = new mutable.HashMap[String, Int] + private var totalCoresAcquired = 0 + private var totalGpusAcquired = 0 + + // SlaveID -> Slave + // This map accumulates entries for the duration of the job. Slaves are never deleted, because + // we need to maintain e.g. failure state and connection state. + private val slaves = new mutable.HashMap[String, Slave] + + /** + * The total number of executors we aim to have. Undefined when not using dynamic allocation. + * Initially set to 0 when using dynamic allocation, the executor allocation manager will send + * the real initial limit later. + */ + private var executorLimitOption: Option[Int] = { + if (Utils.isDynamicAllocationEnabled(conf)) { + Some(0) + } else { + None + } + } + + /** + * Return the current executor limit, which may be [[Int.MaxValue]] + * before properly initialized. + */ + private[mesos] def executorLimit: Int = executorLimitOption.getOrElse(Int.MaxValue) + + // private lock object protecting mutable state above. Using the intrinsic lock + // may lead to deadlocks since the superclass might also try to lock + private val stateLock = new ReentrantLock + + private val extraCoresPerExecutor = conf.getInt("spark.mesos.extra.cores", 0) + + // Offer constraints + private val slaveOfferConstraints = + parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + + // Reject offers with mismatched constraints in seconds + private val rejectOfferDurationForUnmetConstraints = + getRejectOfferDurationForUnmetConstraints(sc.conf) + + // Reject offers when we reached the maximum number of cores for this framework + private val rejectOfferDurationForReachedMaxCores = + getRejectOfferDurationForReachedMaxCores(sc.conf) + + // A client for talking to the external shuffle service + private val mesosExternalShuffleClient: Option[MesosExternalShuffleClient] = { + if (shuffleServiceEnabled) { + Some(getShuffleClient()) + } else { + None + } + } + + // This method is factored out for testability + protected def getShuffleClient(): MesosExternalShuffleClient = { + new MesosExternalShuffleClient( + SparkTransportConf.fromSparkConf(conf, "shuffle"), + securityManager, + securityManager.isAuthenticationEnabled()) + } + + private var nextMesosTaskId = 0 + + @volatile var appId: String = _ + + private var schedulerDriver: SchedulerDriver = _ + + def newMesosTaskId(): String = { + val id = nextMesosTaskId + nextMesosTaskId += 1 + id.toString + } + + override def start() { + super.start() + val driver = createSchedulerDriver( + master, + MesosCoarseGrainedSchedulerBackend.this, + sc.sparkUser, + sc.appName, + sc.conf, + sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), + None, + None, + sc.conf.getOption("spark.mesos.driver.frameworkId") + ) + + unsetFrameworkID(sc) + startScheduler(driver) + } + + def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { + val environment = Environment.newBuilder() + val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") + + environment.addVariables( + Environment.Variable.newBuilder() + .setName("SPARK_EXECUTOR_OPTS") + .setValue(extraJavaOpts) + .build()) + + sc.executorEnvs.foreach { case (key, value) => + environment.addVariables(Environment.Variable.newBuilder() + .setName(key) + .setValue(value) + .build()) + } + val command = CommandInfo.newBuilder() + .setEnvironment(environment) + + val uri = conf.getOption("spark.executor.uri") + .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + + if (uri.isEmpty) { + val executorSparkHome = conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } + val runScript = new File(executorSparkHome, "./bin/spark-class").getPath + command.setValue( + "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend" + .format(prefixEnv, runScript) + + s" --driver-url $driverURL" + + s" --executor-id $taskId" + + s" --hostname ${executorHostname(offer)}" + + s" --cores $numCores" + + s" --app-id $appId") + } else { + // Grab everything to the first '.'. We'll use that and '*' to + // glob the directory "correctly". + val basename = uri.get.split('/').last.split('.').head + command.setValue( + s"cd $basename*; $prefixEnv " + + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend" + + s" --driver-url $driverURL" + + s" --executor-id $taskId" + + s" --hostname ${executorHostname(offer)}" + + s" --cores $numCores" + + s" --app-id $appId") + command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache)) + } + + conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache)) + + command.build() + } + + protected def driverURL: String = { + if (conf.contains("spark.testing")) { + "driverURL" + } else { + RpcEndpointAddress( + conf.get("spark.driver.host"), + conf.get("spark.driver.port").toInt, + CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + } + } + + override def offerRescinded(d: org.apache.mesos.SchedulerDriver, o: OfferID) {} + + override def registered( + driver: org.apache.mesos.SchedulerDriver, + frameworkId: FrameworkID, + masterInfo: MasterInfo) { + this.appId = frameworkId.getValue + this.mesosExternalShuffleClient.foreach(_.init(appId)) + this.schedulerDriver = driver + markRegistered() + } + + override def sufficientResourcesRegistered(): Boolean = { + totalCoreCount.get >= maxCoresOption.getOrElse(0) * minRegisteredRatio + } + + override def disconnected(d: org.apache.mesos.SchedulerDriver) {} + + override def reregistered(d: org.apache.mesos.SchedulerDriver, masterInfo: MasterInfo) {} + + /** + * Method called by Mesos to offer resources on slaves. We respond by launching an executor, + * unless we've already launched more than we wanted to. + */ + override def resourceOffers(d: org.apache.mesos.SchedulerDriver, offers: JList[Offer]) { + stateLock.synchronized { + if (stopCalled) { + logDebug("Ignoring offers during shutdown") + // Driver should simply return a stopped status on race + // condition between this.stop() and completing here + offers.asScala.map(_.getId).foreach(d.declineOffer) + return + } + + logDebug(s"Received ${offers.size} resource offers.") + + val (matchedOffers, unmatchedOffers) = offers.asScala.partition { offer => + val offerAttributes = toAttributeMap(offer.getAttributesList) + matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + } + + declineUnmatchedOffers(d, unmatchedOffers) + handleMatchedOffers(d, matchedOffers) + } + } + + private def declineUnmatchedOffers( + driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { + offers.foreach { offer => + declineOffer( + driver, + offer, + Some("unmet constraints"), + Some(rejectOfferDurationForUnmetConstraints)) + } + } + + /** + * Launches executors on accepted offers, and declines unused offers. Executors are launched + * round-robin on offers. + * + * @param driver SchedulerDriver + * @param offers Mesos offers that match attribute constraints + */ + private def handleMatchedOffers( + driver: org.apache.mesos.SchedulerDriver, offers: mutable.Buffer[Offer]): Unit = { + val tasks = buildMesosTasks(offers) + for (offer <- offers) { + val offerAttributes = toAttributeMap(offer.getAttributesList) + val offerMem = getResource(offer.getResourcesList, "mem") + val offerCpus = getResource(offer.getResourcesList, "cpus") + val offerPorts = getRangeResource(offer.getResourcesList, "ports") + val id = offer.getId.getValue + + if (tasks.contains(offer.getId)) { // accept + val offerTasks = tasks(offer.getId) + + logDebug(s"Accepting offer: $id with attributes: $offerAttributes " + + s"mem: $offerMem cpu: $offerCpus ports: $offerPorts." + + s" Launching ${offerTasks.size} Mesos tasks.") + + for (task <- offerTasks) { + val taskId = task.getTaskId + val mem = getResource(task.getResourcesList, "mem") + val cpus = getResource(task.getResourcesList, "cpus") + val ports = getRangeResource(task.getResourcesList, "ports").mkString(",") + + logDebug(s"Launching Mesos task: ${taskId.getValue} with mem: $mem cpu: $cpus" + + s" ports: $ports") + } + + driver.launchTasks( + Collections.singleton(offer.getId), + offerTasks.asJava) + } else if (totalCoresAcquired >= maxCores) { + // Reject an offer for a configurable amount of time to avoid starving other frameworks + declineOffer(driver, + offer, + Some("reached spark.cores.max"), + Some(rejectOfferDurationForReachedMaxCores)) + } else { + declineOffer( + driver, + offer) + } + } + } + + /** + * Returns a map from OfferIDs to the tasks to launch on those offers. In order to maximize + * per-task memory and IO, tasks are round-robin assigned to offers. + * + * @param offers Mesos offers that match attribute constraints + * @return A map from OfferID to a list of Mesos tasks to launch on that offer + */ + private def buildMesosTasks(offers: mutable.Buffer[Offer]): Map[OfferID, List[MesosTaskInfo]] = { + // offerID -> tasks + val tasks = new mutable.HashMap[OfferID, List[MesosTaskInfo]].withDefaultValue(Nil) + + // offerID -> resources + val remainingResources = mutable.Map(offers.map(offer => + (offer.getId.getValue, offer.getResourcesList)): _*) + + var launchTasks = true + + // TODO(mgummelt): combine offers for a single slave + // + // round-robin create executors on the available offers + while (launchTasks) { + launchTasks = false + + for (offer <- offers) { + val slaveId = offer.getSlaveId.getValue + val offerId = offer.getId.getValue + val resources = remainingResources(offerId) + + if (canLaunchTask(slaveId, resources)) { + // Create a task + launchTasks = true + val taskId = newMesosTaskId() + val offerCPUs = getResource(resources, "cpus").toInt + val taskGPUs = Math.min( + Math.max(0, maxGpus - totalGpusAcquired), getResource(resources, "gpus").toInt) + + val taskCPUs = executorCores(offerCPUs) + val taskMemory = executorMemory(sc) + + slaves.getOrElseUpdate(slaveId, new Slave(offer.getHostname)).taskIDs.add(taskId) + + val (resourcesLeft, resourcesToUse) = + partitionTaskResources(resources, taskCPUs, taskMemory, taskGPUs) + + val taskBuilder = MesosTaskInfo.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId.toString).build()) + .setSlaveId(offer.getSlaveId) + .setCommand(createCommand(offer, taskCPUs + extraCoresPerExecutor, taskId)) + .setName(s"${sc.appName} $taskId") + + taskBuilder.addAllResources(resourcesToUse.asJava) + taskBuilder.setContainer(MesosSchedulerBackendUtil.containerInfo(sc.conf)) + + val labelsBuilder = taskBuilder.getLabelsBuilder + val labels = buildMesosLabels().asJava + + labelsBuilder.addAllLabels(labels) + + taskBuilder.setLabels(labelsBuilder) + + tasks(offer.getId) ::= taskBuilder.build() + remainingResources(offerId) = resourcesLeft.asJava + totalCoresAcquired += taskCPUs + coresByTaskId(taskId) = taskCPUs + if (taskGPUs > 0) { + totalGpusAcquired += taskGPUs + gpusByTaskId(taskId) = taskGPUs + } + } + } + } + tasks.toMap + } + + private def buildMesosLabels(): List[Label] = { + taskLabels.split(",").flatMap(label => + label.split(":") match { + case Array(key, value) => + Some(Label.newBuilder() + .setKey(key) + .setValue(value) + .build()) + case _ => + logWarning(s"Unable to parse $label into a key:value label for the task.") + None + } + ).toList + } + + /** Extracts task needed resources from a list of available resources. */ + private def partitionTaskResources( + resources: JList[Resource], + taskCPUs: Int, + taskMemory: Int, + taskGPUs: Int) + : (List[Resource], List[Resource]) = { + + // partition cpus & mem + val (afterCPUResources, cpuResourcesToUse) = partitionResources(resources, "cpus", taskCPUs) + val (afterMemResources, memResourcesToUse) = + partitionResources(afterCPUResources.asJava, "mem", taskMemory) + val (afterGPUResources, gpuResourcesToUse) = + partitionResources(afterMemResources.asJava, "gpus", taskGPUs) + + // If user specifies port numbers in SparkConfig then consecutive tasks will not be launched + // on the same host. This essentially means one executor per host. + // TODO: handle network isolator case + val (nonPortResources, portResourcesToUse) = + partitionPortResources(nonZeroPortValuesFromConfig(sc.conf), afterGPUResources) + + (nonPortResources, + cpuResourcesToUse ++ memResourcesToUse ++ portResourcesToUse ++ gpuResourcesToUse) + } + + private def canLaunchTask(slaveId: String, resources: JList[Resource]): Boolean = { + val offerMem = getResource(resources, "mem") + val offerCPUs = getResource(resources, "cpus").toInt + val cpus = executorCores(offerCPUs) + val mem = executorMemory(sc) + val ports = getRangeResource(resources, "ports") + val meetsPortRequirements = checkPorts(sc.conf, ports) + + cpus > 0 && + cpus <= offerCPUs && + cpus + totalCoresAcquired <= maxCores && + mem <= offerMem && + numExecutors() < executorLimit && + slaves.get(slaveId).map(_.taskFailures).getOrElse(0) < MAX_SLAVE_FAILURES && + meetsPortRequirements + } + + private def executorCores(offerCPUs: Int): Int = { + sc.conf.getInt("spark.executor.cores", + math.min(offerCPUs, maxCores - totalCoresAcquired)) + } + + override def statusUpdate(d: org.apache.mesos.SchedulerDriver, status: TaskStatus) { + val taskId = status.getTaskId.getValue + val slaveId = status.getSlaveId.getValue + val state = mesosToTaskState(status.getState) + + logInfo(s"Mesos task $taskId is now ${status.getState}") + + stateLock.synchronized { + val slave = slaves(slaveId) + + // If the shuffle service is enabled, have the driver register with each one of the + // shuffle services. This allows the shuffle services to clean up state associated with + // this application when the driver exits. There is currently not a great way to detect + // this through Mesos, since the shuffle services are set up independently. + if (state.equals(TaskState.RUNNING) && + shuffleServiceEnabled && + !slave.shuffleRegistered) { + assume(mesosExternalShuffleClient.isDefined, + "External shuffle client was not instantiated even though shuffle service is enabled.") + // TODO: Remove this and allow the MesosExternalShuffleService to detect + // framework termination when new Mesos Framework HTTP API is available. + val externalShufflePort = conf.getInt("spark.shuffle.service.port", 7337) + + logDebug(s"Connecting to shuffle service on slave $slaveId, " + + s"host ${slave.hostname}, port $externalShufflePort for app ${conf.getAppId}") + + mesosExternalShuffleClient.get + .registerDriverWithShuffleService( + slave.hostname, + externalShufflePort, + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", + s"${sc.conf.getTimeAsMs("spark.network.timeout", "120s")}ms"), + sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")) + slave.shuffleRegistered = true + } + + if (TaskState.isFinished(state)) { + // Remove the cores we have remembered for this task, if it's in the hashmap + for (cores <- coresByTaskId.get(taskId)) { + totalCoresAcquired -= cores + coresByTaskId -= taskId + } + // Also remove the gpus we have remembered for this task, if it's in the hashmap + for (gpus <- gpusByTaskId.get(taskId)) { + totalGpusAcquired -= gpus + gpusByTaskId -= taskId + } + // If it was a failure, mark the slave as failed for blacklisting purposes + if (TaskState.isFailed(state)) { + slave.taskFailures += 1 + + if (slave.taskFailures >= MAX_SLAVE_FAILURES) { + logInfo(s"Blacklisting Mesos slave $slaveId due to too many failures; " + + "is Spark installed on it?") + } + } + executorTerminated(d, slaveId, taskId, s"Executor finished with state $state") + // In case we'd rejected everything before but have now lost a node + d.reviveOffers() + } + } + } + + override def error(d: org.apache.mesos.SchedulerDriver, message: String) { + logError(s"Mesos error: $message") + scheduler.error(message) + } + + override def stop() { + // Make sure we're not launching tasks during shutdown + stateLock.synchronized { + if (stopCalled) { + logWarning("Stop called multiple times, ignoring") + return + } + stopCalled = true + super.stop() + } + + // Wait for executors to report done, or else mesosDriver.stop() will forcefully kill them. + // See SPARK-12330 + val startTime = System.nanoTime() + + // slaveIdsWithExecutors has no memory barrier, so this is eventually consistent + while (numExecutors() > 0 && + System.nanoTime() - startTime < shutdownTimeoutMS * 1000L * 1000L) { + Thread.sleep(100) + } + + if (numExecutors() > 0) { + logWarning(s"Timed out waiting for ${numExecutors()} remaining executors " + + s"to terminate within $shutdownTimeoutMS ms. This may leave temporary files " + + "on the mesos nodes.") + } + + // Close the mesos external shuffle client if used + mesosExternalShuffleClient.foreach(_.close()) + + if (schedulerDriver != null) { + schedulerDriver.stop() + } + } + + override def frameworkMessage( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} + + /** + * Called when a slave is lost or a Mesos task finished. Updates local view on + * what tasks are running. It also notifies the driver that an executor was removed. + */ + private def executorTerminated( + d: org.apache.mesos.SchedulerDriver, + slaveId: String, + taskId: String, + reason: String): Unit = { + stateLock.synchronized { + // Do not call removeExecutor() after this scheduler backend was stopped because + // removeExecutor() internally will send a message to the driver endpoint but + // the driver endpoint is not available now, otherwise an exception will be thrown. + if (!stopCalled) { + removeExecutor(taskId, SlaveLost(reason)) + } + slaves(slaveId).taskIDs.remove(taskId) + } + } + + override def slaveLost(d: org.apache.mesos.SchedulerDriver, slaveId: SlaveID): Unit = { + logInfo(s"Mesos slave lost: ${slaveId.getValue}") + } + + override def executorLost( + d: org.apache.mesos.SchedulerDriver, e: ExecutorID, s: SlaveID, status: Int): Unit = { + logInfo("Mesos executor lost: %s".format(e.getValue)) + } + + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + + override def doRequestTotalExecutors(requestedTotal: Int): Future[Boolean] = Future.successful { + // We don't truly know if we can fulfill the full amount of executors + // since at coarse grain it depends on the amount of slaves available. + logInfo("Capping the total amount of executors to " + requestedTotal) + executorLimitOption = Some(requestedTotal) + true + } + + override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future.successful { + if (schedulerDriver == null) { + logWarning("Asked to kill executors before the Mesos driver was started.") + false + } else { + for (executorId <- executorIds) { + val taskId = TaskID.newBuilder().setValue(executorId).build() + schedulerDriver.killTask(taskId) + } + // no need to adjust `executorLimitOption` since the AllocationManager already communicated + // the desired limit through a call to `doRequestTotalExecutors`. + // See [[o.a.s.scheduler.cluster.CoarseGrainedSchedulerBackend.killExecutors]] + true + } + } + + private def numExecutors(): Int = { + slaves.values.map(_.taskIDs.size).sum + } + + private def executorHostname(offer: Offer): String = { + if (sc.conf.getOption("spark.mesos.network.name").isDefined) { + // The agent's IP is not visible in a CNI container, so we bind to 0.0.0.0 + "0.0.0.0" + } else { + offer.getHostname + } + } +} + +private class Slave(val hostname: String) { + val taskIDs = new mutable.HashSet[String]() + var taskFailures = 0 + var shuffleRegistered = false +} 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 new file mode 100644 index 0000000000000..9d81025a3016b --- /dev/null +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -0,0 +1,558 @@ +/* + * 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.util.{List => JList} +import java.util.concurrent.CountDownLatch + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal + +import com.google.common.base.Splitter +import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler, SchedulerDriver} +import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.FrameworkInfo.Capability +import org.apache.mesos.protobuf.{ByteString, GeneratedMessage} + +import org.apache.spark.{SparkConf, SparkContext, SparkException} +import org.apache.spark.TaskState +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + + + +/** + * Shared trait for implementing a Mesos Scheduler. This holds common state and helper + * methods and Mesos scheduler will use. + */ +trait MesosSchedulerUtils extends Logging { + // Lock used to wait for scheduler to be registered + private final val registerLatch = new CountDownLatch(1) + + /** + * Creates a new MesosSchedulerDriver that communicates to the Mesos master. + * + * @param masterUrl The url to connect to Mesos master + * @param scheduler the scheduler class to receive scheduler callbacks + * @param sparkUser User to impersonate with when running tasks + * @param appName The framework name to display on the Mesos UI + * @param conf Spark configuration + * @param webuiUrl The WebUI url to link from Mesos UI + * @param checkpoint Option to checkpoint tasks for failover + * @param failoverTimeout Duration Mesos master expect scheduler to reconnect on disconnect + * @param frameworkId The id of the new framework + */ + 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 = { + val fwInfoBuilder = FrameworkInfo.newBuilder().setUser(sparkUser).setName(appName) + val credBuilder = Credential.newBuilder() + webuiUrl.foreach { url => fwInfoBuilder.setWebuiUrl(url) } + checkpoint.foreach { checkpoint => fwInfoBuilder.setCheckpoint(checkpoint) } + failoverTimeout.foreach { timeout => fwInfoBuilder.setFailoverTimeout(timeout) } + frameworkId.foreach { id => + fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build()) + } + fwInfoBuilder.setHostname(Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse( + conf.get(DRIVER_HOST_ADDRESS))) + conf.getOption("spark.mesos.principal").foreach { principal => + fwInfoBuilder.setPrincipal(principal) + credBuilder.setPrincipal(principal) + } + conf.getOption("spark.mesos.secret").foreach { secret => + credBuilder.setSecret(secret) + } + if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) { + throw new SparkException( + "spark.mesos.principal must be configured when spark.mesos.secret is set") + } + conf.getOption("spark.mesos.role").foreach { role => + fwInfoBuilder.setRole(role) + } + val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + if (maxGpus > 0) { + fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES)) + } + if (credBuilder.hasPrincipal) { + new MesosSchedulerDriver( + scheduler, fwInfoBuilder.build(), masterUrl, credBuilder.build()) + } else { + new MesosSchedulerDriver(scheduler, fwInfoBuilder.build(), masterUrl) + } + } + + /** + * Starts the MesosSchedulerDriver and stores the current running driver to this new instance. + * This driver is expected to not be running. + * This method returns only after the scheduler has registered with Mesos. + */ + def startScheduler(newDriver: SchedulerDriver): Unit = { + synchronized { + @volatile + var error: Option[Exception] = None + + // We create a new thread that will block inside `mesosDriver.run` + // until the scheduler exists + new Thread(Utils.getFormattedClassName(this) + "-mesos-driver") { + setDaemon(true) + override def run() { + try { + val ret = newDriver.run() + logInfo("driver.run() returned with code " + ret) + if (ret != null && ret.equals(Status.DRIVER_ABORTED)) { + error = Some(new SparkException("Error starting driver, DRIVER_ABORTED")) + markErr() + } + } catch { + case e: Exception => + logError("driver.run() failed", e) + error = Some(e) + markErr() + } + } + }.start() + + registerLatch.await() + + // propagate any error to the calling thread. This ensures that SparkContext creation fails + // without leaving a broken context that won't be able to schedule any tasks + error.foreach(throw _) + } + } + + def getResource(res: JList[Resource], name: String): Double = { + // A resource can have multiple values in the offer since it can either be from + // a specific role or wildcard. + res.asScala.filter(_.getName == name).map(_.getScalar.getValue).sum + } + + /** + * Transforms a range resource to a list of ranges + * + * @param res the mesos resource list + * @param name the name of the resource + * @return the list of ranges returned + */ + protected def getRangeResource(res: JList[Resource], name: String): List[(Long, Long)] = { + // A resource can have multiple values in the offer since it can either be from + // a specific role or wildcard. + res.asScala.filter(_.getName == name).flatMap(_.getRanges.getRangeList.asScala + .map(r => (r.getBegin, r.getEnd)).toList).toList + } + + /** + * Signal that the scheduler has registered with Mesos. + */ + protected def markRegistered(): Unit = { + registerLatch.countDown() + } + + protected def markErr(): Unit = { + registerLatch.countDown() + } + + def createResource(name: String, amount: Double, role: Option[String] = None): Resource = { + val builder = Resource.newBuilder() + .setName(name) + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(amount).build()) + + role.foreach { r => builder.setRole(r) } + + builder.build() + } + + /** + * Partition the existing set of resources into two groups, those remaining to be + * scheduled and those requested to be used for a new task. + * + * @param resources The full list of available resources + * @param resourceName The name of the resource to take from the available resources + * @param amountToUse The amount of resources to take from the available resources + * @return The remaining resources list and the used resources list. + */ + def partitionResources( + resources: JList[Resource], + resourceName: String, + amountToUse: Double): (List[Resource], List[Resource]) = { + var remain = amountToUse + var requestedResources = new ArrayBuffer[Resource] + val remainingResources = resources.asScala.map { + case r => + if (remain > 0 && + r.getType == Value.Type.SCALAR && + r.getScalar.getValue > 0.0 && + r.getName == resourceName) { + val usage = Math.min(remain, r.getScalar.getValue) + requestedResources += createResource(resourceName, usage, Some(r.getRole)) + remain -= usage + createResource(resourceName, r.getScalar.getValue - usage, Some(r.getRole)) + } else { + r + } + } + + // Filter any resource that has depleted. + val filteredResources = + remainingResources.filter(r => r.getType != Value.Type.SCALAR || r.getScalar.getValue > 0.0) + + (filteredResources.toList, requestedResources.toList) + } + + /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ + protected def getAttribute(attr: Attribute): (String, Set[String]) = { + (attr.getName, attr.getText.getValue.split(',').toSet) + } + + + /** Build a Mesos resource protobuf object */ + protected def createResource(resourceName: String, quantity: Double): Protos.Resource = { + Resource.newBuilder() + .setName(resourceName) + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) + .build() + } + + /** + * Converts the attributes from the resource offer into a Map of name to Attribute Value + * The attribute values are the mesos attribute types and they are + * + * @param offerAttributes the attributes offered + * @return + */ + protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { + offerAttributes.asScala.map { attr => + val attrValue = attr.getType match { + case Value.Type.SCALAR => attr.getScalar + case Value.Type.RANGES => attr.getRanges + case Value.Type.SET => attr.getSet + case Value.Type.TEXT => attr.getText + } + (attr.getName, attrValue) + }.toMap + } + + + /** + * Match the requirements (if any) to the offer attributes. + * if attribute requirements are not specified - return true + * else if attribute is defined and no values are given, simple attribute presence is performed + * else if attribute name and value is specified, subset match is performed on slave attributes + */ + def matchesAttributeRequirements( + slaveOfferConstraints: Map[String, Set[String]], + offerAttributes: Map[String, GeneratedMessage]): Boolean = { + slaveOfferConstraints.forall { + // offer has the required attribute and subsumes the required values for that attribute + case (name, requiredValues) => + offerAttributes.get(name) match { + case None => false + case Some(_) if requiredValues.isEmpty => true // empty value matches presence + case Some(scalarValue: Value.Scalar) => + // check if provided values is less than equal to the offered values + requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) + case Some(rangeValue: Value.Range) => + val offerRange = rangeValue.getBegin to rangeValue.getEnd + // Check if there is some required value that is between the ranges specified + // Note: We only support the ability to specify discrete values, in the future + // we may expand it to subsume ranges specified with a XX..YY value or something + // similar to that. + requiredValues.map(_.toLong).exists(offerRange.contains(_)) + case Some(offeredValue: Value.Set) => + // check if the specified required values is a subset of offered set + requiredValues.subsetOf(offeredValue.getItemList.asScala.toSet) + case Some(textValue: Value.Text) => + // check if the specified value is equal, if multiple values are specified + // we succeed if any of them match. + requiredValues.contains(textValue.getValue) + } + } + } + + /** + * Parses the attributes constraints provided to spark and build a matching data struct: + * {@literal Map[, Set[values-to-match]} + * The constraints are specified as ';' separated key-value pairs where keys and values + * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for + * multiple values (comma separated). For example: + * {{{ + * parseConstraintString("os:centos7;zone:us-east-1a,us-east-1b") + * // would result in + * + * Map( + * "os" -> Set("centos7"), + * "zone": -> Set("us-east-1a", "us-east-1b") + * ) + * }}} + * + * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ + * https://github.com/apache/mesos/blob/master/src/common/values.cpp + * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp + * + * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated + * by ':') + * @return Map of constraints to match resources offers. + */ + def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { + /* + Based on mesos docs: + attributes : attribute ( ";" attribute )* + attribute : labelString ":" ( labelString | "," )+ + labelString : [a-zA-Z0-9_/.-] + */ + val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') + // kv splitter + if (constraintsVal.isEmpty) { + Map() + } else { + try { + splitter.split(constraintsVal).asScala.toMap.mapValues(v => + if (v == null || v.isEmpty) { + Set[String]() + } else { + v.split(',').toSet + } + ) + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) + } + } + } + + // These defaults copied from YARN + private val MEMORY_OVERHEAD_FRACTION = 0.10 + private val MEMORY_OVERHEAD_MINIMUM = 384 + + /** + * Return the amount of memory to allocate to each executor, taking into account + * container overheads. + * + * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value + * @return memory requirement as (0.1 * memoryOverhead) or MEMORY_OVERHEAD_MINIMUM + * (whichever is larger) + */ + def executorMemory(sc: SparkContext): Int = { + sc.conf.getInt("spark.mesos.executor.memoryOverhead", + math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + + sc.executorMemory + } + + def setupUris(uris: String, + builder: CommandInfo.Builder, + useFetcherCache: Boolean = false): Unit = { + uris.split(",").foreach { uri => + builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache)) + } + } + + private def getRejectOfferDurationStr(conf: SparkConf): String = { + conf.get("spark.mesos.rejectOfferDuration", "120s") + } + + protected def getRejectOfferDuration(conf: SparkConf): Long = { + Utils.timeStringAsSeconds(getRejectOfferDurationStr(conf)) + } + + protected def getRejectOfferDurationForUnmetConstraints(conf: SparkConf): Long = { + conf.getTimeAsSeconds( + "spark.mesos.rejectOfferDurationForUnmetConstraints", + getRejectOfferDurationStr(conf)) + } + + protected def getRejectOfferDurationForReachedMaxCores(conf: SparkConf): Long = { + conf.getTimeAsSeconds( + "spark.mesos.rejectOfferDurationForReachedMaxCores", + getRejectOfferDurationStr(conf)) + } + + /** + * Checks executor ports if they are within some range of the offered list of ports ranges, + * + * @param conf the Spark Config + * @param ports the list of ports to check + * @return true if ports are within range false otherwise + */ + protected def checkPorts(conf: SparkConf, ports: List[(Long, Long)]): Boolean = { + + def checkIfInRange(port: Long, ps: List[(Long, Long)]): Boolean = { + ps.exists{case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port } + } + + val portsToCheck = nonZeroPortValuesFromConfig(conf) + val withinRange = portsToCheck.forall(p => checkIfInRange(p, ports)) + // make sure we have enough ports to allocate per offer + val enoughPorts = + ports.map{case (rangeStart, rangeEnd) => rangeEnd - rangeStart + 1}.sum >= portsToCheck.size + enoughPorts && withinRange + } + + /** + * Partitions port resources. + * + * @param requestedPorts non-zero ports to assign + * @param offeredResources the resources offered + * @return resources left, port resources to be used. + */ + def partitionPortResources(requestedPorts: List[Long], offeredResources: List[Resource]) + : (List[Resource], List[Resource]) = { + if (requestedPorts.isEmpty) { + (offeredResources, List[Resource]()) + } else { + // partition port offers + val (resourcesWithoutPorts, portResources) = filterPortResources(offeredResources) + + val portsAndRoles = requestedPorts. + map(x => (x, findPortAndGetAssignedRangeRole(x, portResources))) + + val assignedPortResources = createResourcesFromPorts(portsAndRoles) + + // ignore non-assigned port resources, they will be declined implicitly by mesos + // no need for splitting port resources. + (resourcesWithoutPorts, assignedPortResources) + } + } + + val managedPortNames = List("spark.executor.port", BLOCK_MANAGER_PORT.key) + + /** + * The values of the non-zero ports to be used by the executor process. + * + * @param conf the spark config to use + * @return the ono-zero values of the ports + */ + def nonZeroPortValuesFromConfig(conf: SparkConf): List[Long] = { + managedPortNames.map(conf.getLong(_, 0)).filter( _ != 0) + } + + /** Creates a mesos resource for a specific port number. */ + private def createResourcesFromPorts(portsAndRoles: List[(Long, String)]) : List[Resource] = { + portsAndRoles.flatMap{ case (port, role) => + createMesosPortResource(List((port, port)), Some(role))} + } + + /** Helper to create mesos resources for specific port ranges. */ + private def createMesosPortResource( + ranges: List[(Long, Long)], + role: Option[String] = None): List[Resource] = { + ranges.map { case (rangeStart, rangeEnd) => + val rangeValue = Value.Range.newBuilder() + .setBegin(rangeStart) + .setEnd(rangeEnd) + val builder = Resource.newBuilder() + .setName("ports") + .setType(Value.Type.RANGES) + .setRanges(Value.Ranges.newBuilder().addRange(rangeValue)) + role.foreach(r => builder.setRole(r)) + builder.build() + } + } + + /** + * Helper to assign a port to an offered range and get the latter's role + * info to use it later on. + */ + private def findPortAndGetAssignedRangeRole(port: Long, portResources: List[Resource]) + : String = { + + val ranges = portResources. + map(resource => + (resource.getRole, resource.getRanges.getRangeList.asScala + .map(r => (r.getBegin, r.getEnd)).toList)) + + val rangePortRole = ranges + .find { case (role, rangeList) => rangeList + .exists{ case (rangeStart, rangeEnd) => rangeStart <= port & rangeEnd >= port}} + // this is safe since we have previously checked about the ranges (see checkPorts method) + rangePortRole.map{ case (role, rangeList) => role}.get + } + + /** Retrieves the port resources from a list of mesos offered resources */ + private def filterPortResources(resources: List[Resource]): (List[Resource], List[Resource]) = { + resources.partition { r => !(r.getType == Value.Type.RANGES && r.getName == "ports") } + } + + /** + * spark.mesos.driver.frameworkId is set by the cluster dispatcher to correlate driver + * submissions with frameworkIDs. However, this causes issues when a driver process launches + * more than one framework (more than one SparkContext(, because they all try to register with + * the same frameworkID. To enforce that only the first driver registers with the configured + * framework ID, the driver calls this method after the first registration. + */ + def unsetFrameworkID(sc: SparkContext) { + sc.conf.remove("spark.mesos.driver.frameworkId") + System.clearProperty("spark.mesos.driver.frameworkId") + } + + def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match { + case MesosTaskState.TASK_STAGING | MesosTaskState.TASK_STARTING => TaskState.LAUNCHING + case MesosTaskState.TASK_RUNNING | MesosTaskState.TASK_KILLING => TaskState.RUNNING + case MesosTaskState.TASK_FINISHED => TaskState.FINISHED + case MesosTaskState.TASK_FAILED => TaskState.FAILED + case MesosTaskState.TASK_KILLED => TaskState.KILLED + case MesosTaskState.TASK_LOST | MesosTaskState.TASK_ERROR => TaskState.LOST + } + + def taskStateToMesos(state: TaskState.TaskState): MesosTaskState = state match { + case TaskState.LAUNCHING => MesosTaskState.TASK_STARTING + case TaskState.RUNNING => MesosTaskState.TASK_RUNNING + case TaskState.FINISHED => MesosTaskState.TASK_FINISHED + case TaskState.FAILED => MesosTaskState.TASK_FAILED + case TaskState.KILLED => MesosTaskState.TASK_KILLED + case TaskState.LOST => MesosTaskState.TASK_LOST + } + + protected def declineOffer( + driver: org.apache.mesos.SchedulerDriver, + offer: Offer, + reason: Option[String] = None, + refuseSeconds: Option[Long] = None): Unit = { + + val id = offer.getId.getValue + val offerAttributes = toAttributeMap(offer.getAttributesList) + val mem = getResource(offer.getResourcesList, "mem") + val cpus = getResource(offer.getResourcesList, "cpus") + val ports = getRangeResource(offer.getResourcesList, "ports") + + logDebug(s"Declining offer: $id with " + + s"attributes: $offerAttributes " + + s"mem: $mem " + + s"cpu: $cpus " + + s"port: $ports " + + refuseSeconds.map(s => s"for ${s} seconds ").getOrElse("") + + reason.map(r => s" (reason: $r)").getOrElse("")) + + refuseSeconds match { + case Some(seconds) => + val filters = Filters.newBuilder().setRefuseSeconds(seconds).build() + driver.declineOffer(offer.getId, filters) + case _ => + driver.declineOffer(offer.getId) + } + } +} 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 new file mode 100644 index 0000000000000..c040f05d93b3a --- /dev/null +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -0,0 +1,654 @@ +/* + * 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.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.reflect.ClassTag + +import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} +import org.apache.mesos.Protos._ +import org.mockito.Matchers +import org.mockito.Matchers._ +import org.mockito.Mockito._ +import org.scalatest.concurrent.ScalaFutures +import org.scalatest.mock.MockitoSugar +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} +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.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.mesos.Utils._ + +class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite + with LocalSparkContext + with MockitoSugar + with BeforeAndAfter + with ScalaFutures { + + private var sparkConf: SparkConf = _ + private var driver: SchedulerDriver = _ + private var taskScheduler: TaskSchedulerImpl = _ + private var backend: MesosCoarseGrainedSchedulerBackend = _ + private var externalShuffleClient: MesosExternalShuffleClient = _ + private var driverEndpoint: RpcEndpointRef = _ + @volatile private var stopCalled = false + + // All 'requests' to the scheduler run immediately on the same thread, so + // demand that all futures have their value available immediately. + implicit override val patienceConfig = PatienceConfig(timeout = Duration(0, TimeUnit.SECONDS)) + + test("mesos supports killing and limiting executors") { + setBackend() + sparkConf.set("spark.driver.host", "driverHost") + sparkConf.set("spark.driver.port", "1234") + + val minMem = backend.executorMemory(sc) + val minCpu = 4 + val offers = List(Resources(minMem, minCpu)) + + // launches a task on a valid offer + offerResources(offers) + verifyTaskLaunched(driver, "o1") + + // kills executors + assert(backend.doRequestTotalExecutors(0).futureValue) + assert(backend.doKillExecutors(Seq("0")).futureValue) + val taskID0 = createTaskId("0") + verify(driver, times(1)).killTask(taskID0) + + // doesn't launch a new task when requested executors == 0 + offerResources(offers, 2) + verifyDeclinedOffer(driver, createOfferId("o2")) + + // Launches a new task when requested executors is positive + backend.doRequestTotalExecutors(2) + offerResources(offers, 2) + verifyTaskLaunched(driver, "o2") + } + + test("mesos supports killing and relaunching tasks with executors") { + setBackend() + + // launches a task on a valid offer + val minMem = backend.executorMemory(sc) + 1024 + val minCpu = 4 + val offer1 = Resources(minMem, minCpu) + val offer2 = Resources(minMem, 1) + offerResources(List(offer1, offer2)) + verifyTaskLaunched(driver, "o1") + + // accounts for a killed task + val status = createTaskStatus("0", "s1", TaskState.TASK_KILLED) + backend.statusUpdate(driver, status) + verify(driver, times(1)).reviveOffers() + + // Launches a new task on a valid offer from the same slave + offerResources(List(offer2)) + verifyTaskLaunched(driver, "o2") + } + + test("mesos supports spark.executor.cores") { + val executorCores = 4 + setBackend(Map("spark.executor.cores" -> executorCores.toString)) + + val executorMemory = backend.executorMemory(sc) + val offers = List(Resources(executorMemory * 2, executorCores + 1)) + offerResources(offers) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") + assert(cpus == executorCores) + } + + test("mesos supports unset spark.executor.cores") { + setBackend() + + val executorMemory = backend.executorMemory(sc) + val offerCores = 10 + offerResources(List(Resources(executorMemory * 2, offerCores))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") + assert(cpus == offerCores) + } + + test("mesos does not acquire more than spark.cores.max") { + val maxCores = 10 + setBackend(Map("spark.cores.max" -> maxCores.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory, maxCores + 1))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val cpus = backend.getResource(taskInfos.head.getResourcesList, "cpus") + assert(cpus == maxCores) + } + + test("mesos does not acquire gpus if not specified") { + setBackend() + + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory, 1, 1))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") + assert(gpus == 0.0) + } + + + test("mesos does not acquire more than spark.mesos.gpus.max") { + val maxGpus = 5 + setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 1) + + val gpus = backend.getResource(taskInfos.head.getResourcesList, "gpus") + assert(gpus == maxGpus) + } + + + test("mesos declines offers that violate attribute constraints") { + setBackend(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)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List( + Resources(executorMemory, maxCores + 1), + Resources(executorMemory, maxCores + 1))) + + verifyTaskLaunched(driver, "o1") + verifyDeclinedOffer(driver, createOfferId("o2"), true) + } + + test("mesos assigns tasks round-robin on offers") { + val executorCores = 4 + val maxCores = executorCores * 2 + setBackend(Map("spark.executor.cores" -> executorCores.toString, + "spark.cores.max" -> maxCores.toString)) + + val executorMemory = backend.executorMemory(sc) + offerResources(List( + Resources(executorMemory * 2, executorCores * 2), + Resources(executorMemory * 2, executorCores * 2))) + + verifyTaskLaunched(driver, "o1") + verifyTaskLaunched(driver, "o2") + } + + test("mesos creates multiple executors on a single slave") { + val executorCores = 4 + setBackend(Map("spark.executor.cores" -> executorCores.toString)) + + // offer with room for two executors + val executorMemory = backend.executorMemory(sc) + offerResources(List(Resources(executorMemory * 2, executorCores * 2))) + + // verify two executors were started on a single offer + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.length == 2) + } + + test("mesos doesn't register twice with the same shuffle service") { + setBackend(Map("spark.shuffle.service.enabled" -> "true")) + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + verifyTaskLaunched(driver, "o1") + + val offer2 = createOffer("o2", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer2).asJava) + verifyTaskLaunched(driver, "o2") + + val status1 = createTaskStatus("0", "s1", TaskState.TASK_RUNNING) + backend.statusUpdate(driver, status1) + + val status2 = createTaskStatus("1", "s1", TaskState.TASK_RUNNING) + backend.statusUpdate(driver, status2) + verify(externalShuffleClient, times(1)) + .registerDriverWithShuffleService(anyString, anyInt, anyLong, anyLong) + } + + test("Port offer decline when there is no appropriate range") { + setBackend(Map(BLOCK_MANAGER_PORT.key -> "30100")) + val offeredPorts = (31100L, 31200L) + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) + backend.resourceOffers(driver, List(offer1).asJava) + verify(driver, times(1)).declineOffer(offer1.getId) + } + + test("Port offer accepted when ephemeral ports are used") { + setBackend() + val offeredPorts = (31100L, 31200L) + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) + backend.resourceOffers(driver, List(offer1).asJava) + verifyTaskLaunched(driver, "o1") + } + + test("Port offer accepted with user defined port numbers") { + val port = 30100 + setBackend(Map(BLOCK_MANAGER_PORT.key -> s"$port")) + val offeredPorts = (30000L, 31000L) + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu, Some(offeredPorts)) + backend.resourceOffers(driver, List(offer1).asJava) + val taskInfo = verifyTaskLaunched(driver, "o1") + + val taskPortResources = taskInfo.head.getResourcesList.asScala. + find(r => r.getType == Value.Type.RANGES && r.getName == "ports") + + val isPortInOffer = (r: Resource) => { + r.getRanges().getRangeList + .asScala.exists(range => range.getBegin == port && range.getEnd == port) + } + assert(taskPortResources.exists(isPortInOffer)) + } + + test("mesos kills an executor when told") { + setBackend() + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + verifyTaskLaunched(driver, "o1") + + backend.doKillExecutors(List("0")) + verify(driver, times(1)).killTask(createTaskId("0")) + } + + test("weburi is set in created scheduler driver") { + initializeSparkConf() + sc = new SparkContext(sparkConf) + + val taskScheduler = mock[TaskSchedulerImpl] + when(taskScheduler.sc).thenReturn(sc) + + val driver = mock[SchedulerDriver] + when(driver.start()).thenReturn(Protos.Status.DRIVER_RUNNING) + + val securityManager = mock[SecurityManager] + + 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 = { + markRegistered() + assert(webuiUrl.isDefined) + assert(webuiUrl.get.equals("http://webui")) + driver + } + } + + backend.start() + } + + test("honors unset spark.mesos.containerizer") { + setBackend(Map("spark.mesos.executor.docker.image" -> "test")) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.DOCKER) + } + + test("honors spark.mesos.containerizer=\"mesos\"") { + setBackend(Map( + "spark.mesos.executor.docker.image" -> "test", + "spark.mesos.containerizer" -> "mesos")) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val taskInfos = verifyTaskLaunched(driver, "o1") + assert(taskInfos.head.getContainer.getType == ContainerInfo.Type.MESOS) + } + + test("docker settings are reflected in created tasks") { + setBackend(Map( + "spark.mesos.executor.docker.image" -> "some_image", + "spark.mesos.executor.docker.forcePullImage" -> "true", + "spark.mesos.executor.docker.volumes" -> "/host_vol:/container_vol:ro", + "spark.mesos.executor.docker.portmaps" -> "8080:80:tcp" + )) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val launchedTasks = verifyTaskLaunched(driver, "o1") + assert(launchedTasks.size == 1) + + val containerInfo = launchedTasks.head.getContainer + assert(containerInfo.getType == ContainerInfo.Type.DOCKER) + + val volumes = containerInfo.getVolumesList.asScala + assert(volumes.size == 1) + + val volume = volumes.head + assert(volume.getHostPath == "/host_vol") + assert(volume.getContainerPath == "/container_vol") + assert(volume.getMode == Volume.Mode.RO) + + val dockerInfo = containerInfo.getDocker + + val portMappings = dockerInfo.getPortMappingsList.asScala + assert(portMappings.size == 1) + + val portMapping = portMappings.head + assert(portMapping.getHostPort == 8080) + assert(portMapping.getContainerPort == 80) + assert(portMapping.getProtocol == "tcp") + } + + test("force-pull-image option is disabled by default") { + setBackend(Map( + "spark.mesos.executor.docker.image" -> "some_image" + )) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val launchedTasks = verifyTaskLaunched(driver, "o1") + assert(launchedTasks.size == 1) + + val containerInfo = launchedTasks.head.getContainer + assert(containerInfo.getType == ContainerInfo.Type.DOCKER) + + val dockerInfo = containerInfo.getDocker + + assert(dockerInfo.getImage == "some_image") + assert(!dockerInfo.getForcePullImage) + } + + test("mesos supports spark.executor.uri") { + val url = "spark.spark.spark.com" + setBackend(Map( + "spark.executor.uri" -> url + ), null) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val launchedTasks = verifyTaskLaunched(driver, "o1") + assert(launchedTasks.head.getCommand.getUrisList.asScala(0).getValue == url) + } + + test("mesos supports setting fetcher cache") { + val url = "spark.spark.spark.com" + setBackend(Map( + "spark.mesos.fetcherCache.enable" -> "true", + "spark.executor.uri" -> url + ), null) + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.size() == 1) + assert(uris.asScala.head.getCache) + } + + test("mesos supports disabling fetcher cache") { + val url = "spark.spark.spark.com" + setBackend(Map( + "spark.mesos.fetcherCache.enable" -> "false", + "spark.executor.uri" -> url + ), null) + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.size() == 1) + assert(!uris.asScala.head.getCache) + } + + test("mesos sets task name to spark.app.name") { + setBackend() + + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + + // Add " 0" to the taskName to match the executor number that is appended + assert(launchedTasks.head.getName == "test-mesos-dynamic-alloc 0") + } + + test("mesos sets configurable labels on tasks") { + val taskLabelsString = "mesos:test,label:test" + setBackend(Map( + "spark.mesos.task.labels" -> taskLabelsString + )) + + // Build up the labels + val taskLabels = Protos.Labels.newBuilder() + .addLabels(Protos.Label.newBuilder() + .setKey("mesos").setValue("test").build()) + .addLabels(Protos.Label.newBuilder() + .setKey("label").setValue("test").build()) + .build() + + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + + val labels = launchedTasks.head.getLabels + + assert(launchedTasks.head.getLabels.equals(taskLabels)) + } + + test("mesos ignored invalid labels and sets configurable labels on tasks") { + val taskLabelsString = "mesos:test,label:test,incorrect:label:here" + setBackend(Map( + "spark.mesos.task.labels" -> taskLabelsString + )) + + // Build up the labels + val taskLabels = Protos.Labels.newBuilder() + .addLabels(Protos.Label.newBuilder() + .setKey("mesos").setValue("test").build()) + .addLabels(Protos.Label.newBuilder() + .setKey("label").setValue("test").build()) + .build() + + val offers = List(Resources(backend.executorMemory(sc), 1)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + + val labels = launchedTasks.head.getLabels + + assert(launchedTasks.head.getLabels.equals(taskLabels)) + } + + test("mesos supports spark.mesos.network.name") { + setBackend(Map( + "spark.mesos.network.name" -> "test-network-name" + )) + + val (mem, cpu) = (backend.executorMemory(sc), 4) + + val offer1 = createOffer("o1", "s1", mem, cpu) + backend.resourceOffers(driver, List(offer1).asJava) + + val launchedTasks = verifyTaskLaunched(driver, "o1") + val networkInfos = launchedTasks.head.getContainer.getNetworkInfosList + assert(networkInfos.size == 1) + assert(networkInfos.get(0).getName == "test-network-name") + } + + test("supports spark.scheduler.minRegisteredResourcesRatio") { + val expectedCores = 1 + setBackend(Map( + "spark.cores.max" -> expectedCores.toString, + "spark.scheduler.minRegisteredResourcesRatio" -> "1.0")) + + val offers = List(Resources(backend.executorMemory(sc), expectedCores)) + offerResources(offers) + val launchedTasks = verifyTaskLaunched(driver, "o1") + assert(!backend.isReady) + + registerMockExecutor(launchedTasks(0).getTaskId.getValue, "s1", expectedCores) + assert(backend.isReady) + } + + private case class Resources(mem: Int, cpus: Int, gpus: Int = 0) + + private def registerMockExecutor(executorId: String, slaveId: String, cores: Integer) = { + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty) + + backend.driverEndpoint.askSync[Boolean](message) + } + + private def verifyDeclinedOffer(driver: SchedulerDriver, + offerId: OfferID, + filter: Boolean = false): Unit = { + if (filter) { + verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters]) + } else { + verify(driver, times(1)).declineOffer(Matchers.eq(offerId)) + } + } + + private def offerResources(offers: List[Resources], startId: Int = 1): Unit = { + val mesosOffers = offers.zipWithIndex.map {case (offer, i) => + createOffer(s"o${i + startId}", s"s${i + startId}", offer.mem, offer.cpus, None, offer.gpus)} + + backend.resourceOffers(driver, mesosOffers.asJava) + } + + private def createTaskStatus(taskId: String, slaveId: String, state: TaskState): TaskStatus = { + TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(taskId).build()) + .setSlaveId(SlaveID.newBuilder().setValue(slaveId).build()) + .setState(state) + .build + } + + private def createSchedulerBackend( + taskScheduler: TaskSchedulerImpl, + driver: SchedulerDriver, + shuffleClient: MesosExternalShuffleClient) = { + val securityManager = mock[SecurityManager] + + 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 + + override protected def getShuffleClient(): MesosExternalShuffleClient = shuffleClient + + // override to avoid race condition with the driver thread on `mesosDriver` + override def startScheduler(newDriver: SchedulerDriver): Unit = {} + + override def stopExecutors(): Unit = { + stopCalled = true + } + } + backend.start() + backend.registered(driver, Utils.TEST_FRAMEWORK_ID, Utils.TEST_MASTER_INFO) + backend + } + + private def initializeSparkConf( + sparkConfVars: Map[String, String] = null, + home: String = "/path"): Unit = { + sparkConf = (new SparkConf) + .setMaster("local[*]") + .setAppName("test-mesos-dynamic-alloc") + .set("spark.mesos.driver.webui.url", "http://webui") + + if (home != null) { + sparkConf.setSparkHome(home) + } + + 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) + } +} 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 12158d0f2d867..cbc6e60e839c1 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, None) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { From 810c6b26e3830e0f4e08e66df2d6a6f50cc65c7b Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 13:14:16 -0700 Subject: [PATCH 19/40] test ConfigurableCredentialManager.obtainUserTokens --- .../security/ConfigurableCredentialManager.scala | 2 +- .../apache/spark/HeartbeatReceiverSuite.scala | 2 +- .../ConfigurableCredentialManagerSuite.scala | 16 ++++++++++++---- 3 files changed, 14 insertions(+), 6 deletions(-) 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 e8bf551b7383a..4487e387127e9 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 @@ -98,7 +98,7 @@ private[spark] class ConfigurableCredentialManager( .getOrElse(isEnabledDeprecated) } - protected def loadCredentialProviders: List[ServiceCredentialProvider] = { + private def loadCredentialProviders: List[ServiceCredentialProvider] = { ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) .asScala.toList } diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 14d0575a13cc7..88916488c0def 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, None) { + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { 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 414e65a6fad19..1a8cf46247136 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 @@ -19,10 +19,9 @@ 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.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token import org.scalatest.{BeforeAndAfter, Matchers} - import org.apache.spark.{SparkConf, SparkFunSuite} class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -43,10 +42,11 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) credentialManager.getServiceCredentialProvider("hbase") should not be (None) credentialManager.getServiceCredentialProvider("hive") should not be (None) + credentialManager.getServiceCredentialProvider("bogus") should be (None) } test("disable hive credential provider") { - sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") + sparkConf.set("spark.security.credentials.hive.enabled", "false") credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) @@ -56,7 +56,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit test("using deprecated configurations") { sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") - sparkConf.set("spark.yarn.security.tokens.hive.enabled", "false") + sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) credentialManager.getServiceCredentialProvider("hadoopfs") should be (None) @@ -77,6 +77,14 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit tokens.iterator().next().getService should be (new Text("test")) } + test("verify obtaining user credentials") { + credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + + val initNumTokens = UserGroupInformation.getCurrentUser.getCredentials.numberOfTokens + val creds = credentialManager.obtainUserCredentials + creds.numberOfTokens() should be (initNumTokens + 1) + } + test("verify getting credential renewal info") { credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) val creds = new Credentials() From ad4e33b9f379538ddcbdb9468f4bb39cafc46057 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 14:03:41 -0700 Subject: [PATCH 20/40] add tests --- ....deploy.security.ServiceCredentialProvider | 3 ++ .../deploy/security/HadoopAccessManager.scala | 5 ++- .../ConfigurableCredentialManagerSuite.scala | 10 +++-- .../security/ServiceCredentialProvider.scala | 6 +-- .../YARNConfigurableCredentialManager.scala | 2 +- ...oy.yarn.security.ServiceCredentialProvider | 1 + ...NConfigurableCredentialManagerSuite.scala} | 45 ++++++++++++++----- ...ala => YARNHadoopAccessManagerSuite.scala} | 45 +++++-------------- 8 files changed, 62 insertions(+), 55 deletions(-) create mode 100644 core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider create mode 100644 resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider rename resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/{ConfigurableCredentialManagerSuite.scala => YARNConfigurableCredentialManagerSuite.scala} (53%) rename resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/{YARNHadoopFSCredentialProviderSuite.scala => YARNHadoopAccessManagerSuite.scala} (56%) diff --git a/core/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 new file mode 100644 index 0000000000000..9ffeb4d500296 --- /dev/null +++ b/core/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/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala index 8d6b4849fd510..e617e94440f8a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala @@ -20,8 +20,9 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -/** [[HadoopAccessManager]] returns information related to how Hadoop delegation tokens should be - * fetched. +/** + * Methods in [[HadoopAccessManager]] return scheduler-specific information related to how Hadoop + * delegation tokens should be fetched. */ private[spark] trait HadoopAccessManager { 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 1a8cf46247136..4368882532c7a 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 @@ -17,11 +17,13 @@ package org.apache.spark.deploy.security +import org.scalatest.{BeforeAndAfter, Matchers} + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token -import org.scalatest.{BeforeAndAfter, Matchers} + import org.apache.spark.{SparkConf, SparkFunSuite} class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -136,9 +138,9 @@ class TestCredentialProvider extends ServiceCredentialProvider { override def credentialsRequired(conf: Configuration): Boolean = true override def obtainCredentials( - hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, - creds: Credentials): Option[Long] = { + hadoopConf: Configuration, + hadoopAccessManager: HadoopAccessManager, + creds: Credentials): Option[Long] = { if (creds == null) { // Guard out other unit test failures. return None 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 index 6cb77adffcb44..7f5aa92089fcc 100644 --- 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 @@ -53,9 +53,9 @@ trait ServiceCredentialProvider { * renewal, otherwise None should be returned. */ def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] } 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 index 443b0d152db99..42e225baeb896 100644 --- 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 @@ -40,7 +40,7 @@ private[yarn] class YARNConfigurableCredentialManager( hadoopConf, new YARNHadoopAccessManager(hadoopConf, sparkConf)) { - private val deprecatedCredentialProviders = getDeprecatedCredentialProviders + val deprecatedCredentialProviders = getDeprecatedCredentialProviders def getDeprecatedCredentialProviders: Map[String, org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = { 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.yarn.security.ServiceCredentialProvider new file mode 100644 index 0000000000000..dfea80b38aed7 --- /dev/null +++ b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider @@ -0,0 +1 @@ +org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider \ No newline at end of file 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/YARNConfigurableCredentialManagerSuite.scala similarity index 53% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/ConfigurableCredentialManagerSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index 0f65265af8b2b..df13c9eb19ff0 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/YARNConfigurableCredentialManagerSuite.scala @@ -18,12 +18,15 @@ package org.apache.spark.deploy.yarn.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.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.security.ConfigurableCredentialManager -class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { +class YARNConfigurableCredentialManagerSuite + extends SparkFunSuite with Matchers with BeforeAndAfter { private var credentialManager: YARNConfigurableCredentialManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -33,21 +36,39 @@ 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") + test("Correctly loads deprecated credential providers") { + credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) - super.afterAll() + credentialManager.deprecatedCredentialProviders.get("yarn-test") should not be (None) } +} - test("Correctly load ") { - credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) +class YARNTestCredentialProvider extends ServiceCredentialProvider { + val tokenRenewalInterval = 86400 * 1000L + var timeOfNextTokenRenewal = 0L + + override def serviceName: String = "yarn-test" + + override def credentialsRequired(conf: Configuration): Boolean = true + + override def obtainCredentials( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + if (creds == null) { + // Guard out other unit test failures. + return None + } + + val emptyToken = new Token() + emptyToken.setService(new Text(serviceName)) + creds.addToken(emptyToken.getService, emptyToken) + + val currTime = System.currentTimeMillis() + timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval - assert(credentialManager - .getServiceCredentialProvider("hadoopfs") - .get - .isInstanceOf[YARNHadoopAccessManager]) + Some(timeOfNextTokenRenewal) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala similarity index 56% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala index 0e3a2afab909c..6f882e5a3adce 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopFSCredentialProviderSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala @@ -18,53 +18,32 @@ package org.apache.spark.deploy.yarn.security import org.apache.hadoop.conf.Configuration -import org.scalatest.{Matchers, PrivateMethodTester} +import org.scalatest.Matchers -import org.apache.spark.{SparkException, SparkFunSuite} -import org.apache.spark.deploy.yarn.security.YARNHadoopAccessManager +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} -class YARNHadoopFSCredentialProviderSuite - extends SparkFunSuite - with PrivateMethodTester - with Matchers { - private val _getTokenRenewer = PrivateMethod[String]('getTokenRenewer) - - private def getTokenRenewer( - fsCredentialProvider: YARNHadoopAccessManager, conf: Configuration): String = { - fsCredentialProvider invokePrivate _getTokenRenewer(conf) - } - - private var hadoopFsCredentialProvider: YARNHadoopAccessManager = null - - override def beforeAll() { - super.beforeAll() - - if (hadoopFsCredentialProvider == null) { - hadoopFsCredentialProvider = new YARNHadoopAccessManager() - } - } - - override def afterAll() { - if (hadoopFsCredentialProvider != null) { - hadoopFsCredentialProvider = null - } - - super.afterAll() - } +class YARNHadoopAccessManagerSuite extends SparkFunSuite with Matchers { test("check token renewer") { val hadoopConf = new Configuration() hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - val renewer = getTokenRenewer(hadoopFsCredentialProvider, hadoopConf) + + val sparkConf = new SparkConf() + val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf) + + val renewer = yarnHadoopAccessManager.getTokenRenewer renewer should be ("yarn/myrm:8032@SPARKTEST.COM") } test("check token renewer default") { val hadoopConf = new Configuration() + val sparkConf = new SparkConf() + val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf) + val caught = intercept[SparkException] { - getTokenRenewer(hadoopFsCredentialProvider, hadoopConf) + yarnHadoopAccessManager.getTokenRenewer } assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") } From e15f1abcd708d32d863523135ba9fe8690ba2d9c Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 14:12:36 -0700 Subject: [PATCH 21/40] rat-excludes --- .../security/ConfigurableCredentialManager.scala | 13 ------------- .../ConfigurableCredentialManagerSuite.scala | 8 -------- dev/.rat-excludes | 2 +- .../yarn/security/ServiceCredentialProvider.scala | 6 ++---- 4 files changed, 3 insertions(+), 26 deletions(-) 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 4487e387127e9..ca233a883303c 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 @@ -130,17 +130,4 @@ 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/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala index 4368882532c7a..532e3ae67b6df 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 @@ -79,14 +79,6 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit tokens.iterator().next().getService should be (new Text("test")) } - test("verify obtaining user credentials") { - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) - - val initNumTokens = UserGroupInformation.getCurrentUser.getCredentials.numberOfTokens - val creds = credentialManager.obtainUserCredentials - creds.numberOfTokens() should be (initNumTokens + 1) - } - test("verify getting credential renewal info") { credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) val creds = new Credentials() diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 4ae5b0b083241..45e5f63c5438d 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.security.ServiceCredentialProvider +META-INF/services/* spark-warehouse structured-streaming/* kafka-source-initial-offset-version-2.1.0.bin 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 index 7f5aa92089fcc..d0cf4469e4600 100644 --- 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 @@ -47,7 +47,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 SparkConf + * @param sparkConf Spark configuration. * @param creds Credentials to add tokens and security keys to. * @return If this Credential is renewable and can be renewed, return the time of the next * renewal, otherwise None should be returned. @@ -56,6 +56,4 @@ trait ServiceCredentialProvider { hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] -} - - +} \ No newline at end of file From a546aab923520ccec7683c3b320a5b92dedc3f1e Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 14:26:47 -0700 Subject: [PATCH 22/40] fix RAT --- dev/.rat-excludes | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 45e5f63c5438d..607234b4068d0 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -93,16 +93,13 @@ INDEX .lintr gen-java.* .*avpr -org.apache.spark.sql.sources.DataSourceRegister -org.apache.spark.scheduler.SparkHistoryListenerFactory .*parquet spark-deps-.* .*csv .*tsv -org.apache.spark.scheduler.ExternalClusterManager .*\.sql .Rbuildignore -META-INF/services/* +META-INF/* spark-warehouse structured-streaming/* kafka-source-initial-offset-version-2.1.0.bin From d6d21d165a451ce7a285baa98387cbf341fb4739 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 21 Apr 2017 14:37:24 -0700 Subject: [PATCH 23/40] style --- .../deploy/security/ConfigurableCredentialManagerSuite.scala | 5 ++--- .../deploy/yarn/security/ServiceCredentialProvider.scala | 2 +- ...ache.spark.deploy.yarn.security.ServiceCredentialProvider | 2 +- 3 files changed, 4 insertions(+), 5 deletions(-) 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 532e3ae67b6df..347f6ff356a68 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 @@ -17,12 +17,11 @@ package org.apache.spark.deploy.security -import org.scalatest.{BeforeAndAfter, Matchers} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.Text -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.Token +import org.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark.{SparkConf, SparkFunSuite} 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 index d0cf4469e4600..f6883b1d5b656 100644 --- 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 @@ -56,4 +56,4 @@ trait ServiceCredentialProvider { hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] -} \ No newline at end of file +} 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.yarn.security.ServiceCredentialProvider index dfea80b38aed7..f31c232693133 100644 --- 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.yarn.security.ServiceCredentialProvider @@ -1 +1 @@ -org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider \ No newline at end of file +org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider From 092aac7b6e04bb5968293da11eb7cc96c6d9edbb Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 24 Apr 2017 12:43:47 -0700 Subject: [PATCH 24/40] Remove unneeded import --- .../spark/deploy/security/ConfigurableCredentialManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ca233a883303c..f16cae21d0b22 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, UserGroupInformation} +import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil From 38adaaec80cd172f7b5b92e5b85e78f5879577eb Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 18 May 2017 12:58:48 -0700 Subject: [PATCH 25/40] Make ServiceCredentialProvider private --- ....deploy.security.ServiceCredentialProvider | 3 -- .../ConfigurableCredentialManager.scala | 9 ++-- .../security/ServiceCredentialProvider.scala | 3 +- ....deploy.security.ServiceCredentialProvider | 1 - .../ConfigurableCredentialManagerSuite.scala | 48 ++----------------- 5 files changed, 7 insertions(+), 57 deletions(-) delete mode 100644 core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider delete mode 100644 core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider diff --git a/core/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 deleted file mode 100644 index 9ffeb4d500296..0000000000000 --- a/core/src/main/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider +++ /dev/null @@ -1,3 +0,0 @@ -org.apache.spark.deploy.security.HadoopFSCredentialProvider -org.apache.spark.deploy.security.HBaseCredentialProvider -org.apache.spark.deploy.security.HiveCredentialProvider 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 f16cae21d0b22..604de62fa68fc 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 @@ -66,7 +66,9 @@ private[spark] class ConfigurableCredentialManager( } private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { - val providers = loadCredentialProviders + val providers = List(new HadoopFSCredentialProvider, + new HiveCredentialProvider, + new HBaseCredentialProvider) // Filter out credentials in which spark.security.credentials.{service}.enabled is false. providers @@ -98,11 +100,6 @@ private[spark] class ConfigurableCredentialManager( .getOrElse(isEnabledDeprecated) } - private def loadCredentialProviders: List[ServiceCredentialProvider] = { - ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) - .asScala.toList - } - /** * Get credential provider for the specified service. */ 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 46b52f618a466..f8991da5c6fb0 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 @@ -27,8 +27,7 @@ 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 { +private[spark] trait ServiceCredentialProvider { /** * Name of the service to provide credentials. This name should unique, Spark internally will diff --git a/core/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 deleted file mode 100644 index 2676a0ad589fa..0000000000000 --- a/core/src/test/resources/META-INF/services/org.apache.spark.deploy.security.ServiceCredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.deploy.security.TestCredentialProvider 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 347f6ff356a68..2d0454aa0c803 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 @@ -62,31 +62,17 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit credentialManager.getServiceCredentialProvider("hadoopfs") should be (None) credentialManager.getServiceCredentialProvider("hive") should be (None) - credentialManager.getServiceCredentialProvider("test") should not be (None) credentialManager.getServiceCredentialProvider("hbase") should not be (None) } - test("verify obtaining credentials from provider") { + test("verify no credentials are obtained") { credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) val creds = new Credentials() - // Tokens can only be obtained from TestTokenProvider, for hdfs, hbase and hive tokens cannot - // be obtained. + // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. credentialManager.obtainCredentials(hadoopConf, creds) val tokens = creds.getAllTokens - tokens.size() should be (1) - tokens.iterator().next().getService should be (new Text("test")) - } - - test("verify getting credential renewal info") { - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) - val creds = new Credentials() - - val testCredentialProvider = credentialManager.getServiceCredentialProvider("test").get - .asInstanceOf[TestCredentialProvider] - // Only TestTokenProvider can get the time of next token renewal - val nextRenewal = credentialManager.obtainCredentials(hadoopConf, creds) - nextRenewal should be (testCredentialProvider.timeOfNextTokenRenewal) + tokens.size() should be (0) } test("obtain tokens For HiveMetastore") { @@ -119,31 +105,3 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit creds.getAllTokens.size should be (0) } } - -class TestCredentialProvider extends ServiceCredentialProvider { - val tokenRenewalInterval = 86400 * 1000L - var timeOfNextTokenRenewal = 0L - - override def serviceName: String = "test" - - override def credentialsRequired(conf: Configuration): Boolean = true - - override def obtainCredentials( - hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, - creds: Credentials): Option[Long] = { - if (creds == null) { - // Guard out other unit test failures. - return None - } - - val emptyToken = new Token() - emptyToken.setService(new Text("test")) - creds.addToken(emptyToken.getService, emptyToken) - - val currTime = System.currentTimeMillis() - timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval - - Some(timeOfNextTokenRenewal) - } -} From 92ac3f0790647f2159f02766a8cb20077ee6c9e5 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 18 May 2017 13:30:09 -0700 Subject: [PATCH 26/40] Addressed style comments --- .../ConfigurableCredentialManager.scala | 8 +++---- .../security/DefaultHadoopAccessManager.scala | 3 ++- .../security/HBaseCredentialProvider.scala | 2 +- ...la => HadoopDelegationTokenProvider.scala} | 7 +++---- .../security/HadoopFSCredentialProvider.scala | 2 +- .../security/HiveCredentialProvider.scala | 2 +- docs/configuration.md | 6 +++--- .../security/ServiceCredentialProvider.scala | 2 +- .../YARNConfigurableCredentialManager.scala | 8 +++---- .../security/YARNHadoopAccessManager.scala | 8 +++---- ...RNConfigurableCredentialManagerSuite.scala | 21 ++----------------- 11 files changed, 26 insertions(+), 43 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/security/{ServiceCredentialProvider.scala => HadoopDelegationTokenProvider.scala} (88%) 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 604de62fa68fc..6e0f2564f5225 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 @@ -34,7 +34,7 @@ import org.apache.spark.util.Utils * APIs for other modules to obtain credentials as well as renewal time. By default * [[HadoopFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will * be loaded in if not explicitly disabled, any plugged-in credential provider wants to be - * managed by ConfigurableCredentialManager needs to implement [[ServiceCredentialProvider]] + * managed by ConfigurableCredentialManager needs to implement [[HadoopDelegationTokenProvider]] * interface and put into resources/META-INF/services to be loaded by ServiceLoader. * * Also each credential provider is controlled by @@ -65,7 +65,7 @@ private[spark] class ConfigurableCredentialManager( this(sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) } - private def getCredentialProviders(): Map[String, ServiceCredentialProvider] = { + private def getCredentialProviders: Map[String, HadoopDelegationTokenProvider] = { val providers = List(new HadoopFSCredentialProvider, new HiveCredentialProvider, new HBaseCredentialProvider) @@ -83,7 +83,7 @@ private[spark] class ConfigurableCredentialManager( deprecatedProviderEnabledConfigs.foreach { pattern => val deprecatedKey = pattern.format(serviceName) if (sparkConf.contains(deprecatedKey)) { - logWarning(s"${deprecatedKey} is deprecated, using ${key} instead") + logWarning(s"${deprecatedKey} is deprecated. Please use ${key} instead.") } } @@ -103,7 +103,7 @@ private[spark] class ConfigurableCredentialManager( /** * Get credential provider for the specified service. */ - def getServiceCredentialProvider(service: String): Option[ServiceCredentialProvider] = { + def getServiceCredentialProvider(service: String): Option[HadoopDelegationTokenProvider] = { credentialProviders.get(service) } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala index 7dd37c50ebfb4..1865232cc509d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala @@ -22,7 +22,8 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.security.UserGroupInformation -class DefaultHadoopAccessManager(hadoopConf: Configuration) extends HadoopAccessManager { +private[spark] class DefaultHadoopAccessManager(hadoopConf: Configuration) + extends HadoopAccessManager { def getTokenRenewer: String = { UserGroupInformation.getCurrentUser.getShortUserName diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala index d1486773b96c5..f3bdcf7f263d4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala @@ -28,7 +28,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[security] class HBaseCredentialProvider extends ServiceCredentialProvider with Logging { +private[security] class HBaseCredentialProvider extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "hbase" diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala index f8991da5c6fb0..7ab310b8e5fb8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ServiceCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -24,10 +24,9 @@ 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. + * Hadoop delegation token provider. */ -private[spark] trait ServiceCredentialProvider { +private[spark] trait HadoopDelegationTokenProvider { /** * Name of the service to provide credentials. This name should unique, Spark internally will @@ -36,7 +35,7 @@ private[spark] trait ServiceCredentialProvider { def serviceName: String /** - * To decide whether credential is required for this service. By default it based on whether + * Returns true if credentials are required for this service. By default, it is based on whether * Hadoop security is enabled. */ def credentialsRequired(hadoopConf: Configuration): Boolean = { 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 fe027c9eb5f46..73dcab27a10c1 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 @@ -26,7 +26,7 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.internal.Logging private[deploy] class HadoopFSCredentialProvider - extends ServiceCredentialProvider with Logging { + extends HadoopDelegationTokenProvider 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. diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 0048fb329e520..22fda3ce1180f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -33,7 +33,7 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[security] class HiveCredentialProvider extends ServiceCredentialProvider with Logging { +private[security] class HiveCredentialProvider extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "hive" diff --git a/docs/configuration.md b/docs/configuration.md index 0f8a4254b7961..2687f542b8bd3 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/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 index f6883b1d5b656..84172e03afdc6 100644 --- 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 @@ -36,7 +36,7 @@ trait ServiceCredentialProvider { def serviceName: String /** - * To decide whether credential is required for this service. By default it based on whether + * Returns true if credentials are required by this service. By default, it is based on whether * Hadoop security is enabled. */ def credentialsRequired(hadoopConf: Configuration): Boolean = { 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 index 42e225baeb896..e3952baadeda5 100644 --- 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 @@ -30,7 +30,7 @@ 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]]. + * deprecated [[ServiceCredentialProvider]]. */ private[yarn] class YARNConfigurableCredentialManager( sparkConf: SparkConf, @@ -43,7 +43,7 @@ private[yarn] class YARNConfigurableCredentialManager( val deprecatedCredentialProviders = getDeprecatedCredentialProviders def getDeprecatedCredentialProviders: - Map[String, org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = { + Map[String, ServiceCredentialProvider] = { val deprecatedProviders = loadDeprecatedCredentialProviders deprecatedProviders. @@ -53,9 +53,9 @@ private[yarn] class YARNConfigurableCredentialManager( } def loadDeprecatedCredentialProviders: - List[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider] = { + List[ServiceCredentialProvider] = { ServiceLoader.load( - classOf[org.apache.spark.deploy.yarn.security.ServiceCredentialProvider], + classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) .asScala .toList diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala index 3a2087764b56d..97abe578fde97 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala @@ -38,15 +38,15 @@ private[yarn] class YARNHadoopAccessManager( sparkConf: SparkConf) extends HadoopAccessManager with Logging { def getTokenRenewer: String = { - val delegTokenRenewer = Master.getMasterPrincipal(hadoopConf) - logDebug("delegation token renewer is: " + delegTokenRenewer) - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val tokenRenewer = Master.getMasterPrincipal(hadoopConf) + logDebug("delegation token renewer is: " + tokenRenewer) + if (tokenRenewer == null || tokenRenewer.length() == 0) { val errorMessage = "Can't get Master Kerberos principal for use as renewer" logError(errorMessage) throw new SparkException(errorMessage) } - delegTokenRenewer + tokenRenewer } def hadoopFSsToAccess: Set[Path] = { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index df13c9eb19ff0..726658cb0644c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark.{SparkConf, SparkFunSuite} class YARNConfigurableCredentialManagerSuite - extends SparkFunSuite with Matchers with BeforeAndAfter { + extends SparkFunSuite with Matchers { private var credentialManager: YARNConfigurableCredentialManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -46,9 +46,6 @@ class YARNConfigurableCredentialManagerSuite } class YARNTestCredentialProvider extends ServiceCredentialProvider { - val tokenRenewalInterval = 86400 * 1000L - var timeOfNextTokenRenewal = 0L - override def serviceName: String = "yarn-test" override def credentialsRequired(conf: Configuration): Boolean = true @@ -56,19 +53,5 @@ class YARNTestCredentialProvider extends ServiceCredentialProvider { override def obtainCredentials( hadoopConf: Configuration, sparkConf: SparkConf, - creds: Credentials): Option[Long] = { - if (creds == null) { - // Guard out other unit test failures. - return None - } - - val emptyToken = new Token() - emptyToken.setService(new Text(serviceName)) - creds.addToken(emptyToken.getService, emptyToken) - - val currTime = System.currentTimeMillis() - timeOfNextTokenRenewal = (currTime - currTime % tokenRenewalInterval) + tokenRenewalInterval - - Some(timeOfNextTokenRenewal) - } + creds: Credentials): Option[Long] = Some(0) } From cd58b6c3e6d498a756a8e61d2c25d96b352537c1 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 22 May 2017 10:17:35 -0700 Subject: [PATCH 27/40] review comments --- .../ConfigurableCredentialManager.scala | 30 +++---- .../security/DefaultHadoopAccessManager.scala | 39 -------- .../security/HBaseCredentialProvider.scala | 7 +- .../deploy/security/HadoopAccessManager.scala | 38 -------- .../HadoopDelegationTokenProvider.scala | 8 +- .../security/HadoopFSCredentialProvider.scala | 89 +++++++++++++++---- .../security/HiveCredentialProvider.scala | 4 +- .../ConfigurableCredentialManagerSuite.scala | 33 +++++-- .../spark/deploy/yarn/ApplicationMaster.scala | 6 +- .../org/apache/spark/deploy/yarn/Client.scala | 15 ++-- .../deploy/yarn/YarnSparkHadoopUtil.scala | 27 +++++- .../yarn/security/AMCredentialRenewer.scala | 4 +- .../security/ServiceCredentialProvider.scala | 1 - .../YARNConfigurableCredentialManager.scala | 63 +++++++------ .../security/YARNHadoopAccessManager.scala | 84 ----------------- ...RNConfigurableCredentialManagerSuite.scala | 11 ++- 16 files changed, 201 insertions(+), 258 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala delete mode 100644 core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala delete mode 100644 resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.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 6e0f2564f5225..54444191d1f2c 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 @@ -20,10 +20,9 @@ package org.apache.spark.deploy.security import java.util.ServiceLoader import scala.collection.JavaConverters._ - import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials - import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging @@ -41,11 +40,15 @@ import org.apache.spark.util.Utils * 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.security.credentials.hive.enabled. + * + * @param sparkConf Spark configuration + * @param hadoopConf Hadoop configuration + * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. */ private[spark] class ConfigurableCredentialManager( sparkConf: SparkConf, hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager) + fileSystems: Set[FileSystem]) extends Logging { private val deprecatedProviderEnabledConfigs = List( @@ -57,16 +60,8 @@ private[spark] class ConfigurableCredentialManager( private val credentialProviders = getCredentialProviders logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") - def this(sparkConf: SparkConf, hadoopConf: Configuration) { - this(sparkConf, hadoopConf, new DefaultHadoopAccessManager(hadoopConf)) - } - - def this(sparkConf: SparkConf) { - this(sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) - } - private def getCredentialProviders: Map[String, HadoopDelegationTokenProvider] = { - val providers = List(new HadoopFSCredentialProvider, + val providers = List(new HadoopFSCredentialProvider(fileSystems), new HiveCredentialProvider, new HBaseCredentialProvider) @@ -77,7 +72,7 @@ private[spark] class ConfigurableCredentialManager( .toMap } - protected def isServiceEnabled(serviceName: String): Boolean = { + def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) deprecatedProviderEnabledConfigs.foreach { pattern => @@ -111,15 +106,14 @@ private[spark] 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. + * @return Time after which the fetched delegation tokens should be renewed. */ def obtainCredentials( - hadoopConf: Configuration, - creds: Credentials): Long = { + hadoopConf: Configuration, + creds: Credentials): Long = { credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { - provider.obtainCredentials(hadoopConf, hadoopAccessManager, creds) + provider.obtainCredentials(hadoopConf, creds) } else { logDebug(s"Service ${provider.serviceName} does not require a token." + s" Check your configuration to see if security is disabled or not.") diff --git a/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala deleted file mode 100644 index 1865232cc509d..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/security/DefaultHadoopAccessManager.scala +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path -import org.apache.hadoop.security.UserGroupInformation - -private[spark] class DefaultHadoopAccessManager(hadoopConf: Configuration) - extends HadoopAccessManager { - - def getTokenRenewer: String = { - UserGroupInformation.getCurrentUser.getShortUserName - } - - def hadoopFSsToAccess: Set[Path] = { - Set(FileSystem.get(hadoopConf).getHomeDirectory) - } - - def getTokenRenewalInterval: Option[Long] = { - None - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala index f3bdcf7f263d4..272b3d8664a2e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala @@ -19,22 +19,21 @@ package org.apache.spark.deploy.security import scala.reflect.runtime.universe import scala.util.control.NonFatal - import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.{Token, TokenIdentifier} - import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[security] class HBaseCredentialProvider extends HadoopDelegationTokenProvider with Logging { +private[security] class HBaseCredentialProvider + extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "hbase" override def obtainCredentials( hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { try { val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala deleted file mode 100644 index e617e94440f8a..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopAccessManager.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path - -/** - * Methods in [[HadoopAccessManager]] return scheduler-specific information related to how Hadoop - * delegation tokens should be fetched. - */ -private[spark] trait HadoopAccessManager { - - /** The user allowed to renew delegation tokens */ - def getTokenRenewer: String - - /** The renewal interval, or [[None]] if the token shouldn't be renewed */ - def getTokenRenewalInterval: Option[Long] - - /** The set of hadoop file systems to fetch delegation tokens for */ - def hadoopFSsToAccess: Set[Path] -} - diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala index 7ab310b8e5fb8..83930f0b14404 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -18,8 +18,8 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.{Credentials, UserGroupInformation} - import org.apache.spark.SparkConf import org.apache.spark.annotation.InterfaceStability @@ -38,20 +38,16 @@ private[spark] trait HadoopDelegationTokenProvider { * Returns true if credentials are required for this service. By default, it is based on whether * Hadoop security is enabled. */ - def credentialsRequired(hadoopConf: Configuration): Boolean = { - UserGroupInformation.isSecurityEnabled - } + def credentialsRequired(hadoopConf: Configuration): Boolean /** * Obtain credentials for this service and get the time of the next renewal. * @param hadoopConf Configuration of current Hadoop Compatible system. - * @param hadoopAccessManager HadoopAccessManager * @param creds Credentials to add tokens and security keys to. * @return If this Credential is renewable and can be renewed, return the time of the next * renewal, otherwise None should be returned. */ def obtainCredentials( hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] } 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 73dcab27a10c1..b283c65f8b776 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 @@ -18,14 +18,17 @@ package org.apache.spark.deploy.security import scala.collection.JavaConverters._ - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapred.Master +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier - +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging -private[deploy] class HadoopFSCredentialProvider +import scala.util.Try + +private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) extends HadoopDelegationTokenProvider 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, @@ -36,35 +39,85 @@ private[deploy] class HadoopFSCredentialProvider override def obtainCredentials( hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { - // NameNode to access, used to get tokens from different FileSystems - val tmpCreds = new Credentials() - val tokenRenewer = hadoopAccessManager.getTokenRenewer - hadoopAccessManager.hadoopFSsToAccess.foreach { dst => - val dstFs = dst.getFileSystem(hadoopConf) - logInfo("getting token for: " + dst) - dstFs.addDelegationTokens(tokenRenewer, tmpCreds) - } + val newCreds = fetchDelegationTokens( + getTokenRenewer(hadoopConf), + fileSystems) // Get the token renewal interval if it is not set. It will only be called once. if (tokenRenewalInterval == null) { - tokenRenewalInterval = hadoopAccessManager.getTokenRenewalInterval + tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, fileSystems) } // Get the time of next renewal. val nextRenewalDate = tokenRenewalInterval.flatMap { interval => - val nextRenewalDates = tmpCreds.getAllTokens.asScala + val nextRenewalDates = newCreds.getAllTokens.asScala .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) - .map { t => - val identifier = t.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + .map { token => + val identifier = token + .decodeIdentifier() + .asInstanceOf[AbstractDelegationTokenIdentifier] identifier.getIssueDate + interval } if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) } - creds.addAll(tmpCreds) + creds.addAll(newCreds) nextRenewalDate } + + def credentialsRequired(hadoopConf: Configuration): Boolean = { + UserGroupInformation.isSecurityEnabled + } + + private def getTokenRenewer(hadoopConf: Configuration): String = { + val tokenRenewer = Master.getMasterPrincipal(hadoopConf) + logDebug("Delegation token renewer is: " + tokenRenewer) + + if (tokenRenewer == null || tokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer." + logError(errorMessage) + throw new SparkException(errorMessage) + } + + tokenRenewer + } + + private def fetchDelegationTokens( + renewer: String, + filesystems: Set[FileSystem]): Credentials = { + val creds = new Credentials() + + filesystems.foreach { fs => + logInfo("getting token for: " + fs) + fs.addDelegationTokens(renewer, creds) + } + + creds + } + + private def getTokenRenewalInterval( + hadoopConf: Configuration, + filesystems: Set[FileSystem]): 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. + val creds = fetchDelegationTokens( + UserGroupInformation.getCurrentUser.getUserName, + filesystems) + + val renewIntervals = creds.getAllTokens.asScala.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] + }.flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") + interval + }.toOption + } + if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 22fda3ce1180f..8d97bfc0021f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -22,13 +22,12 @@ import java.security.PrivilegedExceptionAction import scala.reflect.runtime.universe import scala.util.control.NonFatal - import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token - import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.Utils @@ -62,7 +61,6 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv override def obtainCredentials( hadoopConf: Configuration, - hadoopAccessManager: HadoopAccessManager, creds: Credentials): Option[Long] = { val conf = hiveConf(hadoopConf) 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 2d0454aa0c803..bf71392fee9f1 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 @@ -18,11 +18,12 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.Text import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.Token +import org.apache.spark.deploy.SparkHadoopUtil import org.scalatest.{BeforeAndAfter, Matchers} - import org.apache.spark.{SparkConf, SparkFunSuite} class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -38,7 +39,10 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit } test("Correctly load default credential providers") { - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + credentialManager = new ConfigurableCredentialManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess(hadoopConf)) credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) credentialManager.getServiceCredentialProvider("hbase") should not be (None) @@ -48,7 +52,10 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit test("disable hive credential provider") { sparkConf.set("spark.security.credentials.hive.enabled", "false") - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + credentialManager = new ConfigurableCredentialManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess(hadoopConf)) credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) credentialManager.getServiceCredentialProvider("hbase") should not be (None) @@ -58,7 +65,10 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit test("using deprecated configurations") { sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + credentialManager = new ConfigurableCredentialManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess(hadoopConf)) credentialManager.getServiceCredentialProvider("hadoopfs") should be (None) credentialManager.getServiceCredentialProvider("hive") should be (None) @@ -66,11 +76,16 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit } test("verify no credentials are obtained") { - credentialManager = new ConfigurableCredentialManager(sparkConf, hadoopConf) + credentialManager = new ConfigurableCredentialManager( + sparkConf, + hadoopConf, + hadoopFSsToAccess(hadoopConf)) val creds = new Credentials() // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. - credentialManager.obtainCredentials(hadoopConf, creds) + credentialManager.obtainCredentials( + hadoopConf, + creds) val tokens = creds.getAllTokens tokens.size() should be (0) } @@ -85,7 +100,6 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit val credentials = new Credentials() hiveCredentialProvider.obtainCredentials( hadoopConf, - new DefaultHadoopAccessManager(hadoopConf), credentials) credentials.getAllTokens.size() should be (0) @@ -99,9 +113,12 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers wit val creds = new Credentials() hbaseTokenProvider.obtainCredentials( hadoopConf, - new DefaultHadoopAccessManager(hadoopConf), creds) creds.getAllTokens.size should be (0) } + + private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = { + Set(FileSystem.get(hadoopConf)) + } } 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 a2248d91d8b8a..37ae0499005af 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 @@ -247,7 +247,11 @@ 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 YARNConfigurableCredentialManager(sparkConf, yarnConf) + val credentialManager = new YARNConfigurableCredentialManager( + sparkConf, + yarnConf, + YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(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 537a43c355cd8..e7bb858e166ef 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,13 +27,13 @@ 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 import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.util.StringUtils @@ -45,9 +45,9 @@ 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.YARNConfigurableCredentialManager import org.apache.spark.internal.Logging @@ -121,7 +121,10 @@ private[spark] class Client( private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) } .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) - private val credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) + private val credentialManager = new YARNConfigurableCredentialManager( + sparkConf, + hadoopConf, + YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) @@ -368,7 +371,9 @@ private[spark] class Client( val fs = destDir.getFileSystem(hadoopConf) // Merge credentials obtained from registered providers - val nearestTimeOfNextRenewal = credentialManager.obtainCredentials(hadoopConf, credentials) + val nearestTimeOfNextRenewal = credentialManager.obtainYARNCredentials( + hadoopConf, + credentials) if (credentials != null) { // Add credentials to current user's UGI, so that following operations don't need to use the 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 da4f5edad0018..7617dcc674dc4 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,17 +22,16 @@ 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.fs.FileSystem import org.apache.hadoop.io.Text -import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapred.{JobConf, Master} import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation 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.CredentialUpdater @@ -40,6 +39,8 @@ 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 +import org.apache.spark.deploy.yarn.config._ +import org.apache.hadoop.fs.Path /** * Contains util methods to interact with Hadoop from spark. @@ -89,7 +90,10 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = { val hadoopConf = newConfiguration(sparkConf) - val credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) + val credentialManager = new YARNConfigurableCredentialManager( + sparkConf, + hadoopConf, + YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) credentialUpdater.start() } @@ -105,6 +109,21 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) ConverterUtils.toContainerId(containerIdString) } + + /** The filesystems for which YARN should fetch delegation tokens. */ + private[spark] def yarnHadoopFSsToAccess( + sparkConf: SparkConf, + hadoopConf: Configuration): Set[FileSystem] = { + val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS) + .map(new Path(_).getFileSystem(hadoopConf)) + .toSet + + val stagingFS = sparkConf.get(STAGING_DIR) + .map(new Path(_).getFileSystem(hadoopConf)) + .getOrElse(FileSystem.get(hadoopConf)) + + filesystemsToAccess + stagingFS + } } object YarnSparkHadoopUtil { 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 e81d072c5ff7a..2da75b892c6f4 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 @@ -174,7 +174,9 @@ private[yarn] class AMCredentialRenewer( keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { // Get a copy of the credentials override def run(): Void = { - nearestNextRenewalTime = credentialManager.obtainCredentials(freshHadoopConf, tempCreds) + nearestNextRenewalTime = credentialManager.obtainYARNCredentials( + freshHadoopConf, + tempCreds) null } }) 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 index 84172e03afdc6..cc24ac4d9bcf6 100644 --- 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 @@ -26,7 +26,6 @@ import org.apache.spark.SparkConf * A credential provider for a service. User must implement this if they need to access a * secure service from Spark. */ -@deprecated("Use org.apache.spark.deploy.security.ServiceCredentialProvider", "2.3.0") trait 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 index e3952baadeda5..3dff7d780dcc3 100644 --- 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 @@ -19,14 +19,20 @@ package org.apache.spark.deploy.yarn.security import java.util.ServiceLoader -import scala.collection.JavaConverters._ +import com.google.common.annotations.VisibleForTesting +import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials - -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.security.ConfigurableCredentialManager +import org.apache.spark.internal.Logging import org.apache.spark.util.Utils +import org.apache.spark.internal.config._ +import org.apache.spark.deploy.yarn.config._ +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.Master /** * This class exists for backwards compatibility. It loads services registered under the @@ -34,25 +40,46 @@ import org.apache.spark.util.Utils */ private[yarn] class YARNConfigurableCredentialManager( sparkConf: SparkConf, - hadoopConf: Configuration) - extends ConfigurableCredentialManager( - sparkConf, - hadoopConf, - new YARNHadoopAccessManager(hadoopConf, sparkConf)) { + hadoopConf: Configuration, + fileSystems: Set[FileSystem]) extends Logging { + private val configurableCredentialManager = + new ConfigurableCredentialManager(sparkConf, hadoopConf, fileSystems) + + // public for testing val deprecatedCredentialProviders = getDeprecatedCredentialProviders - def getDeprecatedCredentialProviders: + def obtainYARNCredentials( + hadoopConf: Configuration, + creds: Credentials): Long = { + + val superInterval = configurableCredentialManager.obtainCredentials( + hadoopConf, + creds) + + deprecatedCredentialProviders.values.flatMap { provider => + if (provider.credentialsRequired(hadoopConf)) { + provider.obtainCredentials(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") + None + } + }.foldLeft(superInterval)(math.min) + } + + + private def getDeprecatedCredentialProviders: Map[String, ServiceCredentialProvider] = { val deprecatedProviders = loadDeprecatedCredentialProviders deprecatedProviders. - filter(p => isServiceEnabled(p.serviceName)) + filter(p => configurableCredentialManager.isServiceEnabled(p.serviceName)) .map(p => (p.serviceName, p)) .toMap } - def loadDeprecatedCredentialProviders: + private def loadDeprecatedCredentialProviders: List[ServiceCredentialProvider] = { ServiceLoader.load( classOf[ServiceCredentialProvider], @@ -60,18 +87,4 @@ private[yarn] class YARNConfigurableCredentialManager( .asScala .toList } - - override def obtainCredentials(hadoopConf: Configuration, creds: Credentials): Long = { - val superInterval = super.obtainCredentials(hadoopConf, creds) - - deprecatedCredentialProviders.values.flatMap { provider => - if (provider.credentialsRequired(hadoopConf)) { - provider.obtainCredentials(hadoopConf, sparkConf, creds) - } else { - logDebug(s"Service ${provider.serviceName} does not require a token." + - s" Check your configuration to see if security is disabled or not.") - None - } - }.foldLeft(superInterval)(math.min) - } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala deleted file mode 100644 index 97abe578fde97..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManager.scala +++ /dev/null @@ -1,84 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn.security - -import scala.collection.JavaConverters._ -import scala.util.Try - -import org.apache.hadoop.conf.Configuration -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.{HadoopAccessManager} -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ - -private[yarn] class YARNHadoopAccessManager( - hadoopConf: Configuration, - sparkConf: SparkConf) extends HadoopAccessManager with Logging { - - def getTokenRenewer: String = { - val tokenRenewer = Master.getMasterPrincipal(hadoopConf) - logDebug("delegation token renewer is: " + tokenRenewer) - if (tokenRenewer == null || tokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - - tokenRenewer - } - - def hadoopFSsToAccess: Set[Path] = { - sparkConf.get(FILESYSTEMS_TO_ACCESS).map(new Path(_)).toSet + - sparkConf.get(STAGING_DIR).map(new Path(_)) - .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory) - } - - def getTokenRenewalInterval: 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. - sparkConf.get(PRINCIPAL).flatMap { renewer => - val creds = new Credentials() - hadoopFSsToAccess.foreach { dst => - val dstFs = dst.getFileSystem(hadoopConf) - dstFs.addDelegationTokens(renewer, creds) - } - - val renewIntervals = creds.getAllTokens.asScala.filter { - _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] - }.flatMap { token => - Try { - val newExpiration = token.renew(hadoopConf) - val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] - val interval = newExpiration - identifier.getIssueDate - logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") - interval - }.toOption - } - if (renewIntervals.isEmpty) None else Some(renewIntervals.min) - } - } - -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index 726658cb0644c..1e01d6a3cc33f 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.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.yarn.YarnSparkHadoopUtil import org.scalatest.{BeforeAndAfter, Matchers} - import org.apache.spark.{SparkConf, SparkFunSuite} class YARNConfigurableCredentialManagerSuite @@ -34,12 +34,17 @@ class YARNConfigurableCredentialManagerSuite override def beforeAll(): Unit = { super.beforeAll() + System.setProperty("SPARK_YARN_MODE", "true") + sparkConf = new SparkConf() hadoopConf = new Configuration() } test("Correctly loads deprecated credential providers") { - credentialManager = new YARNConfigurableCredentialManager(sparkConf, hadoopConf) + credentialManager = new YARNConfigurableCredentialManager( + sparkConf, + hadoopConf, + YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) credentialManager.deprecatedCredentialProviders.get("yarn-test") should not be (None) } @@ -53,5 +58,5 @@ class YARNTestCredentialProvider extends ServiceCredentialProvider { override def obtainCredentials( hadoopConf: Configuration, sparkConf: SparkConf, - creds: Credentials): Option[Long] = Some(0) + creds: Credentials): Option[Long] = None } From bf758e64f699f3211e15767d0f1854cd47cecb29 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 22 May 2017 17:29:48 -0700 Subject: [PATCH 28/40] style --- .../security/ConfigurableCredentialManager.scala | 6 +----- .../deploy/security/HBaseCredentialProvider.scala | 4 ++-- .../security/HadoopDelegationTokenProvider.scala | 5 +---- .../deploy/security/HadoopFSCredentialProvider.scala | 8 +++++--- .../deploy/security/HiveCredentialProvider.scala | 4 ++-- .../security/ConfigurableCredentialManagerSuite.scala | 8 +++----- .../scala/org/apache/spark/deploy/yarn/Client.scala | 6 +++--- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 8 +++++--- .../security/YARNConfigurableCredentialManager.scala | 11 ++++------- .../YARNConfigurableCredentialManagerSuite.scala | 7 +++---- 10 files changed, 29 insertions(+), 38 deletions(-) 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 54444191d1f2c..0660b41f37b0a 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 @@ -17,16 +17,12 @@ package org.apache.spark.deploy.security -import java.util.ServiceLoader - -import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials + import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils /** * A ConfigurableCredentialManager to manage all the registered credential providers and offer diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala index 272b3d8664a2e..5e808474792f7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala @@ -19,11 +19,11 @@ package org.apache.spark.deploy.security import scala.reflect.runtime.universe import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.token.{Token, TokenIdentifier} -import org.apache.spark.SparkConf + import org.apache.spark.internal.Logging import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala index 83930f0b14404..c6f42030f1fd7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -18,10 +18,7 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.SparkConf -import org.apache.spark.annotation.InterfaceStability +import org.apache.hadoop.security.Credentials /** * Hadoop delegation token provider. 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 b283c65f8b776..d0060966baf39 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 @@ -18,15 +18,17 @@ package org.apache.spark.deploy.security 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.mapred.Master import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier -import org.apache.spark.{SparkConf, SparkException} + +import org.apache.spark.SparkException import org.apache.spark.internal.Logging -import scala.util.Try private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) extends HadoopDelegationTokenProvider with Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 8d97bfc0021f0..34b8efc65db7a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -22,13 +22,13 @@ import java.security.PrivilegedExceptionAction import scala.reflect.runtime.universe import scala.util.control.NonFatal + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token -import org.apache.spark.SparkConf + import org.apache.spark.internal.Logging import org.apache.spark.util.Utils 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 bf71392fee9f1..10446b18f8d47 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 @@ -19,14 +19,12 @@ package org.apache.spark.deploy.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.io.Text import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.token.Token -import org.apache.spark.deploy.SparkHadoopUtil -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.Matchers + import org.apache.spark.{SparkConf, SparkFunSuite} -class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { +class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { private var credentialManager: ConfigurableCredentialManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null 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 e7bb858e166ef..3c108ead21176 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} @@ -27,13 +27,13 @@ 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 import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.util.StringUtils @@ -45,9 +45,9 @@ 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.YARNConfigurableCredentialManager import org.apache.spark.internal.Logging 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 7617dcc674dc4..0aa9098ef54fa 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,8 +22,9 @@ 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.fs.FileSystem +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.{JobConf, Master} import org.apache.hadoop.security.Credentials @@ -32,15 +33,16 @@ 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.config._ 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 -import org.apache.spark.deploy.yarn.config._ -import org.apache.hadoop.fs.Path + /** * Contains util methods to interact with Hadoop from spark. 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 index 3dff7d780dcc3..61125a0272816 100644 --- 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 @@ -19,20 +19,17 @@ package org.apache.spark.deploy.yarn.security import java.util.ServiceLoader -import com.google.common.annotations.VisibleForTesting - import scala.collection.JavaConverters._ + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials -import org.apache.spark.{SparkConf, SparkException} + +import org.apache.spark.SparkConf import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -import org.apache.spark.internal.config._ -import org.apache.spark.deploy.yarn.config._ -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapred.Master + /** * This class exists for backwards compatibility. It loads services registered under the diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index 1e01d6a3cc33f..28a750ab435a0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala @@ -18,12 +18,11 @@ package org.apache.spark.deploy.yarn.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.yarn.YarnSparkHadoopUtil -import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.Matchers + import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil class YARNConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { From e820b093d4d9c8ba18b3e5d71d04de1791426264 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 22 May 2017 23:59:18 -0700 Subject: [PATCH 29/40] Remove YARNHadoopAccessManagerSuite.scala --- .../YARNHadoopAccessManagerSuite.scala | 50 ------------------- 1 file changed, 50 deletions(-) delete mode 100644 resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala deleted file mode 100644 index 6f882e5a3adce..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopAccessManagerSuite.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn.security - -import org.apache.hadoop.conf.Configuration -import org.scalatest.Matchers - -import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} - -class YARNHadoopAccessManagerSuite extends SparkFunSuite with Matchers { - - test("check token renewer") { - val hadoopConf = new Configuration() - hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") - hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") - - val sparkConf = new SparkConf() - val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf) - - val renewer = yarnHadoopAccessManager.getTokenRenewer - renewer should be ("yarn/myrm:8032@SPARKTEST.COM") - } - - test("check token renewer default") { - val hadoopConf = new Configuration() - val sparkConf = new SparkConf() - val yarnHadoopAccessManager = new YARNHadoopAccessManager(hadoopConf, sparkConf) - - val caught = - intercept[SparkException] { - yarnHadoopAccessManager.getTokenRenewer - } - assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") - } -} From 7f4ca86b9051815eda7f9debff9d67886522aafe Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 31 May 2017 16:42:22 -0700 Subject: [PATCH 30/40] Move thrifts deps back to yarn/pom.xml --- core/pom.xml | 22 +++++----------------- resource-managers/yarn/pom.xml | 10 ++++++++++ 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index d96841008ae34..351206bbf9108 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -365,26 +365,14 @@ ${hive.group} hive-exec - - - ${hive.group} - hive-metastore - - - org.apache.thrift - libthrift - test - - - org.apache.thrift - libfb303 test + + + + + - - org.apache.hadoop - hadoop-yarn-api - target/scala-${scala.binary.version}/classes diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 6d2af5e02b43d..98c0ccc65736b 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -167,6 +167,16 @@ ${jersey-1.version} + + org.apache.thrift + libthrift + test + + + org.apache.thrift + libfb303 + test + From cda35387b9960f71a052fcca6f12a91efa567958 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 1 Jun 2017 17:34:02 -0700 Subject: [PATCH 31/40] dependency testing --- core/pom.xml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/pom.xml b/core/pom.xml index 351206bbf9108..49467a44a963a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,6 +28,7 @@ spark-core_2.11 core + test jar Spark Project Core @@ -365,8 +366,15 @@ ${hive.group} hive-exec - test + + + + + + + + From 376dba09a490afc68285d26025f45af8b9da5510 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 2 Jun 2017 12:19:26 -0700 Subject: [PATCH 32/40] Fix dependency issues, and address style comments --- core/pom.xml | 38 ++++---- .../ConfigurableCredentialManager.scala | 16 ++-- .../HadoopDelegationTokenProvider.scala | 2 +- .../security/HadoopFSCredentialProvider.scala | 1 - .../security/HiveCredentialProvider.scala | 91 +++++++++---------- resource-managers/yarn/pom.xml | 11 +-- .../YARNConfigurableCredentialManager.scala | 21 ++--- ...RNConfigurableCredentialManagerSuite.scala | 4 +- 8 files changed, 85 insertions(+), 99 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 49467a44a963a..37612727ebb55 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,6 @@ spark-core_2.11 core - test jar Spark Project Core @@ -360,26 +359,31 @@ + Testing Hive reflection needs hive on the test classpath only, however, while adding hive-exec to the test + scope works fine in Maven, it causes the sbt build to fail. When the scope is set to "test", SBT seems to fail to + associate the hive-exec dependency entry here with the dependencyManagement entry in spark-parent, resulting in + dependency resolution issues. To fix this, hive-exec is instead placed in the "provided" scope. + --> ${hive.group} hive-exec + provided + + + ${hive.group} + hive-metastore + test + + + org.apache.thrift + libthrift + test + + + org.apache.thrift + libfb303 + test - - - - - - - - - - - - - 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 0660b41f37b0a..d2b0e539c34e7 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 @@ -28,14 +28,12 @@ import org.apache.spark.internal.Logging * A ConfigurableCredentialManager to manage all the registered credential providers and offer * APIs for other modules to obtain credentials as well as renewal time. By default * [[HadoopFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will - * be loaded in if not explicitly disabled, any plugged-in credential provider wants to be - * managed by ConfigurableCredentialManager needs to implement [[HadoopDelegationTokenProvider]] - * interface and put into resources/META-INF/services to be loaded by ServiceLoader. + * be loaded in if not explicitly disabled. * - * Also each credential provider is controlled by - * 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.security.credentials.hive.enabled. + * Also each credential provider is controlled by 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.security.credentials.hive.enabled. * * @param sparkConf Spark configuration * @param hadoopConf Hadoop configuration @@ -63,8 +61,8 @@ private[spark] class ConfigurableCredentialManager( // Filter out credentials in which spark.security.credentials.{service}.enabled is false. providers - .filter(p => isServiceEnabled(p.serviceName)) - .map(p => (p.serviceName, p)) + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } .toMap } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala index c6f42030f1fd7..fb89e289aa493 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.security.Credentials private[spark] trait HadoopDelegationTokenProvider { /** - * Name of the service to provide credentials. This name should unique, Spark internally will + * Name of the service to provide credentials. This name should be unique, Spark internally will * use this name to differentiate credential provider. */ def serviceName: String 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 d0060966baf39..ee6892ff9b3d1 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 @@ -29,7 +29,6 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.SparkException import org.apache.spark.internal.Logging - private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) extends HadoopDelegationTokenProvider with Logging { // Token renewal interval, this value will be set in the first call, diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 34b8efc65db7a..4647473c8902f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -20,11 +20,12 @@ package org.apache.spark.deploy.security import java.lang.reflect.UndeclaredThrowableException import java.security.PrivilegedExceptionAction -import scala.reflect.runtime.universe import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.Token @@ -36,21 +37,19 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv override def serviceName: String = "hive" + private val classNotFoundErrorStr = "You are attempting to use the HiveCredentialProvider," + + "but your Spark distribution is not built with Hive libraries." + private def hiveConf(hadoopConf: Configuration): Configuration = { try { - val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) - // the hive configuration class is a subclass of Hadoop Configuration, so can be cast down - // to a Configuration and used without reflection - val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - // using the (Configuration, Class) constructor allows the current configuration to be - // included in the hive config. - val ctor = hiveConfClass.getDeclaredConstructor(classOf[Configuration], - classOf[Object].getClass) - ctor.newInstance(hadoopConf, hiveConfClass).asInstanceOf[Configuration] + new HiveConf(hadoopConf, classOf[HiveConf]) } catch { case NonFatal(e) => logDebug("Fail to create Hive Configuration", e) hadoopConf + case e: ClassNotFoundException => + logError(classNotFoundErrorStr) + throw e } } @@ -62,48 +61,44 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv override def obtainCredentials( hadoopConf: Configuration, creds: Credentials): Option[Long] = { - val conf = hiveConf(hadoopConf) - - val principalKey = "hive.metastore.kerberos.principal" - val principal = conf.getTrimmed(principalKey, "") - require(principal.nonEmpty, s"Hive principal $principalKey undefined") - val metastoreUri = conf.getTrimmed("hive.metastore.uris", "") - require(metastoreUri.nonEmpty, "Hive metastore uri undefined") - - val currentUser = UserGroupInformation.getCurrentUser() - logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " + - s"$principal at $metastoreUri") - - val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) - val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive") - val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") - val closeCurrent = hiveClass.getMethod("closeCurrent") - try { - // get all the instance methods before invoking any - val getDelegationToken = hiveClass.getMethod("getDelegationToken", - classOf[String], classOf[String]) - val getHive = hiveClass.getMethod("get", hiveConfClass) - - doAsRealUser { - val hive = getHive.invoke(null, conf) - val tokenStr = getDelegationToken.invoke(hive, currentUser.getUserName(), principal) - .asInstanceOf[String] - val hive2Token = new Token[DelegationTokenIdentifier]() - hive2Token.decodeFromUrlString(tokenStr) - logInfo(s"Get Token from hive metastore: ${hive2Token.toString}") - creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) + val conf = hiveConf(hadoopConf) + + val principalKey = "hive.metastore.kerberos.principal" + val principal = conf.getTrimmed(principalKey, "") + require(principal.nonEmpty, s"Hive principal $principalKey undefined") + val metastoreUri = conf.getTrimmed("hive.metastore.uris", "") + require(metastoreUri.nonEmpty, "Hive metastore uri undefined") + + val currentUser = UserGroupInformation.getCurrentUser() + logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " + + s"$principal at $metastoreUri") + + try { + doAsRealUser { + val hive = Hive.get(conf, classOf[HiveConf]) + val tokenStr = hive.getDelegationToken(currentUser.getUserName(), principal) + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + logInfo(s"Get Token from hive metastore: ${hive2Token.toString}") + creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) + } + } catch { + case NonFatal(e) => + logDebug(s"Fail to get token from service $serviceName", e) + } finally { + Utils.tryLogNonFatalError { + Hive.closeCurrent() + } } + + None } catch { - case NonFatal(e) => - logDebug(s"Fail to get token from service $serviceName", e) - } finally { - Utils.tryLogNonFatalError { - closeCurrent.invoke(null) - } + case e: ClassNotFoundException => + logError(classNotFoundErrorStr) + throw e } - - None } /** diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 98c0ccc65736b..4688dac16a8dd 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -167,16 +167,7 @@ ${jersey-1.version} - - org.apache.thrift - libthrift - test - - - org.apache.thrift - libfb303 - test - + 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 index 61125a0272816..fbdd9fbcfe4ab 100644 --- 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 @@ -32,8 +32,8 @@ import org.apache.spark.util.Utils /** - * This class exists for backwards compatibility. It loads services registered under the - * deprecated [[ServiceCredentialProvider]]. + * This class loads credential providers registered under the YARN-specific [[ServiceCredentialProvider]] interface, + * as well as the builtin credential providers defined in [[ConfigurableCredentialManager]]. */ private[yarn] class YARNConfigurableCredentialManager( sparkConf: SparkConf, @@ -44,7 +44,7 @@ private[yarn] class YARNConfigurableCredentialManager( new ConfigurableCredentialManager(sparkConf, hadoopConf, fileSystems) // public for testing - val deprecatedCredentialProviders = getDeprecatedCredentialProviders + val credentialProviders = getCredentialProviders def obtainYARNCredentials( hadoopConf: Configuration, @@ -54,7 +54,7 @@ private[yarn] class YARNConfigurableCredentialManager( hadoopConf, creds) - deprecatedCredentialProviders.values.flatMap { provider => + credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { provider.obtainCredentials(hadoopConf, sparkConf, creds) } else { @@ -65,18 +65,17 @@ private[yarn] class YARNConfigurableCredentialManager( }.foldLeft(superInterval)(math.min) } - - private def getDeprecatedCredentialProviders: + private def getCredentialProviders: Map[String, ServiceCredentialProvider] = { - val deprecatedProviders = loadDeprecatedCredentialProviders + val providers = loadCredentialProviders - deprecatedProviders. - filter(p => configurableCredentialManager.isServiceEnabled(p.serviceName)) - .map(p => (p.serviceName, p)) + providers. + filter { p => configurableCredentialManager.isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } .toMap } - private def loadDeprecatedCredentialProviders: + private def loadCredentialProviders: List[ServiceCredentialProvider] = { ServiceLoader.load( classOf[ServiceCredentialProvider], diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index 28a750ab435a0..7a8c5b2dcd3a8 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala @@ -39,13 +39,13 @@ class YARNConfigurableCredentialManagerSuite hadoopConf = new Configuration() } - test("Correctly loads deprecated credential providers") { + test("Correctly loads credential providers") { credentialManager = new YARNConfigurableCredentialManager( sparkConf, hadoopConf, YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) - credentialManager.deprecatedCredentialProviders.get("yarn-test") should not be (None) + credentialManager.credentialProviders.get("yarn-test") should not be (None) } } From 0ffe8f077867ebf4eb84c0e27f98e475e2b0af9d Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 2 Jun 2017 13:12:48 -0700 Subject: [PATCH 33/40] Fix scalastyle --- .../yarn/security/YARNConfigurableCredentialManager.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 index fbdd9fbcfe4ab..1d93799242809 100644 --- 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 @@ -32,8 +32,9 @@ import org.apache.spark.util.Utils /** - * This class loads credential providers registered under the YARN-specific [[ServiceCredentialProvider]] interface, - * as well as the builtin credential providers defined in [[ConfigurableCredentialManager]]. + * This class loads credential providers registered under the YARN-specific + * [[ServiceCredentialProvider]] interface, as well as the builtin credential providers defined + * in [[ConfigurableCredentialManager]]. */ private[yarn] class YARNConfigurableCredentialManager( sparkConf: SparkConf, From 7796e14791f73f6bc97d9b7d4c891cbf48add8f0 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Fri, 2 Jun 2017 13:31:53 -0700 Subject: [PATCH 34/40] Add other deps to provided scope --- core/pom.xml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 37612727ebb55..145bd0d705ca1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -372,17 +372,17 @@ ${hive.group} hive-metastore - test + provided org.apache.thrift libthrift - test + provided org.apache.thrift libfb303 - test + provided From 1479c60b3059e17a29e23a309f1b38e364bb2451 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 5 Jun 2017 16:10:44 -0700 Subject: [PATCH 35/40] Replicate deps in yarn to fix transitivity issue --- resource-managers/yarn/pom.xml | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 4688dac16a8dd..34f2d12cd2cc0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -167,6 +167,26 @@ ${jersey-1.version} + + ${hive.group} + hive-exec + provided + + + ${hive.group} + hive-metastore + provided + + + org.apache.thrift + libthrift + provided + + + org.apache.thrift + libfb303 + provided + From 4d57f7bb76e424b48e71b18250cb018e09e2d173 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Tue, 6 Jun 2017 10:30:52 -0700 Subject: [PATCH 36/40] update comments --- resource-managers/yarn/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 34f2d12cd2cc0..3869b016ace24 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -167,6 +167,8 @@ ${jersey-1.version} + ${hive.group} hive-exec From 7e2f90d083c9bb3a9ab1ebc2edede92e1eb6259f Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 8 Jun 2017 14:33:34 -0700 Subject: [PATCH 37/40] style --- .../security/HiveCredentialProvider.scala | 4 ++-- .../ConfigurableCredentialManagerSuite.scala | 12 +++--------- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 4 ++-- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 8 ++++---- .../yarn/security/AMCredentialRenewer.scala | 2 +- .../YARNConfigurableCredentialManager.scala | 13 +++++-------- ...YARNConfigurableCredentialManagerSuite.scala | 17 +++++++++++------ 8 files changed, 29 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index 4647473c8902f..e6c27bb114300 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -47,7 +47,7 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv case NonFatal(e) => logDebug("Fail to create Hive Configuration", e) hadoopConf - case e: ClassNotFoundException => + case e: NoClassDefFoundError => logError(classNotFoundErrorStr) throw e } @@ -95,7 +95,7 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv None } catch { - case e: ClassNotFoundException => + case e: NoClassDefFoundError => logError(classNotFoundErrorStr) throw e } 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 10446b18f8d47..7050bfb2741d1 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 @@ -81,9 +81,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { val creds = new Credentials() // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. - credentialManager.obtainCredentials( - hadoopConf, - creds) + credentialManager.obtainCredentials(hadoopConf, creds) val tokens = creds.getAllTokens tokens.size() should be (0) } @@ -96,9 +94,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { val hiveCredentialProvider = new HiveCredentialProvider() val credentials = new Credentials() - hiveCredentialProvider.obtainCredentials( - hadoopConf, - credentials) + hiveCredentialProvider.obtainCredentials(hadoopConf, credentials) credentials.getAllTokens.size() should be (0) } @@ -109,9 +105,7 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { val hbaseTokenProvider = new HBaseCredentialProvider() val creds = new Credentials() - hbaseTokenProvider.obtainCredentials( - hadoopConf, - creds) + hbaseTokenProvider.obtainCredentials(hadoopConf, creds) creds.getAllTokens.size should be (0) } 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 37ae0499005af..61c4ddd6bc37a 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 @@ -250,7 +250,7 @@ private[spark] class ApplicationMaster( val credentialManager = new YARNConfigurableCredentialManager( sparkConf, yarnConf, - YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, yarnConf)) + YarnSparkHadoopUtil.get.hadoopFSsToAccess(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 3c108ead21176..9ad35e10332eb 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 @@ -124,7 +124,7 @@ private[spark] class Client( private val credentialManager = new YARNConfigurableCredentialManager( sparkConf, hadoopConf, - YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) + YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) @@ -371,7 +371,7 @@ private[spark] class Client( val fs = destDir.getFileSystem(hadoopConf) // Merge credentials obtained from registered providers - val nearestTimeOfNextRenewal = credentialManager.obtainYARNCredentials( + val nearestTimeOfNextRenewal = credentialManager.obtainCredentials( hadoopConf, credentials) 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 0aa9098ef54fa..979ca04f36eaa 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 @@ -95,7 +95,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val credentialManager = new YARNConfigurableCredentialManager( sparkConf, hadoopConf, - YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) + YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) credentialUpdater = new CredentialUpdater(sparkConf, hadoopConf, credentialManager) credentialUpdater.start() } @@ -113,9 +113,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } /** The filesystems for which YARN should fetch delegation tokens. */ - private[spark] def yarnHadoopFSsToAccess( - sparkConf: SparkConf, - hadoopConf: Configuration): Set[FileSystem] = { + private[spark] def hadoopFSsToAccess( + sparkConf: SparkConf, + hadoopConf: Configuration): Set[FileSystem] = { val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS) .map(new Path(_).getFileSystem(hadoopConf)) .toSet 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 2da75b892c6f4..5b009d8ab080d 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 @@ -174,7 +174,7 @@ private[yarn] class AMCredentialRenewer( keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { // Get a copy of the credentials override def run(): Void = { - nearestNextRenewalTime = credentialManager.obtainYARNCredentials( + nearestNextRenewalTime = credentialManager.obtainCredentials( freshHadoopConf, tempCreds) null 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 index 1d93799242809..b20a9eed50cd5 100644 --- 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 @@ -30,7 +30,6 @@ import org.apache.spark.deploy.security.ConfigurableCredentialManager import org.apache.spark.internal.Logging import org.apache.spark.util.Utils - /** * This class loads credential providers registered under the YARN-specific * [[ServiceCredentialProvider]] interface, as well as the builtin credential providers defined @@ -47,9 +46,9 @@ private[yarn] class YARNConfigurableCredentialManager( // public for testing val credentialProviders = getCredentialProviders - def obtainYARNCredentials( - hadoopConf: Configuration, - creds: Credentials): Long = { + def obtainCredentials( + hadoopConf: Configuration, + creds: Credentials): Long = { val superInterval = configurableCredentialManager.obtainCredentials( hadoopConf, @@ -66,8 +65,7 @@ private[yarn] class YARNConfigurableCredentialManager( }.foldLeft(superInterval)(math.min) } - private def getCredentialProviders: - Map[String, ServiceCredentialProvider] = { + private def getCredentialProviders: Map[String, ServiceCredentialProvider] = { val providers = loadCredentialProviders providers. @@ -76,8 +74,7 @@ private[yarn] class YARNConfigurableCredentialManager( .toMap } - private def loadCredentialProviders: - List[ServiceCredentialProvider] = { + private def loadCredentialProviders: List[ServiceCredentialProvider] = { ServiceLoader.load( classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala index 7a8c5b2dcd3a8..9ad112b783505 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala @@ -24,8 +24,7 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil -class YARNConfigurableCredentialManagerSuite - extends SparkFunSuite with Matchers { +class YARNConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { private var credentialManager: YARNConfigurableCredentialManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -39,11 +38,17 @@ class YARNConfigurableCredentialManagerSuite hadoopConf = new Configuration() } + override def afterAll(): Unit = { + super.afterAll() + + System.clearProperty("SPARK_YARN_MODE") + } + test("Correctly loads credential providers") { credentialManager = new YARNConfigurableCredentialManager( sparkConf, hadoopConf, - YarnSparkHadoopUtil.get.yarnHadoopFSsToAccess(sparkConf, hadoopConf)) + YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) credentialManager.credentialProviders.get("yarn-test") should not be (None) } @@ -55,7 +60,7 @@ class YARNTestCredentialProvider extends ServiceCredentialProvider { override def credentialsRequired(conf: Configuration): Boolean = true override def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] = None + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = None } From 563b80ab7da0e0f5ac0c615bca46a2a9eae58122 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Thu, 8 Jun 2017 14:55:50 -0700 Subject: [PATCH 38/40] Don't throw an exception when Hive classes are not loaded --- .../spark/deploy/security/HiveCredentialProvider.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala index e6c27bb114300..90a961945cd69 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala @@ -48,8 +48,8 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv logDebug("Fail to create Hive Configuration", e) hadoopConf case e: NoClassDefFoundError => - logError(classNotFoundErrorStr) - throw e + logWarning(classNotFoundErrorStr) + hadoopConf } } @@ -96,8 +96,8 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv None } catch { case e: NoClassDefFoundError => - logError(classNotFoundErrorStr) - throw e + logWarning(classNotFoundErrorStr) + None } } From c684d8810824d18f9c9bacdddc69dad4c6fe325b Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Mon, 12 Jun 2017 11:06:05 -0700 Subject: [PATCH 39/40] rename --- core/pom.xml | 8 +-- ...ala => HBaseDelegationTokenProvider.scala} | 6 +-- ...ala => HadoopDelegationTokenManager.scala} | 49 ++++++++++--------- .../HadoopDelegationTokenProvider.scala | 16 +++--- ... => HadoopFSDelegationTokenProvider.scala} | 22 +++++---- ...cala => HiveDelegationTokenProvider.scala} | 43 ++++++++-------- ...> HadoopDelegationTokenManagerSuite.scala} | 42 ++++++++-------- resource-managers/yarn/pom.xml | 2 - .../spark/deploy/yarn/ApplicationMaster.scala | 4 +- .../org/apache/spark/deploy/yarn/Client.scala | 8 ++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 4 +- .../yarn/security/AMCredentialRenewer.scala | 4 +- .../yarn/security/CredentialUpdater.scala | 2 +- ...=> YARNHadoopDelegationTokenManager.scala} | 35 +++++++------ ...RNHadoopDelegationTokenManagerSuite.scala} | 6 +-- 15 files changed, 124 insertions(+), 127 deletions(-) rename core/src/main/scala/org/apache/spark/deploy/security/{HBaseCredentialProvider.scala => HBaseDelegationTokenProvider.scala} (93%) rename core/src/main/scala/org/apache/spark/deploy/security/{ConfigurableCredentialManager.scala => HadoopDelegationTokenManager.scala} (63%) rename core/src/main/scala/org/apache/spark/deploy/security/{HadoopFSCredentialProvider.scala => HadoopFSDelegationTokenProvider.scala} (87%) rename core/src/main/scala/org/apache/spark/deploy/security/{HiveCredentialProvider.scala => HiveDelegationTokenProvider.scala} (76%) rename core/src/test/scala/org/apache/spark/deploy/security/{ConfigurableCredentialManagerSuite.scala => HadoopDelegationTokenManagerSuite.scala} (63%) rename resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/{YARNConfigurableCredentialManager.scala => YARNHadoopDelegationTokenManager.scala} (69%) rename resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/{YARNConfigurableCredentialManagerSuite.scala => YARNHadoopDelegationTokenManagerSuite.scala} (90%) diff --git a/core/pom.xml b/core/pom.xml index 145bd0d705ca1..7dce010a23e33 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -359,10 +359,10 @@ ${hive.group} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala index 5e808474792f7..35621daf9c0d7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala @@ -27,12 +27,12 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[security] class HBaseCredentialProvider +private[security] class HBaseDelegationTokenProvider extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "hbase" - override def obtainCredentials( + override def obtainDelegationTokens( hadoopConf: Configuration, creds: Credentials): Option[Long] = { try { @@ -54,7 +54,7 @@ private[security] class HBaseCredentialProvider None } - override def credentialsRequired(hadoopConf: Configuration): Boolean = { + override def delegationTokensRequired(hadoopConf: Configuration): Boolean = { hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos" } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala similarity index 63% rename from core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index d2b0e539c34e7..89b6f52ba4bca 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/ConfigurableCredentialManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -25,21 +25,21 @@ import org.apache.spark.SparkConf import org.apache.spark.internal.Logging /** - * A ConfigurableCredentialManager to manage all the registered credential providers and offer - * APIs for other modules to obtain credentials as well as renewal time. By default - * [[HadoopFSCredentialProvider]], [[HiveCredentialProvider]] and [[HBaseCredentialProvider]] will - * be loaded in if not explicitly disabled. + * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to + * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], + * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not + * explicitly disabled. * - * Also each credential provider is controlled by 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.security.credentials.hive.enabled. + * Also, each HadoopDelegationTokenProvider is controlled by + * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to + * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be + * enabled/disabled by the configuration spark.security.credentials.hive.enabled. * * @param sparkConf Spark configuration * @param hadoopConf Hadoop configuration * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. */ -private[spark] class ConfigurableCredentialManager( +private[spark] class HadoopDelegationTokenManager( sparkConf: SparkConf, hadoopConf: Configuration, fileSystems: Set[FileSystem]) @@ -50,16 +50,17 @@ private[spark] class ConfigurableCredentialManager( "spark.yarn.security.credentials.%s.enabled") private val providerEnabledConfig = "spark.security.credentials.%s.enabled" - // Maintain all the registered credential providers - private val credentialProviders = getCredentialProviders - logDebug(s"Using the following credential providers: ${credentialProviders.keys.mkString(", ")}.") + // Maintain all the registered delegation token providers + private val delegationTokenProviders = getDelegationTokenProviders + logDebug(s"Using the following delegation token providers: " + + s"${delegationTokenProviders.keys.mkString(", ")}.") - private def getCredentialProviders: Map[String, HadoopDelegationTokenProvider] = { - val providers = List(new HadoopFSCredentialProvider(fileSystems), - new HiveCredentialProvider, - new HBaseCredentialProvider) + private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { + val providers = List(new HadoopFSDelegationTokenProvider(fileSystems), + new HiveDelegationTokenProvider, + new HBaseDelegationTokenProvider) - // Filter out credentials in which spark.security.credentials.{service}.enabled is false. + // Filter out providers for which spark.security.credentials.{service}.enabled is false. providers .filter { p => isServiceEnabled(p.serviceName) } .map { p => (p.serviceName, p) } @@ -90,10 +91,10 @@ private[spark] class ConfigurableCredentialManager( } /** - * Get credential provider for the specified service. + * Get delegation token provider for the specified service. */ - def getServiceCredentialProvider(service: String): Option[HadoopDelegationTokenProvider] = { - credentialProviders.get(service) + def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { + delegationTokenProviders.get(service) } /** @@ -102,12 +103,12 @@ private[spark] class ConfigurableCredentialManager( * * @return Time after which the fetched delegation tokens should be renewed. */ - def obtainCredentials( + def obtainDelegationTokens( hadoopConf: Configuration, creds: Credentials): Long = { - credentialProviders.values.flatMap { provider => - if (provider.credentialsRequired(hadoopConf)) { - provider.obtainCredentials(hadoopConf, creds) + delegationTokenProviders.values.flatMap { provider => + if (provider.delegationTokensRequired(hadoopConf)) { + provider.obtainDelegationTokens(hadoopConf, creds) } else { logDebug(s"Service ${provider.serviceName} does not require a token." + s" Check your configuration to see if security is disabled or not.") diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala index fb89e289aa493..f162e7e58c53a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala @@ -26,25 +26,25 @@ import org.apache.hadoop.security.Credentials private[spark] trait HadoopDelegationTokenProvider { /** - * Name of the service to provide credentials. This name should be unique, Spark internally will - * use this name to differentiate credential provider. + * Name of the service to provide delegation tokens. This name should be unique. Spark will + * internally use this name to differentiate delegation token providers. */ def serviceName: String /** - * Returns true if credentials are required for this service. By default, it is based on whether - * Hadoop security is enabled. + * Returns true if delegation tokens are required for this service. By default, it is based on + * whether Hadoop security is enabled. */ - def credentialsRequired(hadoopConf: Configuration): Boolean + def delegationTokensRequired(hadoopConf: Configuration): Boolean /** - * Obtain credentials for this service and get the time of the next renewal. + * Obtain delegation tokens for this service and get the time of the next renewal. * @param hadoopConf Configuration of current Hadoop Compatible system. * @param creds Credentials to add tokens and security keys to. - * @return If this Credential is renewable and can be renewed, return the time of the next + * @return If the returned tokens are renewable and can be renewed, return the time of the next * renewal, otherwise None should be returned. */ - def obtainCredentials( + def obtainDelegationTokens( hadoopConf: Configuration, creds: Credentials): Option[Long] } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index ee6892ff9b3d1..13157f33e2bf9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -29,16 +29,17 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.SparkException import org.apache.spark.internal.Logging -private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) +private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Set[FileSystem]) extends HadoopDelegationTokenProvider 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. + + // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. + // If None, no token renewer is specified or no token can be renewed, + // so we cannot get the token renewal interval. private var tokenRenewalInterval: Option[Long] = null override val serviceName: String = "hadoopfs" - override def obtainCredentials( + override def obtainDelegationTokens( hadoopConf: Configuration, creds: Credentials): Option[Long] = { @@ -68,7 +69,7 @@ private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) nextRenewalDate } - def credentialsRequired(hadoopConf: Configuration): Boolean = { + def delegationTokensRequired(hadoopConf: Configuration): Boolean = { UserGroupInformation.isSecurityEnabled } @@ -86,8 +87,9 @@ private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) } private def fetchDelegationTokens( - renewer: String, - filesystems: Set[FileSystem]): Credentials = { + renewer: String, + filesystems: Set[FileSystem]): Credentials = { + val creds = new Credentials() filesystems.foreach { fs => @@ -99,8 +101,8 @@ private[deploy] class HadoopFSCredentialProvider(fileSystems: Set[FileSystem]) } private def getTokenRenewalInterval( - hadoopConf: Configuration, - filesystems: Set[FileSystem]): Option[Long] = { + hadoopConf: Configuration, + filesystems: Set[FileSystem]): 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. diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala similarity index 76% rename from core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala rename to core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala index 90a961945cd69..53b9f898c6e7d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveCredentialProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala @@ -33,12 +33,13 @@ import org.apache.hadoop.security.token.Token import org.apache.spark.internal.Logging import org.apache.spark.util.Utils -private[security] class HiveCredentialProvider extends HadoopDelegationTokenProvider with Logging { +private[security] class HiveDelegationTokenProvider + extends HadoopDelegationTokenProvider with Logging { override def serviceName: String = "hive" - private val classNotFoundErrorStr = "You are attempting to use the HiveCredentialProvider," + - "but your Spark distribution is not built with Hive libraries." + private val classNotFoundErrorStr = s"You are attempting to use the " + + s"${getClass.getCanonicalName}, but your Spark distribution is not built with Hive libraries." private def hiveConf(hadoopConf: Configuration): Configuration = { try { @@ -53,12 +54,12 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv } } - override def credentialsRequired(hadoopConf: Configuration): Boolean = { + override def delegationTokensRequired(hadoopConf: Configuration): Boolean = { UserGroupInformation.isSecurityEnabled && hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty } - override def obtainCredentials( + override def obtainDelegationTokens( hadoopConf: Configuration, creds: Credentials): Option[Long] = { try { @@ -74,30 +75,28 @@ private[security] class HiveCredentialProvider extends HadoopDelegationTokenProv logDebug(s"Getting Hive delegation token for ${currentUser.getUserName()} against " + s"$principal at $metastoreUri") - try { - doAsRealUser { - val hive = Hive.get(conf, classOf[HiveConf]) - val tokenStr = hive.getDelegationToken(currentUser.getUserName(), principal) - - val hive2Token = new Token[DelegationTokenIdentifier]() - hive2Token.decodeFromUrlString(tokenStr) - logInfo(s"Get Token from hive metastore: ${hive2Token.toString}") - creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) - } - } catch { - case NonFatal(e) => - logDebug(s"Fail to get token from service $serviceName", e) - } finally { - Utils.tryLogNonFatalError { - Hive.closeCurrent() - } + doAsRealUser { + val hive = Hive.get(conf, classOf[HiveConf]) + val tokenStr = hive.getDelegationToken(currentUser.getUserName(), principal) + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + logInfo(s"Get Token from hive metastore: ${hive2Token.toString}") + creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) } None } catch { + case NonFatal(e) => + logDebug(s"Failed to get token from service $serviceName", e) + None case e: NoClassDefFoundError => logWarning(classNotFoundErrorStr) None + } finally { + Utils.tryLogNonFatalError { + Hive.closeCurrent() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala similarity index 63% rename from core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala rename to core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index 7050bfb2741d1..335f3449cb782 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/ConfigurableCredentialManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -24,8 +24,8 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} -class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { - private var credentialManager: ConfigurableCredentialManager = null +class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { + private var delegationTokenManager: HadoopDelegationTokenManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -37,51 +37,51 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { } test("Correctly load default credential providers") { - credentialManager = new ConfigurableCredentialManager( + delegationTokenManager = new HadoopDelegationTokenManager( sparkConf, hadoopConf, hadoopFSsToAccess(hadoopConf)) - credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) - credentialManager.getServiceCredentialProvider("hbase") should not be (None) - credentialManager.getServiceCredentialProvider("hive") should not be (None) - credentialManager.getServiceCredentialProvider("bogus") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("bogus") should be (None) } test("disable hive credential provider") { sparkConf.set("spark.security.credentials.hive.enabled", "false") - credentialManager = new ConfigurableCredentialManager( + delegationTokenManager = new HadoopDelegationTokenManager( sparkConf, hadoopConf, hadoopFSsToAccess(hadoopConf)) - credentialManager.getServiceCredentialProvider("hadoopfs") should not be (None) - credentialManager.getServiceCredentialProvider("hbase") should not be (None) - credentialManager.getServiceCredentialProvider("hive") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) } test("using deprecated configurations") { sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") - credentialManager = new ConfigurableCredentialManager( + delegationTokenManager = new HadoopDelegationTokenManager( sparkConf, hadoopConf, hadoopFSsToAccess(hadoopConf)) - credentialManager.getServiceCredentialProvider("hadoopfs") should be (None) - credentialManager.getServiceCredentialProvider("hive") should be (None) - credentialManager.getServiceCredentialProvider("hbase") should not be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) + delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) } test("verify no credentials are obtained") { - credentialManager = new ConfigurableCredentialManager( + delegationTokenManager = new HadoopDelegationTokenManager( sparkConf, hadoopConf, hadoopFSsToAccess(hadoopConf)) val creds = new Credentials() // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. - credentialManager.obtainCredentials(hadoopConf, creds) + delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) val tokens = creds.getAllTokens tokens.size() should be (0) } @@ -92,9 +92,9 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { // 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 hiveCredentialProvider = new HiveDelegationTokenProvider() val credentials = new Credentials() - hiveCredentialProvider.obtainCredentials(hadoopConf, credentials) + hiveCredentialProvider.obtainDelegationTokens(hadoopConf, credentials) credentials.getAllTokens.size() should be (0) } @@ -103,9 +103,9 @@ class ConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { val hadoopConf = new Configuration() hadoopConf.set("hbase.security.authentication", "kerberos") - val hbaseTokenProvider = new HBaseCredentialProvider() + val hbaseTokenProvider = new HBaseDelegationTokenProvider() val creds = new Credentials() - hbaseTokenProvider.obtainCredentials(hadoopConf, creds) + hbaseTokenProvider.obtainDelegationTokens(hadoopConf, creds) creds.getAllTokens.size should be (0) } diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 3869b016ace24..5ab9758c5ee9e 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -189,8 +189,6 @@ libfb303 provided - - 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 61c4ddd6bc37a..31f4567a17661 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 @@ -38,7 +38,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, YARNConfigurableCredentialManager} +import org.apache.spark.deploy.yarn.security.{AMCredentialRenewer, YARNHadoopDelegationTokenManager} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.rpc._ @@ -247,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 YARNConfigurableCredentialManager( + val credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, yarnConf, YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, yarnConf)) 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 9ad35e10332eb..8a07c630d5c04 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 @@ -49,7 +49,7 @@ 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.yarn.config._ -import org.apache.spark.deploy.yarn.security.YARNConfigurableCredentialManager +import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager 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 YARNConfigurableCredentialManager( + private val credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, hadoopConf, YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) @@ -371,9 +371,7 @@ private[spark] class Client( val fs = destDir.getFileSystem(hadoopConf) // Merge credentials obtained from registered providers - val nearestTimeOfNextRenewal = credentialManager.obtainCredentials( - hadoopConf, - credentials) + val nearestTimeOfNextRenewal = credentialManager.obtainDelegationTokens(hadoopConf, credentials) if (credentials != null) { // Add credentials to current user's UGI, so that following operations don't need to use the 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 979ca04f36eaa..d4d03654266fc 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 @@ -38,7 +38,7 @@ import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.deploy.yarn.security.CredentialUpdater -import org.apache.spark.deploy.yarn.security.YARNConfigurableCredentialManager +import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils import org.apache.spark.util.Utils @@ -92,7 +92,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { private[spark] override def startCredentialUpdater(sparkConf: SparkConf): Unit = { val hadoopConf = newConfiguration(sparkConf) - val credentialManager = new YARNConfigurableCredentialManager( + val credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, hadoopConf, YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) 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 5b009d8ab080d..68a2e9e70a78b 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 @@ -54,7 +54,7 @@ import org.apache.spark.util.ThreadUtils private[yarn] class AMCredentialRenewer( sparkConf: SparkConf, hadoopConf: Configuration, - credentialManager: YARNConfigurableCredentialManager) extends Logging { + credentialManager: YARNHadoopDelegationTokenManager) extends Logging { private var lastCredentialsFileSuffix = 0 @@ -174,7 +174,7 @@ private[yarn] class AMCredentialRenewer( keytabLoggedInUGI.doAs(new PrivilegedExceptionAction[Void] { // Get a copy of the credentials override def run(): Void = { - nearestNextRenewalTime = credentialManager.obtainCredentials( + nearestNextRenewalTime = credentialManager.obtainDelegationTokens( freshHadoopConf, tempCreds) null 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 11c046052c2b7..fe173dffc22a8 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 @@ -34,7 +34,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class CredentialUpdater( sparkConf: SparkConf, hadoopConf: Configuration, - credentialManager: YARNConfigurableCredentialManager) extends Logging { + credentialManager: YARNHadoopDelegationTokenManager) extends Logging { @volatile private var lastCredentialsFileSuffix = 0 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/YARNHadoopDelegationTokenManager.scala similarity index 69% rename from resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManager.scala rename to resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala index b20a9eed50cd5..bbd17c8fc1272 100644 --- 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/YARNHadoopDelegationTokenManager.scala @@ -26,33 +26,34 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.ConfigurableCredentialManager +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.util.Utils /** - * This class loads credential providers registered under the YARN-specific - * [[ServiceCredentialProvider]] interface, as well as the builtin credential providers defined - * in [[ConfigurableCredentialManager]]. + * This class loads delegation token providers registered under the YARN-specific + * [[ServiceCredentialProvider]] interface, as well as the builtin providers defined + * in [[HadoopDelegationTokenManager]]. */ -private[yarn] class YARNConfigurableCredentialManager( +private[yarn] class YARNHadoopDelegationTokenManager( sparkConf: SparkConf, hadoopConf: Configuration, fileSystems: Set[FileSystem]) extends Logging { - private val configurableCredentialManager = - new ConfigurableCredentialManager(sparkConf, hadoopConf, fileSystems) + private val delegationTokenManager = + new HadoopDelegationTokenManager(sparkConf, hadoopConf, fileSystems) // public for testing val credentialProviders = getCredentialProviders - def obtainCredentials( - hadoopConf: Configuration, - creds: Credentials): Long = { - - val superInterval = configurableCredentialManager.obtainCredentials( - hadoopConf, - creds) + /** + * Writes delegation tokens to creds. Delegation tokens are fetched from all registered + * providers. + * + * @return Time after which the fetched delegation tokens should be renewed. + */ + def obtainDelegationTokens(hadoopConf: Configuration, creds: Credentials): Long = { + val superInterval = delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) credentialProviders.values.flatMap { provider => if (provider.credentialsRequired(hadoopConf)) { @@ -69,15 +70,13 @@ private[yarn] class YARNConfigurableCredentialManager( val providers = loadCredentialProviders providers. - filter { p => configurableCredentialManager.isServiceEnabled(p.serviceName) } + filter { p => delegationTokenManager.isServiceEnabled(p.serviceName) } .map { p => (p.serviceName, p) } .toMap } private def loadCredentialProviders: List[ServiceCredentialProvider] = { - ServiceLoader.load( - classOf[ServiceCredentialProvider], - Utils.getContextOrSparkClassLoader) + ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) .asScala .toList } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala similarity index 90% rename from resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala rename to resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala index 9ad112b783505..2b226eff5ce19 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNConfigurableCredentialManagerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala @@ -24,8 +24,8 @@ import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil -class YARNConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers { - private var credentialManager: YARNConfigurableCredentialManager = null +class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { + private var credentialManager: YARNHadoopDelegationTokenManager = null private var sparkConf: SparkConf = null private var hadoopConf: Configuration = null @@ -45,7 +45,7 @@ class YARNConfigurableCredentialManagerSuite extends SparkFunSuite with Matchers } test("Correctly loads credential providers") { - credentialManager = new YARNConfigurableCredentialManager( + credentialManager = new YARNHadoopDelegationTokenManager( sparkConf, hadoopConf, YarnSparkHadoopUtil.get.hadoopFSsToAccess(sparkConf, hadoopConf)) From c4149ddb940c32285fde8b1c08a5b212fced5d66 Mon Sep 17 00:00:00 2001 From: Michael Gummelt Date: Wed, 14 Jun 2017 17:31:12 -0700 Subject: [PATCH 40/40] fix docs --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 07415952fec52..30fe0885f3996 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.security.ServiceCredentialProvider` +`java.util.ServiceLoader`). To do that, implementations of `org.apache.spark.deploy.yarn.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.security.credentials.{service}.enabled` to `false`, where `{service}` is the name of