Skip to content

Commit

Permalink
Merge branch 'master' into SPARK-20653
Browse files Browse the repository at this point in the history
  • Loading branch information
Marcelo Vanzin committed Dec 8, 2017
2 parents 3f7c25d + e4639fa commit 2606fcd
Show file tree
Hide file tree
Showing 192 changed files with 6,073 additions and 2,582 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -347,6 +347,8 @@ public long spill(long numBytes) throws IOException {
return 0L;
}

updatePeakMemoryUsed();

// TODO: use existing ShuffleWriteMetrics
ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics();

Expand Down Expand Up @@ -424,6 +426,7 @@ public MapIterator iterator() {
* `lookup()`, the behavior of the returned iterator is undefined.
*/
public MapIterator destructiveIterator() {
updatePeakMemoryUsed();
return new MapIterator(numValues, loc, true);
}

Expand Down Expand Up @@ -879,6 +882,7 @@ public LongArray getArray() {
* Reset this map to initialized state.
*/
public void reset() {
updatePeakMemoryUsed();
numKeys = 0;
numValues = 0;
freeArray(longArray);
Expand Down
102 changes: 51 additions & 51 deletions core/src/main/scala/org/apache/spark/SecurityManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,19 @@ package org.apache.spark

import java.lang.{Byte => JByte}
import java.net.{Authenticator, PasswordAuthentication}
import java.nio.charset.StandardCharsets.UTF_8
import java.security.{KeyStore, SecureRandom}
import java.security.cert.X509Certificate
import javax.net.ssl._

import com.google.common.hash.HashCodes
import com.google.common.io.Files
import org.apache.hadoop.io.Text
import org.apache.hadoop.security.{Credentials, UserGroupInformation}

import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.network.sasl.SecretKeyHolder
import org.apache.spark.util.Utils

Expand Down Expand Up @@ -225,7 +227,6 @@ private[spark] class SecurityManager(
setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", ""));
setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", ""));

private val secretKey = generateSecretKey()
logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") +
"; ui acls " + (if (aclsOn) "enabled" else "disabled") +
"; users with view permissions: " + viewAcls.toString() +
Expand Down Expand Up @@ -416,50 +417,6 @@ private[spark] class SecurityManager(

def getIOEncryptionKey(): Option[Array[Byte]] = ioEncryptionKey

/**
* Generates or looks up the secret key.
*
* The way the key is stored depends on the Spark deployment mode. Yarn
* uses the Hadoop UGI.
*
* For non-Yarn deployments, If the config variable is not set
* we throw an exception.
*/
private def generateSecretKey(): String = {
if (!isAuthenticationEnabled) {
null
} else if (SparkHadoopUtil.get.isYarnMode) {
// In YARN mode, the secure cookie will be created by the driver and stashed in the
// user's credentials, where executors can get it. The check for an array of size 0
// is because of the test code in YarnSparkHadoopUtilSuite.
val secretKey = SparkHadoopUtil.get.getSecretKeyFromUserCredentials(SECRET_LOOKUP_KEY)
if (secretKey == null || secretKey.length == 0) {
logDebug("generateSecretKey: yarn mode, secret key from credentials is null")
val rnd = new SecureRandom()
val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE
val secret = new Array[Byte](length)
rnd.nextBytes(secret)

val cookie = HashCodes.fromBytes(secret).toString()
SparkHadoopUtil.get.addSecretKeyToUserCredentials(SECRET_LOOKUP_KEY, cookie)
cookie
} else {
new Text(secretKey).toString
}
} else {
// user must have set spark.authenticate.secret config
// For Master/Worker, auth secret is in conf; for Executors, it is in env variable
Option(sparkConf.getenv(SecurityManager.ENV_AUTH_SECRET))
.orElse(sparkConf.getOption(SecurityManager.SPARK_AUTH_SECRET_CONF)) match {
case Some(value) => value
case None =>
throw new IllegalArgumentException(
"Error: a secret key must be specified via the " +
SecurityManager.SPARK_AUTH_SECRET_CONF + " config")
}
}
}

/**
* Check to see if Acls for the UI are enabled
* @return true if UI authentication is enabled, otherwise false
Expand Down Expand Up @@ -542,7 +499,51 @@ private[spark] class SecurityManager(
* Gets the secret key.
* @return the secret key as a String if authentication is enabled, otherwise returns null
*/
def getSecretKey(): String = secretKey
def getSecretKey(): String = {
if (isAuthenticationEnabled) {
val creds = UserGroupInformation.getCurrentUser().getCredentials()
Option(creds.getSecretKey(SECRET_LOOKUP_KEY))
.map { bytes => new String(bytes, UTF_8) }
.orElse(Option(sparkConf.getenv(ENV_AUTH_SECRET)))
.orElse(sparkConf.getOption(SPARK_AUTH_SECRET_CONF))
.getOrElse {
throw new IllegalArgumentException(
s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config")
}
} else {
null
}
}

/**
* Initialize the authentication secret.
*
* If authentication is disabled, do nothing.
*
* In YARN mode, generate a new secret and store it in the current user's credentials.
*
* In other modes, assert that the auth secret is set in the configuration.
*/
def initializeAuth(): Unit = {
if (!sparkConf.get(NETWORK_AUTH_ENABLED)) {
return
}

if (sparkConf.get(SparkLauncher.SPARK_MASTER, null) != "yarn") {
require(sparkConf.contains(SPARK_AUTH_SECRET_CONF),
s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.")
return
}

val rnd = new SecureRandom()
val length = sparkConf.getInt("spark.authenticate.secretBitLength", 256) / JByte.SIZE
val secretBytes = new Array[Byte](length)
rnd.nextBytes(secretBytes)

val creds = new Credentials()
creds.addSecretKey(SECRET_LOOKUP_KEY, secretBytes)
UserGroupInformation.getCurrentUser().addCredentials(creds)
}

// Default SecurityManager only has a single secret key, so ignore appId.
override def getSaslUser(appId: String): String = getSaslUser()
Expand All @@ -551,13 +552,12 @@ private[spark] class SecurityManager(

private[spark] object SecurityManager {

val SPARK_AUTH_CONF: String = "spark.authenticate"
val SPARK_AUTH_SECRET_CONF: String = "spark.authenticate.secret"
val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key
val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret"
// This is used to set auth secret to an executor's env variable. It should have the same
// value as SPARK_AUTH_SECRET_CONF set in SparkConf
val ENV_AUTH_SECRET = "_SPARK_AUTH_SECRET"

// key used to store the spark secret in the Hadoop UGI
val SECRET_LOOKUP_KEY = "sparkCookie"

val SECRET_LOOKUP_KEY = new Text("sparkCookie")
}
9 changes: 5 additions & 4 deletions core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -413,8 +413,6 @@ class SparkContext(config: SparkConf) extends Logging {
}
}

if (master == "yarn" && deployMode == "client") System.setProperty("SPARK_YARN_MODE", "true")

_listenerBus = new LiveListenerBus(_conf)

// Initialize the app status store and listener before SparkEnv is created so that it gets
Expand Down Expand Up @@ -1837,7 +1835,11 @@ class SparkContext(config: SparkConf) extends Logging {
Utils.validateURL(uri)
uri.getScheme match {
// A JAR file which exists only on the driver node
case null | "file" => addJarFile(new File(uri.getPath))
case null =>
// SPARK-22585 path without schema is not url encoded
addJarFile(new File(uri.getRawPath))
// A JAR file which exists only on the driver node
case "file" => addJarFile(new File(uri.getPath))
// A JAR file which exists locally on every worker node
case "local" => "file:" + uri.getPath
case _ => path
Expand Down Expand Up @@ -1951,7 +1953,6 @@ class SparkContext(config: SparkConf) extends Logging {
// `SparkContext` is stopped.
localProperties.remove()
// Unset YARN mode system env variable, to allow switching between cluster types.
System.clearProperty("SPARK_YARN_MODE")
SparkContext.clearActiveContext()
logInfo("Successfully stopped SparkContext")
}
Expand Down
4 changes: 4 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -234,6 +234,10 @@ object SparkEnv extends Logging {
}

val securityManager = new SecurityManager(conf, ioEncryptionKey)
if (isDriver) {
securityManager.initializeAuth()
}

ioEncryptionKey.foreach { _ =>
if (!securityManager.isEncryptionEnabled()) {
logWarning("I/O encryption enabled without RPC encryption: keys will be visible on the " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -317,10 +317,6 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
logDebug("Exception thrown after task interruption", e)
throw new TaskKilledException(context.getKillReason().getOrElse("unknown reason"))

case e: Exception if env.isStopped =>
logDebug("Exception thrown after context is stopped", e)
null.asInstanceOf[OUT] // exit silently

case e: Exception if writerThread.exception.isDefined =>
logError("Python worker exited unexpectedly (crashed)", e)
logError("This may have been caused by a prior exception:", writerThread.exception.get)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
private def calcChecksum(block: ByteBuffer): Int = {
val adler = new Adler32()
if (block.hasArray) {
adler.update(block.array, block.arrayOffset + block.position, block.limit - block.position)
adler.update(block.array, block.arrayOffset + block.position(), block.limit()
- block.position())
} else {
val bytes = new Array[Byte](block.remaining())
block.duplicate.get(bytes)
Expand Down
8 changes: 7 additions & 1 deletion core/src/main/scala/org/apache/spark/deploy/Client.scala
Original file line number Diff line number Diff line change
Expand Up @@ -217,8 +217,13 @@ object Client {
println("Use ./bin/spark-submit with \"--master spark://host:port\"")
}
// scalastyle:on println
new ClientApp().start(args, new SparkConf())
}
}

val conf = new SparkConf()
private[spark] class ClientApp extends SparkApplication {

override def start(args: Array[String], conf: SparkConf): Unit = {
val driverArgs = new ClientArguments(args)

if (!conf.contains("spark.rpc.askTimeout")) {
Expand All @@ -235,4 +240,5 @@ object Client {

rpcEnv.awaitTermination()
}

}
48 changes: 6 additions & 42 deletions core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
Original file line number Diff line number Diff line change
Expand Up @@ -75,9 +75,7 @@ class SparkHadoopUtil extends Logging {
}

def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) {
for (token <- source.getTokens.asScala) {
dest.addToken(token)
}
dest.addCredentials(source.getCredentials())
}

/**
Expand Down Expand Up @@ -120,16 +118,9 @@ class SparkHadoopUtil extends Logging {
* Add any user credentials to the job conf which are necessary for running on a secure Hadoop
* cluster.
*/
def addCredentials(conf: JobConf) {}

def isYarnMode(): Boolean = { false }

def addSecretKeyToUserCredentials(key: String, secret: String) {}

def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null }

def getCurrentUserCredentials(): Credentials = {
UserGroupInformation.getCurrentUser().getCredentials()
def addCredentials(conf: JobConf): Unit = {
val jobCreds = conf.getCredentials()
jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials())
}

def addCurrentUserCredentials(creds: Credentials): Unit = {
Expand Down Expand Up @@ -328,17 +319,6 @@ class SparkHadoopUtil extends Logging {
}
}

/**
* Start a thread to periodically update the current user's credentials with new credentials so
* that access to secured service does not fail.
*/
private[spark] def startCredentialUpdater(conf: SparkConf) {}

/**
* Stop the thread that does the credential updates.
*/
private[spark] def stopCredentialUpdater() {}

/**
* Return a fresh Hadoop configuration, bypassing the HDFS cache mechanism.
* This is to prevent the DFSClient from using an old cached token to connect to the NameNode.
Expand Down Expand Up @@ -441,14 +421,7 @@ class SparkHadoopUtil extends Logging {

object SparkHadoopUtil {

private lazy val hadoop = new SparkHadoopUtil
private lazy val yarn = try {
Utils.classForName("org.apache.spark.deploy.yarn.YarnSparkHadoopUtil")
.newInstance()
.asInstanceOf[SparkHadoopUtil]
} catch {
case e: Exception => throw new SparkException("Unable to load YARN support", e)
}
private lazy val instance = new SparkHadoopUtil

val SPARK_YARN_CREDS_TEMP_EXTENSION = ".tmp"

Expand All @@ -462,16 +435,7 @@ object SparkHadoopUtil {
*/
private[spark] val UPDATE_INPUT_METRICS_INTERVAL_RECORDS = 1000

def get: SparkHadoopUtil = {
// Check each time to support changing to/from YARN
val yarnMode = java.lang.Boolean.parseBoolean(
System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))
if (yarnMode) {
yarn
} else {
hadoop
}
}
def get: SparkHadoopUtil = instance

/**
* Given an expiration date (e.g. for Hadoop Delegation Tokens) return a the date
Expand Down

0 comments on commit 2606fcd

Please sign in to comment.