-
+
{providerConfig.map { case (k, v) => - {k}: {v}
}}
@@ -58,7 +58,7 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("")
{
if (allAppsSize > 0) {
++
- ++
+
++
++
++
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index d9c8fda99ef97..b822a48e98e91 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -106,8 +106,8 @@ class HistoryServer(
}
}
- def getSparkUI(appKey: String): Option[SparkUI] = {
- appCache.getSparkUI(appKey)
+ override def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T = {
+ appCache.withSparkUI(appId, attemptId)(fn)
}
initialize()
@@ -140,7 +140,6 @@ class HistoryServer(
override def stop() {
super.stop()
provider.stop()
- appCache.stop()
}
/** Attach a reconstructed UI to this server. Only valid after bind(). */
@@ -158,6 +157,7 @@ class HistoryServer(
override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = {
assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs")
ui.getHandlers.foreach(detachHandler)
+ provider.onUIDetached(appId, attemptId, ui)
}
/**
@@ -224,15 +224,13 @@ class HistoryServer(
*/
private def loadAppUi(appId: String, attemptId: Option[String]): Boolean = {
try {
- appCache.get(appId, attemptId)
+ appCache.withSparkUI(appId, attemptId) { _ =>
+ // Do nothing, just force the UI to load.
+ }
true
} catch {
- case NonFatal(e) => e.getCause() match {
- case nsee: NoSuchElementException =>
- false
-
- case cause: Exception => throw cause
- }
+ case NonFatal(e: NoSuchElementException) =>
+ false
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/config.scala b/core/src/main/scala/org/apache/spark/deploy/history/config.scala
new file mode 100644
index 0000000000000..52dedc1a2ed41
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/history/config.scala
@@ -0,0 +1,42 @@
+/*
+ * 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.history
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config.ConfigBuilder
+
+private[spark] object config {
+
+ val DEFAULT_LOG_DIR = "file:/tmp/spark-events"
+
+ val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory")
+ .stringConf
+ .createWithDefault(DEFAULT_LOG_DIR)
+
+ val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge")
+ .timeConf(TimeUnit.SECONDS)
+ .createWithDefaultString("7d")
+
+ val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path")
+ .doc("Local directory where to cache application history information. By default this is " +
+ "not set, meaning all history information will be kept in memory.")
+ .stringConf
+ .createOptional
+
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index e030cac60a8e4..2c78c15773af2 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
@@ -581,7 +581,13 @@ private[deploy] class Master(
* The number of cores assigned to each executor is configurable. When this is explicitly set,
* multiple executors from the same application may be launched on the same worker if the worker
* has enough cores and memory. Otherwise, each executor grabs all the cores available on the
- * worker by default, in which case only one executor may be launched on each worker.
+ * worker by default, in which case only one executor per application may be launched on each
+ * worker during one single schedule iteration.
+ * Note that when `spark.executor.cores` is not set, we may still launch multiple executors from
+ * the same application on the same worker. Consider appA and appB both have one executor running
+ * on worker1, and appA.coresLeft > 0, then appB is finished and release all its cores on worker1,
+ * thus for the next schedule iteration, appA launches a new executor that grabs all the free
+ * cores on worker1, therefore we get multiple executors from appA running on worker1.
*
* It is important to allocate coresPerExecutor on each worker at a time (instead of 1 core
* at a time). Consider the following example: cluster has 4 workers with 16 cores each.
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
index 0164084ab129e..22b65abce611a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala
@@ -139,7 +139,9 @@ private[rest] class StandaloneSubmitRequestServlet(
val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath")
val superviseDriver = sparkProperties.get("spark.driver.supervise")
val appArgs = request.appArgs
- val environmentVariables = request.environmentVariables
+ // Filter SPARK_LOCAL_(IP|HOSTNAME) environment variables from being set on the remote system.
+ val environmentVariables =
+ request.environmentVariables.filterNot(x => x._1.matches("SPARK_LOCAL_(IP|HOSTNAME)"))
// Construct driver description
val conf = new SparkConf(false)
diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
index 78b0e6b2cbf39..5dcde4ec3a8a4 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala
@@ -56,7 +56,9 @@ private[security] class HBaseDelegationTokenProvider
None
}
- override def delegationTokensRequired(hadoopConf: Configuration): Boolean = {
+ override def delegationTokensRequired(
+ sparkConf: SparkConf,
+ hadoopConf: Configuration): Boolean = {
hbaseConf(hadoopConf).get("hbase.security.authentication") == "kerberos"
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
index c134b7ebe38fa..483d0deec8070 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala
@@ -115,7 +115,7 @@ private[spark] class HadoopDelegationTokenManager(
hadoopConf: Configuration,
creds: Credentials): Long = {
delegationTokenProviders.values.flatMap { provider =>
- if (provider.delegationTokensRequired(hadoopConf)) {
+ if (provider.delegationTokensRequired(sparkConf, hadoopConf)) {
provider.obtainDelegationTokens(hadoopConf, sparkConf, creds)
} else {
logDebug(s"Service ${provider.serviceName} does not require a token." +
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 1ba245e84af4b..ed0905088ab25 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
@@ -37,7 +37,7 @@ private[spark] trait HadoopDelegationTokenProvider {
* Returns true if delegation tokens are required for this service. By default, it is based on
* whether Hadoop security is enabled.
*/
- def delegationTokensRequired(hadoopConf: Configuration): Boolean
+ def delegationTokensRequired(sparkConf: SparkConf, hadoopConf: Configuration): Boolean
/**
* Obtain delegation tokens for this service and get the time of the next renewal.
diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
index 300773c58b183..21ca669ea98f0 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala
@@ -69,7 +69,9 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration
nextRenewalDate
}
- def delegationTokensRequired(hadoopConf: Configuration): Boolean = {
+ override def delegationTokensRequired(
+ sparkConf: SparkConf,
+ hadoopConf: Configuration): Boolean = {
UserGroupInformation.isSecurityEnabled
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
index b31cc595ed83b..ece5ce79c650d 100644
--- a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala
@@ -31,7 +31,9 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.hadoop.security.token.Token
import org.apache.spark.SparkConf
+import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.KEYTAB
import org.apache.spark.util.Utils
private[security] class HiveDelegationTokenProvider
@@ -55,9 +57,21 @@ private[security] class HiveDelegationTokenProvider
}
}
- override def delegationTokensRequired(hadoopConf: Configuration): Boolean = {
+ override def delegationTokensRequired(
+ sparkConf: SparkConf,
+ hadoopConf: Configuration): Boolean = {
+ // Delegation tokens are needed only when:
+ // - trying to connect to a secure metastore
+ // - either deploying in cluster mode without a keytab, or impersonating another user
+ //
+ // Other modes (such as client with or without keytab, or cluster mode with keytab) do not need
+ // a delegation token, since there's a valid kerberos TGT for the right user available to the
+ // driver, which is the only process that connects to the HMS.
+ val deployMode = sparkConf.get("spark.submit.deployMode", "client")
UserGroupInformation.isSecurityEnabled &&
- hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty
+ hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty &&
+ (SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser()) ||
+ (deployMode == "cluster" && !sparkConf.contains(KEYTAB)))
}
override def obtainDelegationTokens(
@@ -83,7 +97,7 @@ private[security] class HiveDelegationTokenProvider
val hive2Token = new Token[DelegationTokenIdentifier]()
hive2Token.decodeFromUrlString(tokenStr)
- logInfo(s"Get Token from hive metastore: ${hive2Token.toString}")
+ logDebug(s"Get Token from hive metastore: ${hive2Token.toString}")
creds.addToken(new Text("hive.server2.delegation.token"), hive2Token)
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
index c1671192e0c64..b19c9904d5982 100644
--- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala
@@ -23,6 +23,7 @@ import org.apache.commons.lang3.StringUtils
import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil, SparkSubmit}
+import org.apache.spark.internal.Logging
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, Utils}
@@ -30,7 +31,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader, U
* Utility object for launching driver programs such that they share fate with the Worker process.
* This is used in standalone cluster mode only.
*/
-object DriverWrapper {
+object DriverWrapper extends Logging {
def main(args: Array[String]) {
args.toList match {
/*
@@ -41,8 +42,10 @@ object DriverWrapper {
*/
case workerUrl :: userJar :: mainClass :: extraArgs =>
val conf = new SparkConf()
- val rpcEnv = RpcEnv.create("Driver",
- Utils.localHostName(), 0, conf, new SecurityManager(conf))
+ val host: String = Utils.localHostName()
+ val port: Int = sys.props.getOrElse("spark.driver.port", "0").toInt
+ val rpcEnv = RpcEnv.create("Driver", host, port, conf, new SecurityManager(conf))
+ logInfo(s"Driver address: ${rpcEnv.address}")
rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl))
val currentLoader = Thread.currentThread.getContextClassLoader
diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala
index 44a2815b81a73..6f0247b73070d 100644
--- a/core/src/main/scala/org/apache/spark/internal/config/package.scala
+++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala
@@ -41,6 +41,29 @@ package object config {
.bytesConf(ByteUnit.MiB)
.createWithDefaultString("1g")
+ private[spark] val EVENT_LOG_COMPRESS =
+ ConfigBuilder("spark.eventLog.compress")
+ .booleanConf
+ .createWithDefault(false)
+
+ private[spark] val EVENT_LOG_BLOCK_UPDATES =
+ ConfigBuilder("spark.eventLog.logBlockUpdates.enabled")
+ .booleanConf
+ .createWithDefault(false)
+
+ private[spark] val EVENT_LOG_TESTING =
+ ConfigBuilder("spark.eventLog.testing")
+ .internal()
+ .booleanConf
+ .createWithDefault(false)
+
+ private[spark] val EVENT_LOG_OUTPUT_BUFFER_SIZE = ConfigBuilder("spark.eventLog.buffer.kb")
+ .bytesConf(ByteUnit.KiB)
+ .createWithDefaultString("100k")
+
+ private[spark] val EVENT_LOG_OVERWRITE =
+ ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false)
+
private[spark] val EXECUTOR_CLASS_PATH =
ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional
@@ -72,6 +95,10 @@ package object config {
private[spark] val DYN_ALLOCATION_MAX_EXECUTORS =
ConfigBuilder("spark.dynamicAllocation.maxExecutors").intConf.createWithDefault(Int.MaxValue)
+ private[spark] val LOCALITY_WAIT = ConfigBuilder("spark.locality.wait")
+ .timeConf(TimeUnit.MILLISECONDS)
+ .createWithDefaultString("3s")
+
private[spark] val SHUFFLE_SERVICE_ENABLED =
ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false)
@@ -199,6 +226,11 @@ package object config {
private[spark] val HISTORY_UI_MAX_APPS =
ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE)
+ private[spark] val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress")
+ .doc("When true, show the progress bar in the console.")
+ .booleanConf
+ .createWithDefault(false)
+
private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled")
.booleanConf
.createWithDefault(false)
@@ -261,6 +293,13 @@ package object config {
.longConf
.createWithDefault(4 * 1024 * 1024)
+ private[spark] val HADOOP_RDD_IGNORE_EMPTY_SPLITS =
+ ConfigBuilder("spark.hadoopRDD.ignoreEmptySplits")
+ .internal()
+ .doc("When true, HadoopRDD/NewHadoopRDD will not create partitions for empty input splits.")
+ .booleanConf
+ .createWithDefault(false)
+
private[spark] val SECRET_REDACTION_PATTERN =
ConfigBuilder("spark.redaction.regex")
.doc("Regex to decide which Spark configuration properties and environment variables in " +
@@ -341,13 +380,15 @@ package object config {
.checkValue(_ > 0, "The max no. of blocks in flight cannot be non-positive.")
.createWithDefault(Int.MaxValue)
- private[spark] val REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM =
- ConfigBuilder("spark.reducer.maxReqSizeShuffleToMem")
- .doc("The blocks of a shuffle request will be fetched to disk when size of the request is " +
+ private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM =
+ ConfigBuilder("spark.maxRemoteBlockSizeFetchToMem")
+ .doc("Remote block will be fetched to disk when size of the block is " +
"above this threshold. This is to avoid a giant request takes too much memory. We can " +
- "enable this config by setting a specific value(e.g. 200m). Note that this config can " +
- "be enabled only when the shuffle shuffle service is newer than Spark-2.2 or the shuffle" +
- " service is disabled.")
+ "enable this config by setting a specific value(e.g. 200m). Note this configuration will " +
+ "affect both shuffle fetch and block manager remote block fetch. For users who " +
+ "enabled external shuffle service, this feature can only be worked when external shuffle" +
+ " service is newer than Spark 2.2.")
+ .withAlternative("spark.reducer.maxReqSizeShuffleToMem")
.bytesConf(ByteUnit.BYTE)
.createWithDefault(Long.MaxValue)
@@ -410,4 +451,28 @@ package object config {
.stringConf
.toSequence
.createWithDefault(Nil)
+
+ private[spark] val UI_X_XSS_PROTECTION =
+ ConfigBuilder("spark.ui.xXssProtection")
+ .doc("Value for HTTP X-XSS-Protection response header")
+ .stringConf
+ .createWithDefaultString("1; mode=block")
+
+ private[spark] val UI_X_CONTENT_TYPE_OPTIONS =
+ ConfigBuilder("spark.ui.xContentTypeOptions.enabled")
+ .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'")
+ .booleanConf
+ .createWithDefault(true)
+
+ private[spark] val UI_STRICT_TRANSPORT_SECURITY =
+ ConfigBuilder("spark.ui.strictTransportSecurity")
+ .doc("Value for HTTP Strict Transport Security Response Header")
+ .stringConf
+ .createOptional
+
+ private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners")
+ .doc("Class names of listeners to add to SparkContext during initialization.")
+ .stringConf
+ .toSequence
+ .createOptional
}
diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
index b1d07ab2c9199..95c99d29c3a9c 100644
--- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala
@@ -20,6 +20,7 @@ package org.apache.spark.internal.io
import java.util.{Date, UUID}
import scala.collection.mutable
+import scala.util.Try
import org.apache.hadoop.conf.Configurable
import org.apache.hadoop.fs.Path
@@ -35,6 +36,9 @@ import org.apache.spark.mapred.SparkHadoopMapRedUtil
* (from the newer mapreduce API, not the old mapred API).
*
* Unlike Hadoop's OutputCommitter, this implementation is serializable.
+ *
+ * @param jobId the job's or stage's id
+ * @param path the job's output path, or null if committer acts as a noop
*/
class HadoopMapReduceCommitProtocol(jobId: String, path: String)
extends FileCommitProtocol with Serializable with Logging {
@@ -44,6 +48,16 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
/** OutputCommitter from Hadoop is not serializable so marking it transient. */
@transient private var committer: OutputCommitter = _
+ /**
+ * Checks whether there are files to be committed to a valid output location.
+ *
+ * As committing and aborting a job occurs on driver, where `addedAbsPathFiles` is always null,
+ * it is necessary to check whether a valid output path is specified.
+ * [[HadoopMapReduceCommitProtocol#path]] need not be a valid [[org.apache.hadoop.fs.Path]] for
+ * committers not writing to distributed file systems.
+ */
+ private val hasValidPath = Try { new Path(path) }.isSuccess
+
/**
* Tracks files staged by this task for absolute output paths. These outputs are not managed by
* the Hadoop OutputCommitter, so we must move these to their final locations on job commit.
@@ -130,17 +144,21 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
val filesToMove = taskCommits.map(_.obj.asInstanceOf[Map[String, String]])
.foldLeft(Map[String, String]())(_ ++ _)
logDebug(s"Committing files staged for absolute locations $filesToMove")
- val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
- for ((src, dst) <- filesToMove) {
- fs.rename(new Path(src), new Path(dst))
+ if (hasValidPath) {
+ val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
+ for ((src, dst) <- filesToMove) {
+ fs.rename(new Path(src), new Path(dst))
+ }
+ fs.delete(absPathStagingDir, true)
}
- fs.delete(absPathStagingDir, true)
}
override def abortJob(jobContext: JobContext): Unit = {
committer.abortJob(jobContext, JobStatus.State.FAILED)
- val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
- fs.delete(absPathStagingDir, true)
+ if (hasValidPath) {
+ val fs = absPathStagingDir.getFileSystem(jobContext.getConfiguration)
+ fs.delete(absPathStagingDir, true)
+ }
}
override def setupTask(taskContext: TaskAttemptContext): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
index fe5fd2da039bb..1d8a266d0079c 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala
@@ -25,8 +25,8 @@ import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
import org.apache.spark.internal.Logging
-import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager}
+import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempFileManager}
import org.apache.spark.storage.{BlockId, StorageLevel}
import org.apache.spark.util.ThreadUtils
@@ -68,7 +68,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
- tempShuffleFileManager: TempShuffleFileManager): Unit
+ tempFileManager: TempFileManager): Unit
/**
* Upload a single block to a remote node, available only after [[init]] is invoked.
@@ -87,7 +87,12 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo
*
* It is also only available after [[init]] is invoked.
*/
- def fetchBlockSync(host: String, port: Int, execId: String, blockId: String): ManagedBuffer = {
+ def fetchBlockSync(
+ host: String,
+ port: Int,
+ execId: String,
+ blockId: String,
+ tempFileManager: TempFileManager): ManagedBuffer = {
// A monitor for the thread to wait on.
val result = Promise[ManagedBuffer]()
fetchBlocks(host, port, execId, Array(blockId),
@@ -96,12 +101,17 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo
result.failure(exception)
}
override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = {
- val ret = ByteBuffer.allocate(data.size.toInt)
- ret.put(data.nioByteBuffer())
- ret.flip()
- result.success(new NioManagedBuffer(ret))
+ data match {
+ case f: FileSegmentManagedBuffer =>
+ result.success(f)
+ case _ =>
+ val ret = ByteBuffer.allocate(data.size.toInt)
+ ret.put(data.nioByteBuffer())
+ ret.flip()
+ result.success(new NioManagedBuffer(ret))
+ }
}
- }, tempShuffleFileManager = null)
+ }, tempFileManager)
ThreadUtils.awaitResult(result.future, Duration.Inf)
}
diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
index ac4d85004bad1..b7d8c35032763 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala
@@ -32,7 +32,7 @@ import org.apache.spark.network.buffer.ManagedBuffer
import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap, TransportClientFactory}
import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap}
import org.apache.spark.network.server._
-import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher, RetryingBlockFetcher, TempShuffleFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher, RetryingBlockFetcher, TempFileManager}
import org.apache.spark.network.shuffle.protocol.UploadBlock
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.serializer.JavaSerializer
@@ -105,14 +105,14 @@ private[spark] class NettyBlockTransferService(
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
- tempShuffleFileManager: TempShuffleFileManager): Unit = {
+ tempFileManager: TempFileManager): Unit = {
logTrace(s"Fetch blocks from $host:$port (executor id $execId)")
try {
val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter {
override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) {
val client = clientFactory.createClient(host, port)
new OneForOneBlockFetcher(client, appId, execId, blockIds, listener,
- transportConf, tempShuffleFileManager).start()
+ transportConf, tempFileManager).start()
}
}
@@ -151,7 +151,7 @@ private[spark] class NettyBlockTransferService(
// Convert or copy nio buffer into array in order to serialize it.
val array = JavaUtils.bufferToArray(blockData.nioByteBuffer())
- client.sendRpc(new UploadBlock(appId, execId, blockId.toString, metadata, array).toByteBuffer,
+ client.sendRpc(new UploadBlock(appId, execId, blockId.name, metadata, array).toByteBuffer,
new RpcResponseCallback {
override def onSuccess(response: ByteBuffer): Unit = {
logTrace(s"Successfully uploaded block $blockId")
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 76ea8b86c53d2..2480559a41b7a 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -35,7 +35,7 @@ import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES
+import org.apache.spark.internal.config._
import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD
import org.apache.spark.scheduler.{HDFSCacheTaskLocation, HostTaskLocation}
import org.apache.spark.storage.StorageLevel
@@ -134,6 +134,8 @@ class HadoopRDD[K, V](
private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES)
+ private val ignoreEmptySplits = sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS)
+
// Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads.
protected def getJobConf(): JobConf = {
val conf: Configuration = broadcastedConf.value.value
@@ -157,20 +159,25 @@ class HadoopRDD[K, V](
if (conf.isInstanceOf[JobConf]) {
logDebug("Re-using user-broadcasted JobConf")
conf.asInstanceOf[JobConf]
- } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) {
- logDebug("Re-using cached JobConf")
- HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf]
} else {
- // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the
- // local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
- // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects.
- // Synchronize to prevent ConcurrentModificationException (SPARK-1097, HADOOP-10456).
- HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized {
- logDebug("Creating new JobConf and caching it for later re-use")
- val newJobConf = new JobConf(conf)
- initLocalJobConfFuncOpt.foreach(f => f(newJobConf))
- HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
- newJobConf
+ Option(HadoopRDD.getCachedMetadata(jobConfCacheKey))
+ .map { conf =>
+ logDebug("Re-using cached JobConf")
+ conf.asInstanceOf[JobConf]
+ }
+ .getOrElse {
+ // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in
+ // the local process. The local cache is accessed through HadoopRDD.putCachedMetadata().
+ // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary
+ // objects. Synchronize to prevent ConcurrentModificationException (SPARK-1097,
+ // HADOOP-10456).
+ HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized {
+ logDebug("Creating new JobConf and caching it for later re-use")
+ val newJobConf = new JobConf(conf)
+ initLocalJobConfFuncOpt.foreach(f => f(newJobConf))
+ HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf)
+ newJobConf
+ }
}
}
}
@@ -190,8 +197,12 @@ class HadoopRDD[K, V](
val jobConf = getJobConf()
// add the credentials here as this can be called before SparkContext initialized
SparkHadoopUtil.get.addCredentials(jobConf)
- val inputFormat = getInputFormat(jobConf)
- val inputSplits = inputFormat.getSplits(jobConf, minPartitions)
+ val allInputSplits = getInputFormat(jobConf).getSplits(jobConf, minPartitions)
+ val inputSplits = if (ignoreEmptySplits) {
+ allInputSplits.filter(_.getLength > 0)
+ } else {
+ allInputSplits
+ }
val array = new Array[Partition](inputSplits.size)
for (i <- 0 until inputSplits.size) {
array(i) = new HadoopPartition(id, i, inputSplits(i))
@@ -360,8 +371,6 @@ private[spark] object HadoopRDD extends Logging {
*/
def getCachedMetadata(key: String): Any = SparkEnv.get.hadoopJobMetadata.get(key)
- def containsCachedMetadata(key: String): Boolean = SparkEnv.get.hadoopJobMetadata.containsKey(key)
-
private def putCachedMetadata(key: String, value: Any): Unit =
SparkEnv.get.hadoopJobMetadata.put(key, value)
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index 482875e6c1ac5..e4dd1b6a82498 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -21,6 +21,7 @@ import java.io.IOException
import java.text.SimpleDateFormat
import java.util.{Date, Locale}
+import scala.collection.JavaConverters.asScalaBufferConverter
import scala.reflect.ClassTag
import org.apache.hadoop.conf.{Configurable, Configuration}
@@ -34,7 +35,7 @@ import org.apache.spark._
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES
+import org.apache.spark.internal.config._
import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.{SerializableConfiguration, ShutdownHookManager}
@@ -89,6 +90,8 @@ class NewHadoopRDD[K, V](
private val ignoreCorruptFiles = sparkContext.conf.get(IGNORE_CORRUPT_FILES)
+ private val ignoreEmptySplits = sparkContext.conf.get(HADOOP_RDD_IGNORE_EMPTY_SPLITS)
+
def getConf: Configuration = {
val conf: Configuration = confBroadcast.value.value
if (shouldCloneJobConf) {
@@ -121,8 +124,12 @@ class NewHadoopRDD[K, V](
configurable.setConf(_conf)
case _ =>
}
- val jobContext = new JobContextImpl(_conf, jobId)
- val rawSplits = inputFormat.getSplits(jobContext).toArray
+ val allRowSplits = inputFormat.getSplits(new JobContextImpl(_conf, jobId)).asScala
+ val rawSplits = if (ignoreEmptySplits) {
+ allRowSplits.filter(_.getLength > 0)
+ } else {
+ allRowSplits
+ }
val result = new Array[Partition](rawSplits.size)
for (i <- 0 until rawSplits.size) {
result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable])
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala
deleted file mode 100644
index 6da8865cd10d3..0000000000000
--- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala
+++ /dev/null
@@ -1,67 +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
-
-/**
- * A simple listener for application events.
- *
- * This listener expects to hear events from a single application only. If events
- * from multiple applications are seen, the behavior is unspecified.
- */
-private[spark] class ApplicationEventListener extends SparkListener {
- var appName: Option[String] = None
- var appId: Option[String] = None
- var appAttemptId: Option[String] = None
- var sparkUser: Option[String] = None
- var startTime: Option[Long] = None
- var endTime: Option[Long] = None
- var viewAcls: Option[String] = None
- var adminAcls: Option[String] = None
- var viewAclsGroups: Option[String] = None
- var adminAclsGroups: Option[String] = None
- var appSparkVersion: Option[String] = None
-
- override def onApplicationStart(applicationStart: SparkListenerApplicationStart) {
- appName = Some(applicationStart.appName)
- appId = applicationStart.appId
- appAttemptId = applicationStart.appAttemptId
- startTime = Some(applicationStart.time)
- sparkUser = Some(applicationStart.sparkUser)
- }
-
- override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) {
- endTime = Some(applicationEnd.time)
- }
-
- override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) {
- synchronized {
- val environmentDetails = environmentUpdate.environmentDetails
- val allProperties = environmentDetails("Spark Properties").toMap
- viewAcls = allProperties.get("spark.ui.view.acls")
- adminAcls = allProperties.get("spark.admin.acls")
- viewAclsGroups = allProperties.get("spark.ui.view.acls.groups")
- adminAclsGroups = allProperties.get("spark.admin.acls.groups")
- }
- }
-
- override def onOtherEvent(event: SparkListenerEvent): Unit = event match {
- case SparkListenerLogStart(sparkVersion) =>
- appSparkVersion = Some(sparkVersion)
- case _ =>
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
index 9dafa0b7646bf..a77adc5ff3545 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -37,6 +37,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.{SPARK_VERSION, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
import org.apache.spark.io.CompressionCodec
import org.apache.spark.util.{JsonProtocol, Utils}
@@ -45,6 +46,7 @@ import org.apache.spark.util.{JsonProtocol, Utils}
*
* Event logging is specified by the following configurable parameters:
* spark.eventLog.enabled - Whether event logging is enabled.
+ * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates
* spark.eventLog.compress - Whether to compress logged events
* spark.eventLog.overwrite - Whether to overwrite any existing files.
* spark.eventLog.dir - Path to the directory in which events are logged.
@@ -64,10 +66,11 @@ private[spark] class EventLoggingListener(
this(appId, appAttemptId, logBaseDir, sparkConf,
SparkHadoopUtil.get.newConfiguration(sparkConf))
- private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false)
- private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false)
- private val testing = sparkConf.getBoolean("spark.eventLog.testing", false)
- private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024
+ private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS)
+ private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE)
+ private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES)
+ private val testing = sparkConf.get(EVENT_LOG_TESTING)
+ private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt
private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf)
private val compressionCodec =
if (shouldCompress) {
@@ -216,8 +219,11 @@ private[spark] class EventLoggingListener(
logEvent(event, flushLogger = true)
}
- // No-op because logging every update would be overkill
- override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {}
+ override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {
+ if (shouldLogBlockUpdates) {
+ logEvent(event, flushLogger = true)
+ }
+ }
// No-op because logging every update would be overkill
override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
index e815b7e0cf6c9..233781f3d9719 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetBlacklist.scala
@@ -61,6 +61,16 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
private val blacklistedExecs = new HashSet[String]()
private val blacklistedNodes = new HashSet[String]()
+ private var latestFailureReason: String = null
+
+ /**
+ * Get the most recent failure reason of this TaskSet.
+ * @return
+ */
+ def getLatestFailureReason: String = {
+ latestFailureReason
+ }
+
/**
* Return true if this executor is blacklisted for the given task. This does *not*
* need to return true if the executor is blacklisted for the entire stage, or blacklisted
@@ -94,7 +104,9 @@ private[scheduler] class TaskSetBlacklist(val conf: SparkConf, val stageId: Int,
private[scheduler] def updateBlacklistForFailedTask(
host: String,
exec: String,
- index: Int): Unit = {
+ index: Int,
+ failureReason: String): Unit = {
+ latestFailureReason = failureReason
val execFailures = execToFailures.getOrElseUpdate(exec, new ExecutorFailuresInTaskSet(host))
execFailures.updateWithFailure(index, clock.getTimeMillis())
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 3804ea863b4f9..de4711f461df2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -27,7 +27,7 @@ import scala.util.control.NonFatal
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.scheduler.SchedulingMode._
import org.apache.spark.util.{AccumulatorV2, Clock, SystemClock, Utils}
import org.apache.spark.util.collection.MedianHeap
@@ -83,6 +83,11 @@ private[spark] class TaskSetManager(
val successful = new Array[Boolean](numTasks)
private val numFailures = new Array[Int](numTasks)
+ // Set the coresponding index of Boolean var when the task killed by other attempt tasks,
+ // this happened while we set the `spark.speculation` to true. The task killed by others
+ // should not resubmit while executor lost.
+ private val killedByOtherAttempt: Array[Boolean] = new Array[Boolean](numTasks)
+
val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
private[scheduler] var tasksSuccessful = 0
@@ -670,9 +675,14 @@ private[spark] class TaskSetManager(
}
if (blacklistedEverywhere) {
val partition = tasks(indexInTaskSet).partitionId
- abort(s"Aborting $taskSet because task $indexInTaskSet (partition $partition) " +
- s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior " +
- s"can be configured via spark.blacklist.*.")
+ abort(s"""
+ |Aborting $taskSet because task $indexInTaskSet (partition $partition)
+ |cannot run anywhere due to node and executor blacklist.
+ |Most recent failure:
+ |${taskSetBlacklist.getLatestFailureReason}
+ |
+ |Blacklisting behavior can be configured via spark.blacklist.*.
+ |""".stripMargin)
}
}
}
@@ -724,6 +734,7 @@ private[spark] class TaskSetManager(
logInfo(s"Killing attempt ${attemptInfo.attemptNumber} for task ${attemptInfo.id} " +
s"in stage ${taskSet.id} (TID ${attemptInfo.taskId}) on ${attemptInfo.host} " +
s"as the attempt ${info.attemptNumber} succeeded on ${info.host}")
+ killedByOtherAttempt(index) = true
sched.backend.killTask(
attemptInfo.taskId,
attemptInfo.executorId,
@@ -837,9 +848,9 @@ private[spark] class TaskSetManager(
sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info)
if (!isZombie && reason.countTowardsTaskFailures) {
- taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask(
- info.host, info.executorId, index))
assert (null != failureReason)
+ taskSetBlacklistHelperOpt.foreach(_.updateBlacklistForFailedTask(
+ info.host, info.executorId, index, failureReason))
numFailures(index) += 1
if (numFailures(index) >= maxTaskFailures) {
logError("Task %d in stage %s failed %d times; aborting job".format(
@@ -910,7 +921,7 @@ private[spark] class TaskSetManager(
&& !isZombie) {
for ((tid, info) <- taskInfos if info.executorId == execId) {
val index = taskInfos(tid).index
- if (successful(index)) {
+ if (successful(index) && !killedByOtherAttempt(index)) {
successful(index) = false
copiesRunning(index) -= 1
tasksSuccessful -= 1
@@ -975,7 +986,7 @@ private[spark] class TaskSetManager(
}
private def getLocalityWait(level: TaskLocality.TaskLocality): Long = {
- val defaultWait = conf.get("spark.locality.wait", "3s")
+ val defaultWait = conf.get(config.LOCALITY_WAIT)
val localityWaitKey = level match {
case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process"
case TaskLocality.NODE_LOCAL => "spark.locality.wait.node"
@@ -984,7 +995,7 @@ private[spark] class TaskSetManager(
}
if (localityWaitKey != null) {
- conf.getTimeAsMs(localityWaitKey, defaultWait)
+ conf.getTimeAsMs(localityWaitKey, defaultWait.toString)
} else {
0L
}
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 a4e2a74341283..505c342a889ee 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
@@ -153,7 +153,7 @@ private[spark] class StandaloneSchedulerBackend(
override def executorAdded(fullId: String, workerId: String, hostPort: String, cores: Int,
memory: Int) {
- logInfo("Granted executor ID %s on hostPort %s with %d cores, %s RAM".format(
+ logInfo("Granted executor ID %s on hostPort %s with %d core(s), %s RAM".format(
fullId, hostPort, cores, Utils.megabytesToString(memory)))
}
diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
index c8d1460300934..0562d45ff57c5 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -52,7 +52,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024,
SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue),
SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS),
- SparkEnv.get.conf.get(config.REDUCER_MAX_REQ_SIZE_SHUFFLE_TO_MEM),
+ SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM),
SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true))
val serializerInstance = dep.serializer.newInstance()
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
new file mode 100644
index 0000000000000..658ae11671c27
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -0,0 +1,566 @@
+/*
+ * 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.status
+
+import java.util.Date
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark._
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.internal.Logging
+import org.apache.spark.scheduler._
+import org.apache.spark.status.api.v1
+import org.apache.spark.storage._
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.kvstore.KVStore
+
+/**
+ * A Spark listener that writes application information to a data store. The types written to the
+ * store are defined in the `storeTypes.scala` file and are based on the public REST API.
+ */
+private[spark] class AppStatusListener(
+ kvstore: KVStore,
+ conf: SparkConf,
+ live: Boolean) extends SparkListener with Logging {
+
+ import config._
+
+ private var sparkVersion = SPARK_VERSION
+ private var appInfo: v1.ApplicationInfo = null
+ private var coresPerTask: Int = 1
+
+ // How often to update live entities. -1 means "never update" when replaying applications,
+ // meaning only the last write will happen. For live applications, this avoids a few
+ // operations that we can live without when rapidly processing incoming task events.
+ private val liveUpdatePeriodNs = if (live) conf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L
+
+ // Keep track of live entities, so that task metrics can be efficiently updated (without
+ // causing too many writes to the underlying store, and other expensive operations).
+ private val liveStages = new HashMap[(Int, Int), LiveStage]()
+ private val liveJobs = new HashMap[Int, LiveJob]()
+ private val liveExecutors = new HashMap[String, LiveExecutor]()
+ private val liveTasks = new HashMap[Long, LiveTask]()
+ private val liveRDDs = new HashMap[Int, LiveRDD]()
+
+ override def onOtherEvent(event: SparkListenerEvent): Unit = event match {
+ case SparkListenerLogStart(version) => sparkVersion = version
+ case _ =>
+ }
+
+ override def onApplicationStart(event: SparkListenerApplicationStart): Unit = {
+ assert(event.appId.isDefined, "Application without IDs are not supported.")
+
+ val attempt = new v1.ApplicationAttemptInfo(
+ event.appAttemptId,
+ new Date(event.time),
+ new Date(-1),
+ new Date(event.time),
+ -1L,
+ event.sparkUser,
+ false,
+ sparkVersion)
+
+ appInfo = new v1.ApplicationInfo(
+ event.appId.get,
+ event.appName,
+ None,
+ None,
+ None,
+ None,
+ Seq(attempt))
+
+ kvstore.write(new ApplicationInfoWrapper(appInfo))
+ }
+
+ override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = {
+ val old = appInfo.attempts.head
+ val attempt = new v1.ApplicationAttemptInfo(
+ old.attemptId,
+ old.startTime,
+ new Date(event.time),
+ new Date(event.time),
+ event.time - old.startTime.getTime(),
+ old.sparkUser,
+ true,
+ old.appSparkVersion)
+
+ appInfo = new v1.ApplicationInfo(
+ appInfo.id,
+ appInfo.name,
+ None,
+ None,
+ None,
+ None,
+ Seq(attempt))
+ kvstore.write(new ApplicationInfoWrapper(appInfo))
+ }
+
+ override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = {
+ // This needs to be an update in case an executor re-registers after the driver has
+ // marked it as "dead".
+ val exec = getOrCreateExecutor(event.executorId)
+ exec.host = event.executorInfo.executorHost
+ exec.isActive = true
+ exec.totalCores = event.executorInfo.totalCores
+ exec.maxTasks = event.executorInfo.totalCores / coresPerTask
+ exec.executorLogs = event.executorInfo.logUrlMap
+ liveUpdate(exec)
+ }
+
+ override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = {
+ liveExecutors.remove(event.executorId).foreach { exec =>
+ exec.isActive = false
+ update(exec)
+ }
+ }
+
+ override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = {
+ updateBlackListStatus(event.executorId, true)
+ }
+
+ override def onExecutorUnblacklisted(event: SparkListenerExecutorUnblacklisted): Unit = {
+ updateBlackListStatus(event.executorId, false)
+ }
+
+ override def onNodeBlacklisted(event: SparkListenerNodeBlacklisted): Unit = {
+ updateNodeBlackList(event.hostId, true)
+ }
+
+ override def onNodeUnblacklisted(event: SparkListenerNodeUnblacklisted): Unit = {
+ updateNodeBlackList(event.hostId, false)
+ }
+
+ private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = {
+ liveExecutors.get(execId).foreach { exec =>
+ exec.isBlacklisted = blacklisted
+ liveUpdate(exec)
+ }
+ }
+
+ private def updateNodeBlackList(host: String, blacklisted: Boolean): Unit = {
+ // Implicitly (un)blacklist every executor associated with the node.
+ liveExecutors.values.foreach { exec =>
+ if (exec.hostname == host) {
+ exec.isBlacklisted = blacklisted
+ liveUpdate(exec)
+ }
+ }
+ }
+
+ override def onJobStart(event: SparkListenerJobStart): Unit = {
+ // Compute (a potential over-estimate of) the number of tasks that will be run by this job.
+ // This may be an over-estimate because the job start event references all of the result
+ // stages' transitive stage dependencies, but some of these stages might be skipped if their
+ // output is available from earlier runs.
+ // See https://github.com/apache/spark/pull/3009 for a more extensive discussion.
+ val numTasks = {
+ val missingStages = event.stageInfos.filter(_.completionTime.isEmpty)
+ missingStages.map(_.numTasks).sum
+ }
+
+ val lastStageInfo = event.stageInfos.lastOption
+ val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)")
+
+ val jobGroup = Option(event.properties)
+ .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) }
+
+ val job = new LiveJob(
+ event.jobId,
+ lastStageName,
+ Some(new Date(event.time)),
+ event.stageIds,
+ jobGroup,
+ numTasks)
+ liveJobs.put(event.jobId, job)
+ liveUpdate(job)
+
+ event.stageInfos.foreach { stageInfo =>
+ // A new job submission may re-use an existing stage, so this code needs to do an update
+ // instead of just a write.
+ val stage = getOrCreateStage(stageInfo)
+ stage.jobs :+= job
+ stage.jobIds += event.jobId
+ liveUpdate(stage)
+ }
+ }
+
+ override def onJobEnd(event: SparkListenerJobEnd): Unit = {
+ liveJobs.remove(event.jobId).foreach { job =>
+ job.status = event.jobResult match {
+ case JobSucceeded => JobExecutionStatus.SUCCEEDED
+ case JobFailed(_) => JobExecutionStatus.FAILED
+ }
+
+ job.completionTime = Some(new Date(event.time))
+ update(job)
+ }
+ }
+
+ override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = {
+ val stage = getOrCreateStage(event.stageInfo)
+ stage.status = v1.StageStatus.ACTIVE
+ stage.schedulingPool = Option(event.properties).flatMap { p =>
+ Option(p.getProperty("spark.scheduler.pool"))
+ }.getOrElse(SparkUI.DEFAULT_POOL_NAME)
+
+ // Look at all active jobs to find the ones that mention this stage.
+ stage.jobs = liveJobs.values
+ .filter(_.stageIds.contains(event.stageInfo.stageId))
+ .toSeq
+ stage.jobIds = stage.jobs.map(_.jobId).toSet
+
+ stage.jobs.foreach { job =>
+ job.completedStages = job.completedStages - event.stageInfo.stageId
+ job.activeStages += 1
+ liveUpdate(job)
+ }
+
+ event.stageInfo.rddInfos.foreach { info =>
+ if (info.storageLevel.isValid) {
+ liveUpdate(liveRDDs.getOrElseUpdate(info.id, new LiveRDD(info)))
+ }
+ }
+
+ liveUpdate(stage)
+ }
+
+ override def onTaskStart(event: SparkListenerTaskStart): Unit = {
+ val task = new LiveTask(event.taskInfo, event.stageId, event.stageAttemptId)
+ liveTasks.put(event.taskInfo.taskId, task)
+ liveUpdate(task)
+
+ liveStages.get((event.stageId, event.stageAttemptId)).foreach { stage =>
+ stage.activeTasks += 1
+ stage.firstLaunchTime = math.min(stage.firstLaunchTime, event.taskInfo.launchTime)
+ liveUpdate(stage)
+
+ stage.jobs.foreach { job =>
+ job.activeTasks += 1
+ liveUpdate(job)
+ }
+ }
+
+ liveExecutors.get(event.taskInfo.executorId).foreach { exec =>
+ exec.activeTasks += 1
+ exec.totalTasks += 1
+ liveUpdate(exec)
+ }
+ }
+
+ override def onTaskGettingResult(event: SparkListenerTaskGettingResult): Unit = {
+ // Call update on the task so that the "getting result" time is written to the store; the
+ // value is part of the mutable TaskInfo state that the live entity already references.
+ liveTasks.get(event.taskInfo.taskId).foreach { task =>
+ maybeUpdate(task)
+ }
+ }
+
+ override def onTaskEnd(event: SparkListenerTaskEnd): Unit = {
+ // TODO: can this really happen?
+ if (event.taskInfo == null) {
+ return
+ }
+
+ val metricsDelta = liveTasks.remove(event.taskInfo.taskId).map { task =>
+ val errorMessage = event.reason match {
+ case Success =>
+ None
+ case k: TaskKilled =>
+ Some(k.reason)
+ case e: ExceptionFailure => // Handle ExceptionFailure because we might have accumUpdates
+ Some(e.toErrorString)
+ case e: TaskFailedReason => // All other failure cases
+ Some(e.toErrorString)
+ case other =>
+ logInfo(s"Unhandled task end reason: $other")
+ None
+ }
+ task.errorMessage = errorMessage
+ val delta = task.updateMetrics(event.taskMetrics)
+ update(task)
+ delta
+ }.orNull
+
+ val (completedDelta, failedDelta) = event.reason match {
+ case Success =>
+ (1, 0)
+ case _ =>
+ (0, 1)
+ }
+
+ liveStages.get((event.stageId, event.stageAttemptId)).foreach { stage =>
+ if (metricsDelta != null) {
+ stage.metrics.update(metricsDelta)
+ }
+ stage.activeTasks -= 1
+ stage.completedTasks += completedDelta
+ stage.failedTasks += failedDelta
+ liveUpdate(stage)
+
+ stage.jobs.foreach { job =>
+ job.activeTasks -= 1
+ job.completedTasks += completedDelta
+ job.failedTasks += failedDelta
+ liveUpdate(job)
+ }
+
+ val esummary = stage.executorSummary(event.taskInfo.executorId)
+ esummary.taskTime += event.taskInfo.duration
+ esummary.succeededTasks += completedDelta
+ esummary.failedTasks += failedDelta
+ if (metricsDelta != null) {
+ esummary.metrics.update(metricsDelta)
+ }
+ liveUpdate(esummary)
+ }
+
+ liveExecutors.get(event.taskInfo.executorId).foreach { exec =>
+ if (event.taskMetrics != null) {
+ val readMetrics = event.taskMetrics.shuffleReadMetrics
+ exec.totalGcTime += event.taskMetrics.jvmGCTime
+ exec.totalInputBytes += event.taskMetrics.inputMetrics.bytesRead
+ exec.totalShuffleRead += readMetrics.localBytesRead + readMetrics.remoteBytesRead
+ exec.totalShuffleWrite += event.taskMetrics.shuffleWriteMetrics.bytesWritten
+ }
+
+ exec.activeTasks -= 1
+ exec.completedTasks += completedDelta
+ exec.failedTasks += failedDelta
+ exec.totalDuration += event.taskInfo.duration
+ liveUpdate(exec)
+ }
+ }
+
+ override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
+ liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptId)).foreach { stage =>
+ stage.info = event.stageInfo
+
+ // Because of SPARK-20205, old event logs may contain valid stages without a submission time
+ // in their start event. In those cases, we can only detect whether a stage was skipped by
+ // waiting until the completion event, at which point the field would have been set.
+ stage.status = event.stageInfo.failureReason match {
+ case Some(_) => v1.StageStatus.FAILED
+ case _ if event.stageInfo.submissionTime.isDefined => v1.StageStatus.COMPLETE
+ case _ => v1.StageStatus.SKIPPED
+ }
+
+ stage.jobs.foreach { job =>
+ stage.status match {
+ case v1.StageStatus.COMPLETE =>
+ job.completedStages += event.stageInfo.stageId
+ case v1.StageStatus.SKIPPED =>
+ job.skippedStages += event.stageInfo.stageId
+ job.skippedTasks += event.stageInfo.numTasks
+ case _ =>
+ job.failedStages += 1
+ }
+ job.activeStages -= 1
+ liveUpdate(job)
+ }
+
+ stage.executorSummaries.values.foreach(update)
+ update(stage)
+ }
+ }
+
+ override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded): Unit = {
+ // This needs to set fields that are already set by onExecutorAdded because the driver is
+ // considered an "executor" in the UI, but does not have a SparkListenerExecutorAdded event.
+ val exec = getOrCreateExecutor(event.blockManagerId.executorId)
+ exec.hostPort = event.blockManagerId.hostPort
+ event.maxOnHeapMem.foreach { _ =>
+ exec.totalOnHeap = event.maxOnHeapMem.get
+ exec.totalOffHeap = event.maxOffHeapMem.get
+ }
+ exec.isActive = true
+ exec.maxMemory = event.maxMem
+ liveUpdate(exec)
+ }
+
+ override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = {
+ // Nothing to do here. Covered by onExecutorRemoved.
+ }
+
+ override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = {
+ liveRDDs.remove(event.rddId)
+ kvstore.delete(classOf[RDDStorageInfoWrapper], event.rddId)
+ }
+
+ override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = {
+ event.accumUpdates.foreach { case (taskId, sid, sAttempt, accumUpdates) =>
+ liveTasks.get(taskId).foreach { task =>
+ val metrics = TaskMetrics.fromAccumulatorInfos(accumUpdates)
+ val delta = task.updateMetrics(metrics)
+ maybeUpdate(task)
+
+ liveStages.get((sid, sAttempt)).foreach { stage =>
+ stage.metrics.update(delta)
+ maybeUpdate(stage)
+
+ val esummary = stage.executorSummary(event.execId)
+ esummary.metrics.update(delta)
+ maybeUpdate(esummary)
+ }
+ }
+ }
+ }
+
+ override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {
+ event.blockUpdatedInfo.blockId match {
+ case block: RDDBlockId => updateRDDBlock(event, block)
+ case _ => // TODO: API only covers RDD storage.
+ }
+ }
+
+ /** Flush all live entities' data to the underlying store. */
+ def flush(): Unit = {
+ liveStages.values.foreach(update)
+ liveJobs.values.foreach(update)
+ liveExecutors.values.foreach(update)
+ liveTasks.values.foreach(update)
+ liveRDDs.values.foreach(update)
+ }
+
+ private def updateRDDBlock(event: SparkListenerBlockUpdated, block: RDDBlockId): Unit = {
+ val executorId = event.blockUpdatedInfo.blockManagerId.executorId
+
+ // Whether values are being added to or removed from the existing accounting.
+ val storageLevel = event.blockUpdatedInfo.storageLevel
+ val diskDelta = event.blockUpdatedInfo.diskSize * (if (storageLevel.useDisk) 1 else -1)
+ val memoryDelta = event.blockUpdatedInfo.memSize * (if (storageLevel.useMemory) 1 else -1)
+
+ // Function to apply a delta to a value, but ensure that it doesn't go negative.
+ def newValue(old: Long, delta: Long): Long = math.max(0, old + delta)
+
+ val updatedStorageLevel = if (storageLevel.isValid) {
+ Some(storageLevel.description)
+ } else {
+ None
+ }
+
+ // We need information about the executor to update some memory accounting values in the
+ // RDD info, so read that beforehand.
+ val maybeExec = liveExecutors.get(executorId)
+ var rddBlocksDelta = 0
+
+ // Update the block entry in the RDD info, keeping track of the deltas above so that we
+ // can update the executor information too.
+ liveRDDs.get(block.rddId).foreach { rdd =>
+ val partition = rdd.partition(block.name)
+
+ val executors = if (updatedStorageLevel.isDefined) {
+ if (!partition.executors.contains(executorId)) {
+ rddBlocksDelta = 1
+ }
+ partition.executors + executorId
+ } else {
+ rddBlocksDelta = -1
+ partition.executors - executorId
+ }
+
+ // Only update the partition if it's still stored in some executor, otherwise get rid of it.
+ if (executors.nonEmpty) {
+ if (updatedStorageLevel.isDefined) {
+ partition.storageLevel = updatedStorageLevel.get
+ }
+ partition.memoryUsed = newValue(partition.memoryUsed, memoryDelta)
+ partition.diskUsed = newValue(partition.diskUsed, diskDelta)
+ partition.executors = executors
+ } else {
+ rdd.removePartition(block.name)
+ }
+
+ maybeExec.foreach { exec =>
+ if (exec.rddBlocks + rddBlocksDelta > 0) {
+ val dist = rdd.distribution(exec)
+ dist.memoryRemaining = newValue(dist.memoryRemaining, -memoryDelta)
+ dist.memoryUsed = newValue(dist.memoryUsed, memoryDelta)
+ dist.diskUsed = newValue(dist.diskUsed, diskDelta)
+
+ if (exec.hasMemoryInfo) {
+ if (storageLevel.useOffHeap) {
+ dist.offHeapUsed = newValue(dist.offHeapUsed, memoryDelta)
+ dist.offHeapRemaining = newValue(dist.offHeapRemaining, -memoryDelta)
+ } else {
+ dist.onHeapUsed = newValue(dist.onHeapUsed, memoryDelta)
+ dist.onHeapRemaining = newValue(dist.onHeapRemaining, -memoryDelta)
+ }
+ }
+ } else {
+ rdd.removeDistribution(exec)
+ }
+ }
+
+ if (updatedStorageLevel.isDefined) {
+ rdd.storageLevel = updatedStorageLevel.get
+ }
+ rdd.memoryUsed = newValue(rdd.memoryUsed, memoryDelta)
+ rdd.diskUsed = newValue(rdd.diskUsed, diskDelta)
+ update(rdd)
+ }
+
+ maybeExec.foreach { exec =>
+ if (exec.hasMemoryInfo) {
+ if (storageLevel.useOffHeap) {
+ exec.usedOffHeap = newValue(exec.usedOffHeap, memoryDelta)
+ } else {
+ exec.usedOnHeap = newValue(exec.usedOnHeap, memoryDelta)
+ }
+ }
+ exec.memoryUsed = newValue(exec.memoryUsed, memoryDelta)
+ exec.diskUsed = newValue(exec.diskUsed, diskDelta)
+ exec.rddBlocks += rddBlocksDelta
+ if (exec.hasMemoryInfo || rddBlocksDelta != 0) {
+ liveUpdate(exec)
+ }
+ }
+ }
+
+ private def getOrCreateExecutor(executorId: String): LiveExecutor = {
+ liveExecutors.getOrElseUpdate(executorId, new LiveExecutor(executorId))
+ }
+
+ private def getOrCreateStage(info: StageInfo): LiveStage = {
+ val stage = liveStages.getOrElseUpdate((info.stageId, info.attemptId), new LiveStage())
+ stage.info = info
+ stage
+ }
+
+ private def update(entity: LiveEntity): Unit = {
+ entity.write(kvstore)
+ }
+
+ /** Update a live entity only if it hasn't been updated in the last configured period. */
+ private def maybeUpdate(entity: LiveEntity): Unit = {
+ if (liveUpdatePeriodNs >= 0) {
+ val now = System.nanoTime()
+ if (now - entity.lastWriteTime > liveUpdatePeriodNs) {
+ update(entity)
+ }
+ }
+ }
+
+ /** Update an entity only if in a live app; avoids redundant writes when replaying logs. */
+ private def liveUpdate(entity: LiveEntity): Unit = {
+ if (live) {
+ update(entity)
+ }
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
new file mode 100644
index 0000000000000..2927a3227cbef
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -0,0 +1,239 @@
+/*
+ * 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.status
+
+import java.io.File
+import java.util.{Arrays, List => JList}
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.{JobExecutionStatus, SparkConf}
+import org.apache.spark.scheduler.LiveListenerBus
+import org.apache.spark.status.api.v1
+import org.apache.spark.util.{Distribution, Utils}
+import org.apache.spark.util.kvstore.{InMemoryStore, KVStore}
+
+/**
+ * A wrapper around a KVStore that provides methods for accessing the API data stored within.
+ */
+private[spark] class AppStatusStore(store: KVStore) {
+
+ def jobsList(statuses: JList[JobExecutionStatus]): Seq[v1.JobData] = {
+ val it = store.view(classOf[JobDataWrapper]).asScala.map(_.info)
+ if (!statuses.isEmpty()) {
+ it.filter { job => statuses.contains(job.status) }.toSeq
+ } else {
+ it.toSeq
+ }
+ }
+
+ def job(jobId: Int): v1.JobData = {
+ store.read(classOf[JobDataWrapper], jobId).info
+ }
+
+ def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = {
+ store.view(classOf[ExecutorSummaryWrapper]).index("active").reverse().first(true)
+ .last(true).asScala.map(_.info).toSeq
+ }
+
+ def stageList(statuses: JList[v1.StageStatus]): Seq[v1.StageData] = {
+ val it = store.view(classOf[StageDataWrapper]).asScala.map(_.info)
+ if (!statuses.isEmpty) {
+ it.filter { s => statuses.contains(s.status) }.toSeq
+ } else {
+ it.toSeq
+ }
+ }
+
+ def stageData(stageId: Int): Seq[v1.StageData] = {
+ store.view(classOf[StageDataWrapper]).index("stageId").first(stageId).last(stageId)
+ .asScala.map(_.info).toSeq
+ }
+
+ def stageAttempt(stageId: Int, stageAttemptId: Int): v1.StageData = {
+ store.read(classOf[StageDataWrapper], Array(stageId, stageAttemptId)).info
+ }
+
+ def taskSummary(
+ stageId: Int,
+ stageAttemptId: Int,
+ quantiles: Array[Double]): v1.TaskMetricDistributions = {
+
+ val stage = Array(stageId, stageAttemptId)
+
+ val rawMetrics = store.view(classOf[TaskDataWrapper])
+ .index("stage")
+ .first(stage)
+ .last(stage)
+ .asScala
+ .flatMap(_.info.taskMetrics)
+ .toList
+ .view
+
+ def metricQuantiles(f: v1.TaskMetrics => Double): IndexedSeq[Double] =
+ Distribution(rawMetrics.map { d => f(d) }).get.getQuantiles(quantiles)
+
+ // We need to do a lot of similar munging to nested metrics here. For each one,
+ // we want (a) extract the values for nested metrics (b) make a distribution for each metric
+ // (c) shove the distribution into the right field in our return type and (d) only return
+ // a result if the option is defined for any of the tasks. MetricHelper is a little util
+ // to make it a little easier to deal w/ all of the nested options. Mostly it lets us just
+ // implement one "build" method, which just builds the quantiles for each field.
+
+ val inputMetrics =
+ new MetricHelper[v1.InputMetrics, v1.InputMetricDistributions](rawMetrics, quantiles) {
+ def getSubmetrics(raw: v1.TaskMetrics): v1.InputMetrics = raw.inputMetrics
+
+ def build: v1.InputMetricDistributions = new v1.InputMetricDistributions(
+ bytesRead = submetricQuantiles(_.bytesRead),
+ recordsRead = submetricQuantiles(_.recordsRead)
+ )
+ }.build
+
+ val outputMetrics =
+ new MetricHelper[v1.OutputMetrics, v1.OutputMetricDistributions](rawMetrics, quantiles) {
+ def getSubmetrics(raw: v1.TaskMetrics): v1.OutputMetrics = raw.outputMetrics
+
+ def build: v1.OutputMetricDistributions = new v1.OutputMetricDistributions(
+ bytesWritten = submetricQuantiles(_.bytesWritten),
+ recordsWritten = submetricQuantiles(_.recordsWritten)
+ )
+ }.build
+
+ val shuffleReadMetrics =
+ new MetricHelper[v1.ShuffleReadMetrics, v1.ShuffleReadMetricDistributions](rawMetrics,
+ quantiles) {
+ def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleReadMetrics =
+ raw.shuffleReadMetrics
+
+ def build: v1.ShuffleReadMetricDistributions = new v1.ShuffleReadMetricDistributions(
+ readBytes = submetricQuantiles { s => s.localBytesRead + s.remoteBytesRead },
+ readRecords = submetricQuantiles(_.recordsRead),
+ remoteBytesRead = submetricQuantiles(_.remoteBytesRead),
+ remoteBytesReadToDisk = submetricQuantiles(_.remoteBytesReadToDisk),
+ remoteBlocksFetched = submetricQuantiles(_.remoteBlocksFetched),
+ localBlocksFetched = submetricQuantiles(_.localBlocksFetched),
+ totalBlocksFetched = submetricQuantiles { s =>
+ s.localBlocksFetched + s.remoteBlocksFetched
+ },
+ fetchWaitTime = submetricQuantiles(_.fetchWaitTime)
+ )
+ }.build
+
+ val shuffleWriteMetrics =
+ new MetricHelper[v1.ShuffleWriteMetrics, v1.ShuffleWriteMetricDistributions](rawMetrics,
+ quantiles) {
+ def getSubmetrics(raw: v1.TaskMetrics): v1.ShuffleWriteMetrics =
+ raw.shuffleWriteMetrics
+
+ def build: v1.ShuffleWriteMetricDistributions = new v1.ShuffleWriteMetricDistributions(
+ writeBytes = submetricQuantiles(_.bytesWritten),
+ writeRecords = submetricQuantiles(_.recordsWritten),
+ writeTime = submetricQuantiles(_.writeTime)
+ )
+ }.build
+
+ new v1.TaskMetricDistributions(
+ quantiles = quantiles,
+ executorDeserializeTime = metricQuantiles(_.executorDeserializeTime),
+ executorDeserializeCpuTime = metricQuantiles(_.executorDeserializeCpuTime),
+ executorRunTime = metricQuantiles(_.executorRunTime),
+ executorCpuTime = metricQuantiles(_.executorCpuTime),
+ resultSize = metricQuantiles(_.resultSize),
+ jvmGcTime = metricQuantiles(_.jvmGcTime),
+ resultSerializationTime = metricQuantiles(_.resultSerializationTime),
+ memoryBytesSpilled = metricQuantiles(_.memoryBytesSpilled),
+ diskBytesSpilled = metricQuantiles(_.diskBytesSpilled),
+ inputMetrics = inputMetrics,
+ outputMetrics = outputMetrics,
+ shuffleReadMetrics = shuffleReadMetrics,
+ shuffleWriteMetrics = shuffleWriteMetrics
+ )
+ }
+
+ def taskList(
+ stageId: Int,
+ stageAttemptId: Int,
+ offset: Int,
+ length: Int,
+ sortBy: v1.TaskSorting): Seq[v1.TaskData] = {
+ val stageKey = Array(stageId, stageAttemptId)
+ val base = store.view(classOf[TaskDataWrapper])
+ val indexed = sortBy match {
+ case v1.TaskSorting.ID =>
+ base.index("stage").first(stageKey).last(stageKey)
+ case v1.TaskSorting.INCREASING_RUNTIME =>
+ base.index("runtime").first(stageKey ++ Array(-1L)).last(stageKey ++ Array(Long.MaxValue))
+ case v1.TaskSorting.DECREASING_RUNTIME =>
+ base.index("runtime").first(stageKey ++ Array(Long.MaxValue)).last(stageKey ++ Array(-1L))
+ .reverse()
+ }
+ indexed.skip(offset).max(length).asScala.map(_.info).toSeq
+ }
+
+ def rddList(): Seq[v1.RDDStorageInfo] = {
+ store.view(classOf[RDDStorageInfoWrapper]).asScala.map(_.info).toSeq
+ }
+
+ def rdd(rddId: Int): v1.RDDStorageInfo = {
+ store.read(classOf[RDDStorageInfoWrapper], rddId).info
+ }
+
+ def close(): Unit = {
+ store.close()
+ }
+
+}
+
+private[spark] object AppStatusStore {
+
+ val CURRENT_VERSION = 1L
+
+ /**
+ * Create an in-memory store for a live application.
+ *
+ * @param conf Configuration.
+ * @param bus Where to attach the listener to populate the store.
+ */
+ def createLiveStore(conf: SparkConf, bus: LiveListenerBus): AppStatusStore = {
+ val store = new InMemoryStore()
+ val stateStore = new AppStatusStore(store)
+ bus.addToStatusQueue(new AppStatusListener(store, conf, true))
+ stateStore
+ }
+
+}
+
+/**
+ * Helper for getting distributions from nested metric types.
+ */
+private abstract class MetricHelper[I, O](
+ rawMetrics: Seq[v1.TaskMetrics],
+ quantiles: Array[Double]) {
+
+ def getSubmetrics(raw: v1.TaskMetrics): I
+
+ def build: O
+
+ val data: Seq[I] = rawMetrics.map(getSubmetrics)
+
+ /** applies the given function to all input metrics, and returns the quantiles */
+ def submetricQuantiles(f: I => Double): IndexedSeq[Double] = {
+ Distribution(data.map { d => f(d) }).get.getQuantiles(quantiles)
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/status/KVUtils.scala b/core/src/main/scala/org/apache/spark/status/KVUtils.scala
new file mode 100644
index 0000000000000..4638511944c61
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/KVUtils.scala
@@ -0,0 +1,73 @@
+/*
+ * 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.status
+
+import java.io.File
+
+import scala.annotation.meta.getter
+import scala.language.implicitConversions
+import scala.reflect.{classTag, ClassTag}
+
+import com.fasterxml.jackson.annotation.JsonInclude
+import com.fasterxml.jackson.module.scala.DefaultScalaModule
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.kvstore._
+
+private[spark] object KVUtils extends Logging {
+
+ /** Use this to annotate constructor params to be used as KVStore indices. */
+ type KVIndexParam = KVIndex @getter
+
+ /**
+ * A KVStoreSerializer that provides Scala types serialization too, and uses the same options as
+ * the API serializer.
+ */
+ private[spark] class KVStoreScalaSerializer extends KVStoreSerializer {
+
+ mapper.registerModule(DefaultScalaModule)
+ mapper.setSerializationInclusion(JsonInclude.Include.NON_NULL)
+
+ }
+
+ /**
+ * Open or create a LevelDB store.
+ *
+ * @param path Location of the store.
+ * @param metadata Metadata value to compare to the data in the store. If the store does not
+ * contain any metadata (e.g. it's a new store), this value is written as
+ * the store's metadata.
+ */
+ def open[M: ClassTag](path: File, metadata: M): LevelDB = {
+ require(metadata != null, "Metadata is required.")
+
+ val db = new LevelDB(path, new KVStoreScalaSerializer())
+ val dbMeta = db.getMetadata(classTag[M].runtimeClass)
+ if (dbMeta == null) {
+ db.setMetadata(metadata)
+ } else if (dbMeta != metadata) {
+ db.close()
+ throw new MetadataMismatchException()
+ }
+
+ db
+ }
+
+ private[spark] class MetadataMismatchException extends Exception
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
new file mode 100644
index 0000000000000..337ef0b3e6c2b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -0,0 +1,529 @@
+/*
+ * 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.status
+
+import java.util.Date
+
+import scala.collection.mutable.HashMap
+
+import org.apache.spark.JobExecutionStatus
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler.{AccumulableInfo, StageInfo, TaskInfo}
+import org.apache.spark.status.api.v1
+import org.apache.spark.storage.RDDInfo
+import org.apache.spark.ui.SparkUI
+import org.apache.spark.util.AccumulatorContext
+import org.apache.spark.util.kvstore.KVStore
+
+/**
+ * A mutable representation of a live entity in Spark (jobs, stages, tasks, et al). Every live
+ * entity uses one of these instances to keep track of their evolving state, and periodically
+ * flush an immutable view of the entity to the app state store.
+ */
+private[spark] abstract class LiveEntity {
+
+ var lastWriteTime = 0L
+
+ def write(store: KVStore): Unit = {
+ store.write(doUpdate())
+ lastWriteTime = System.nanoTime()
+ }
+
+ /**
+ * Returns an updated view of entity data, to be stored in the status store, reflecting the
+ * latest information collected by the listener.
+ */
+ protected def doUpdate(): Any
+
+}
+
+private class LiveJob(
+ val jobId: Int,
+ name: String,
+ submissionTime: Option[Date],
+ val stageIds: Seq[Int],
+ jobGroup: Option[String],
+ numTasks: Int) extends LiveEntity {
+
+ var activeTasks = 0
+ var completedTasks = 0
+ var failedTasks = 0
+
+ var skippedTasks = 0
+ var skippedStages = Set[Int]()
+
+ var status = JobExecutionStatus.RUNNING
+ var completionTime: Option[Date] = None
+
+ var completedStages: Set[Int] = Set()
+ var activeStages = 0
+ var failedStages = 0
+
+ override protected def doUpdate(): Any = {
+ val info = new v1.JobData(
+ jobId,
+ name,
+ None, // description is always None?
+ submissionTime,
+ completionTime,
+ stageIds,
+ jobGroup,
+ status,
+ numTasks,
+ activeTasks,
+ completedTasks,
+ skippedTasks,
+ failedTasks,
+ activeStages,
+ completedStages.size,
+ skippedStages.size,
+ failedStages)
+ new JobDataWrapper(info, skippedStages)
+ }
+
+}
+
+private class LiveTask(
+ info: TaskInfo,
+ stageId: Int,
+ stageAttemptId: Int) extends LiveEntity {
+
+ import LiveEntityHelpers._
+
+ private var recordedMetrics: v1.TaskMetrics = null
+
+ var errorMessage: Option[String] = None
+
+ /**
+ * Update the metrics for the task and return the difference between the previous and new
+ * values.
+ */
+ def updateMetrics(metrics: TaskMetrics): v1.TaskMetrics = {
+ if (metrics != null) {
+ val old = recordedMetrics
+ recordedMetrics = new v1.TaskMetrics(
+ metrics.executorDeserializeTime,
+ metrics.executorDeserializeCpuTime,
+ metrics.executorRunTime,
+ metrics.executorCpuTime,
+ metrics.resultSize,
+ metrics.jvmGCTime,
+ metrics.resultSerializationTime,
+ metrics.memoryBytesSpilled,
+ metrics.diskBytesSpilled,
+ new v1.InputMetrics(
+ metrics.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead),
+ new v1.OutputMetrics(
+ metrics.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten),
+ new v1.ShuffleReadMetrics(
+ metrics.shuffleReadMetrics.remoteBlocksFetched,
+ metrics.shuffleReadMetrics.localBlocksFetched,
+ metrics.shuffleReadMetrics.fetchWaitTime,
+ metrics.shuffleReadMetrics.remoteBytesRead,
+ metrics.shuffleReadMetrics.remoteBytesReadToDisk,
+ metrics.shuffleReadMetrics.localBytesRead,
+ metrics.shuffleReadMetrics.recordsRead),
+ new v1.ShuffleWriteMetrics(
+ metrics.shuffleWriteMetrics.bytesWritten,
+ metrics.shuffleWriteMetrics.writeTime,
+ metrics.shuffleWriteMetrics.recordsWritten))
+ if (old != null) calculateMetricsDelta(recordedMetrics, old) else recordedMetrics
+ } else {
+ null
+ }
+ }
+
+ /**
+ * Return a new TaskMetrics object containing the delta of the various fields of the given
+ * metrics objects. This is currently targeted at updating stage data, so it does not
+ * necessarily calculate deltas for all the fields.
+ */
+ private def calculateMetricsDelta(
+ metrics: v1.TaskMetrics,
+ old: v1.TaskMetrics): v1.TaskMetrics = {
+ val shuffleWriteDelta = new v1.ShuffleWriteMetrics(
+ metrics.shuffleWriteMetrics.bytesWritten - old.shuffleWriteMetrics.bytesWritten,
+ 0L,
+ metrics.shuffleWriteMetrics.recordsWritten - old.shuffleWriteMetrics.recordsWritten)
+
+ val shuffleReadDelta = new v1.ShuffleReadMetrics(
+ 0L, 0L, 0L,
+ metrics.shuffleReadMetrics.remoteBytesRead - old.shuffleReadMetrics.remoteBytesRead,
+ metrics.shuffleReadMetrics.remoteBytesReadToDisk -
+ old.shuffleReadMetrics.remoteBytesReadToDisk,
+ metrics.shuffleReadMetrics.localBytesRead - old.shuffleReadMetrics.localBytesRead,
+ metrics.shuffleReadMetrics.recordsRead - old.shuffleReadMetrics.recordsRead)
+
+ val inputDelta = new v1.InputMetrics(
+ metrics.inputMetrics.bytesRead - old.inputMetrics.bytesRead,
+ metrics.inputMetrics.recordsRead - old.inputMetrics.recordsRead)
+
+ val outputDelta = new v1.OutputMetrics(
+ metrics.outputMetrics.bytesWritten - old.outputMetrics.bytesWritten,
+ metrics.outputMetrics.recordsWritten - old.outputMetrics.recordsWritten)
+
+ new v1.TaskMetrics(
+ 0L, 0L,
+ metrics.executorRunTime - old.executorRunTime,
+ metrics.executorCpuTime - old.executorCpuTime,
+ 0L, 0L, 0L,
+ metrics.memoryBytesSpilled - old.memoryBytesSpilled,
+ metrics.diskBytesSpilled - old.diskBytesSpilled,
+ inputDelta,
+ outputDelta,
+ shuffleReadDelta,
+ shuffleWriteDelta)
+ }
+
+ override protected def doUpdate(): Any = {
+ val task = new v1.TaskData(
+ info.taskId,
+ info.index,
+ info.attemptNumber,
+ new Date(info.launchTime),
+ if (info.finished) Some(info.duration) else None,
+ info.executorId,
+ info.host,
+ info.status,
+ info.taskLocality.toString(),
+ info.speculative,
+ newAccumulatorInfos(info.accumulables),
+ errorMessage,
+ Option(recordedMetrics))
+ new TaskDataWrapper(task, stageId, stageAttemptId)
+ }
+
+}
+
+private class LiveExecutor(val executorId: String) extends LiveEntity {
+
+ var hostPort: String = null
+ var host: String = null
+ var isActive = true
+ var totalCores = 0
+
+ var rddBlocks = 0
+ var memoryUsed = 0L
+ var diskUsed = 0L
+ var maxTasks = 0
+ var maxMemory = 0L
+
+ var totalTasks = 0
+ var activeTasks = 0
+ var completedTasks = 0
+ var failedTasks = 0
+ var totalDuration = 0L
+ var totalGcTime = 0L
+ var totalInputBytes = 0L
+ var totalShuffleRead = 0L
+ var totalShuffleWrite = 0L
+ var isBlacklisted = false
+
+ var executorLogs = Map[String, String]()
+
+ // Memory metrics. They may not be recorded (e.g. old event logs) so if totalOnHeap is not
+ // initialized, the store will not contain this information.
+ var totalOnHeap = -1L
+ var totalOffHeap = 0L
+ var usedOnHeap = 0L
+ var usedOffHeap = 0L
+
+ def hasMemoryInfo: Boolean = totalOnHeap >= 0L
+
+ def hostname: String = if (host != null) host else hostPort.split(":")(0)
+
+ override protected def doUpdate(): Any = {
+ val memoryMetrics = if (totalOnHeap >= 0) {
+ Some(new v1.MemoryMetrics(usedOnHeap, usedOffHeap, totalOnHeap, totalOffHeap))
+ } else {
+ None
+ }
+
+ val info = new v1.ExecutorSummary(
+ executorId,
+ if (hostPort != null) hostPort else host,
+ isActive,
+ rddBlocks,
+ memoryUsed,
+ diskUsed,
+ totalCores,
+ maxTasks,
+ activeTasks,
+ failedTasks,
+ completedTasks,
+ totalTasks,
+ totalDuration,
+ totalGcTime,
+ totalInputBytes,
+ totalShuffleRead,
+ totalShuffleWrite,
+ isBlacklisted,
+ maxMemory,
+ executorLogs,
+ memoryMetrics)
+ new ExecutorSummaryWrapper(info)
+ }
+
+}
+
+/** Metrics tracked per stage (both total and per executor). */
+private class MetricsTracker {
+ var executorRunTime = 0L
+ var executorCpuTime = 0L
+ var inputBytes = 0L
+ var inputRecords = 0L
+ var outputBytes = 0L
+ var outputRecords = 0L
+ var shuffleReadBytes = 0L
+ var shuffleReadRecords = 0L
+ var shuffleWriteBytes = 0L
+ var shuffleWriteRecords = 0L
+ var memoryBytesSpilled = 0L
+ var diskBytesSpilled = 0L
+
+ def update(delta: v1.TaskMetrics): Unit = {
+ executorRunTime += delta.executorRunTime
+ executorCpuTime += delta.executorCpuTime
+ inputBytes += delta.inputMetrics.bytesRead
+ inputRecords += delta.inputMetrics.recordsRead
+ outputBytes += delta.outputMetrics.bytesWritten
+ outputRecords += delta.outputMetrics.recordsWritten
+ shuffleReadBytes += delta.shuffleReadMetrics.localBytesRead +
+ delta.shuffleReadMetrics.remoteBytesRead
+ shuffleReadRecords += delta.shuffleReadMetrics.recordsRead
+ shuffleWriteBytes += delta.shuffleWriteMetrics.bytesWritten
+ shuffleWriteRecords += delta.shuffleWriteMetrics.recordsWritten
+ memoryBytesSpilled += delta.memoryBytesSpilled
+ diskBytesSpilled += delta.diskBytesSpilled
+ }
+
+}
+
+private class LiveExecutorStageSummary(
+ stageId: Int,
+ attemptId: Int,
+ executorId: String) extends LiveEntity {
+
+ var taskTime = 0L
+ var succeededTasks = 0
+ var failedTasks = 0
+ var killedTasks = 0
+
+ val metrics = new MetricsTracker()
+
+ override protected def doUpdate(): Any = {
+ val info = new v1.ExecutorStageSummary(
+ taskTime,
+ failedTasks,
+ succeededTasks,
+ metrics.inputBytes,
+ metrics.outputBytes,
+ metrics.shuffleReadBytes,
+ metrics.shuffleWriteBytes,
+ metrics.memoryBytesSpilled,
+ metrics.diskBytesSpilled)
+ new ExecutorStageSummaryWrapper(stageId, attemptId, executorId, info)
+ }
+
+}
+
+private class LiveStage extends LiveEntity {
+
+ import LiveEntityHelpers._
+
+ var jobs = Seq[LiveJob]()
+ var jobIds = Set[Int]()
+
+ var info: StageInfo = null
+ var status = v1.StageStatus.PENDING
+
+ var schedulingPool: String = SparkUI.DEFAULT_POOL_NAME
+
+ var activeTasks = 0
+ var completedTasks = 0
+ var failedTasks = 0
+
+ var firstLaunchTime = Long.MaxValue
+
+ val metrics = new MetricsTracker()
+
+ val executorSummaries = new HashMap[String, LiveExecutorStageSummary]()
+
+ def executorSummary(executorId: String): LiveExecutorStageSummary = {
+ executorSummaries.getOrElseUpdate(executorId,
+ new LiveExecutorStageSummary(info.stageId, info.attemptId, executorId))
+ }
+
+ override protected def doUpdate(): Any = {
+ val update = new v1.StageData(
+ status,
+ info.stageId,
+ info.attemptId,
+
+ activeTasks,
+ completedTasks,
+ failedTasks,
+
+ metrics.executorRunTime,
+ metrics.executorCpuTime,
+ info.submissionTime.map(new Date(_)),
+ if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) else None,
+ info.completionTime.map(new Date(_)),
+
+ metrics.inputBytes,
+ metrics.inputRecords,
+ metrics.outputBytes,
+ metrics.outputRecords,
+ metrics.shuffleReadBytes,
+ metrics.shuffleReadRecords,
+ metrics.shuffleWriteBytes,
+ metrics.shuffleWriteRecords,
+ metrics.memoryBytesSpilled,
+ metrics.diskBytesSpilled,
+
+ info.name,
+ info.details,
+ schedulingPool,
+
+ newAccumulatorInfos(info.accumulables.values),
+ None,
+ None)
+
+ new StageDataWrapper(update, jobIds)
+ }
+
+}
+
+private class LiveRDDPartition(val blockName: String) {
+
+ var executors = Set[String]()
+ var storageLevel: String = null
+ var memoryUsed = 0L
+ var diskUsed = 0L
+
+ def toApi(): v1.RDDPartitionInfo = {
+ new v1.RDDPartitionInfo(
+ blockName,
+ storageLevel,
+ memoryUsed,
+ diskUsed,
+ executors.toSeq.sorted)
+ }
+
+}
+
+private class LiveRDDDistribution(val exec: LiveExecutor) {
+
+ var memoryRemaining = exec.maxMemory
+ var memoryUsed = 0L
+ var diskUsed = 0L
+
+ var onHeapUsed = 0L
+ var offHeapUsed = 0L
+ var onHeapRemaining = 0L
+ var offHeapRemaining = 0L
+
+ def toApi(): v1.RDDDataDistribution = {
+ new v1.RDDDataDistribution(
+ exec.hostPort,
+ memoryUsed,
+ memoryRemaining,
+ diskUsed,
+ if (exec.hasMemoryInfo) Some(onHeapUsed) else None,
+ if (exec.hasMemoryInfo) Some(offHeapUsed) else None,
+ if (exec.hasMemoryInfo) Some(onHeapRemaining) else None,
+ if (exec.hasMemoryInfo) Some(offHeapRemaining) else None)
+ }
+
+}
+
+private class LiveRDD(info: RDDInfo) extends LiveEntity {
+
+ var storageLevel: String = info.storageLevel.description
+ var memoryUsed = 0L
+ var diskUsed = 0L
+
+ private val partitions = new HashMap[String, LiveRDDPartition]()
+ private val distributions = new HashMap[String, LiveRDDDistribution]()
+
+ def partition(blockName: String): LiveRDDPartition = {
+ partitions.getOrElseUpdate(blockName, new LiveRDDPartition(blockName))
+ }
+
+ def removePartition(blockName: String): Unit = partitions.remove(blockName)
+
+ def distribution(exec: LiveExecutor): LiveRDDDistribution = {
+ distributions.getOrElseUpdate(exec.hostPort, new LiveRDDDistribution(exec))
+ }
+
+ def removeDistribution(exec: LiveExecutor): Unit = {
+ distributions.remove(exec.hostPort)
+ }
+
+ override protected def doUpdate(): Any = {
+ val parts = if (partitions.nonEmpty) {
+ Some(partitions.values.toList.sortBy(_.blockName).map(_.toApi()))
+ } else {
+ None
+ }
+
+ val dists = if (distributions.nonEmpty) {
+ Some(distributions.values.toList.sortBy(_.exec.executorId).map(_.toApi()))
+ } else {
+ None
+ }
+
+ val rdd = new v1.RDDStorageInfo(
+ info.id,
+ info.name,
+ info.numPartitions,
+ partitions.size,
+ storageLevel,
+ memoryUsed,
+ diskUsed,
+ dists,
+ parts)
+
+ new RDDStorageInfoWrapper(rdd)
+ }
+
+}
+
+private object LiveEntityHelpers {
+
+ def newAccumulatorInfos(accums: Iterable[AccumulableInfo]): Seq[v1.AccumulableInfo] = {
+ accums
+ .filter { acc =>
+ // We don't need to store internal or SQL accumulables as their values will be shown in
+ // other places, so drop them to reduce the memory usage.
+ !acc.internal && (!acc.metadata.isDefined ||
+ acc.metadata.get != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER))
+ }
+ .map { acc =>
+ new v1.AccumulableInfo(
+ acc.id,
+ acc.name.map(_.intern()).orNull,
+ acc.update.map(_.toString()),
+ acc.value.map(_.toString()).orNull)
+ }
+ .toSeq
+ }
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
index 4a4ed954d689e..5f69949c618fd 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/AllStagesResource.scala
@@ -71,7 +71,7 @@ private[v1] object AllStagesResource {
val taskData = if (includeDetails) {
Some(stageUiData.taskData.map { case (k, v) =>
- k -> convertTaskData(v, stageUiData.lastUpdateTime) })
+ k -> convertTaskData(v, stageUiData.lastUpdateTime) }.toMap)
} else {
None
}
@@ -88,7 +88,7 @@ private[v1] object AllStagesResource {
memoryBytesSpilled = summary.memoryBytesSpilled,
diskBytesSpilled = summary.diskBytesSpilled
)
- })
+ }.toMap)
} else {
None
}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
index f17b637754826..9d3833086172f 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala
@@ -248,7 +248,13 @@ private[spark] object ApiRootResource {
* interface needed for them all to expose application info as json.
*/
private[spark] trait UIRoot {
- def getSparkUI(appKey: String): Option[SparkUI]
+ /**
+ * Runs some code with the current SparkUI instance for the app / attempt.
+ *
+ * @throws NoSuchElementException If the app / attempt pair does not exist.
+ */
+ def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T
+
def getApplicationInfoList: Iterator[ApplicationInfo]
def getApplicationInfo(appId: String): Option[ApplicationInfo]
@@ -293,15 +299,18 @@ private[v1] trait ApiRequestContext {
* to it. If there is no such app, throw an appropriate exception
*/
def withSparkUI[T](appId: String, attemptId: Option[String])(f: SparkUI => T): T = {
- val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
- uiRoot.getSparkUI(appKey) match {
- case Some(ui) =>
+ try {
+ uiRoot.withSparkUI(appId, attemptId) { ui =>
val user = httpRequest.getRemoteUser()
if (!ui.securityManager.checkUIViewPermissions(user)) {
throw new ForbiddenException(raw"""user "$user" is not authorized""")
}
f(ui)
- case None => throw new NotFoundException("no such app: " + appId)
+ }
+ } catch {
+ case _: NoSuchElementException =>
+ val appKey = attemptId.map(appId + "/" + _).getOrElse(appId)
+ throw new NotFoundException(s"no such app: $appKey")
}
}
}
diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
index 05948f2661056..bff6f90823f40 100644
--- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
+++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala
@@ -16,9 +16,11 @@
*/
package org.apache.spark.status.api.v1
+import java.lang.{Long => JLong}
import java.util.Date
-import scala.collection.Map
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize
import org.apache.spark.JobExecutionStatus
@@ -31,6 +33,9 @@ class ApplicationInfo private[spark](
val memoryPerExecutorMB: Option[Int],
val attempts: Seq[ApplicationAttemptInfo])
+@JsonIgnoreProperties(
+ value = Array("startTimeEpoch", "endTimeEpoch", "lastUpdatedEpoch"),
+ allowGetters = true)
class ApplicationAttemptInfo private[spark](
val attemptId: Option[String],
val startTime: Date,
@@ -40,9 +45,13 @@ class ApplicationAttemptInfo private[spark](
val sparkUser: String,
val completed: Boolean = false,
val appSparkVersion: String) {
- def getStartTimeEpoch: Long = startTime.getTime
- def getEndTimeEpoch: Long = endTime.getTime
- def getLastUpdatedEpoch: Long = lastUpdated.getTime
+
+ def getStartTimeEpoch: Long = startTime.getTime
+
+ def getEndTimeEpoch: Long = endTime.getTime
+
+ def getLastUpdatedEpoch: Long = lastUpdated.getTime
+
}
class ExecutorStageSummary private[spark](
@@ -120,9 +129,13 @@ class RDDDataDistribution private[spark](
val memoryUsed: Long,
val memoryRemaining: Long,
val diskUsed: Long,
+ @JsonDeserialize(contentAs = classOf[JLong])
val onHeapMemoryUsed: Option[Long],
+ @JsonDeserialize(contentAs = classOf[JLong])
val offHeapMemoryUsed: Option[Long],
+ @JsonDeserialize(contentAs = classOf[JLong])
val onHeapMemoryRemaining: Option[Long],
+ @JsonDeserialize(contentAs = classOf[JLong])
val offHeapMemoryRemaining: Option[Long])
class RDDPartitionInfo private[spark](
@@ -170,7 +183,8 @@ class TaskData private[spark](
val index: Int,
val attempt: Int,
val launchTime: Date,
- val duration: Option[Long] = None,
+ @JsonDeserialize(contentAs = classOf[JLong])
+ val duration: Option[Long],
val executorId: String,
val host: String,
val status: String,
diff --git a/core/src/main/scala/org/apache/spark/status/config.scala b/core/src/main/scala/org/apache/spark/status/config.scala
new file mode 100644
index 0000000000000..49144fc883e69
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/config.scala
@@ -0,0 +1,30 @@
+/*
+ * 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.status
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.internal.config._
+
+private[spark] object config {
+
+ val LIVE_ENTITY_UPDATE_PERIOD = ConfigBuilder("spark.ui.liveUpdate.period")
+ .timeConf(TimeUnit.NANOSECONDS)
+ .createWithDefaultString("100ms")
+
+}
diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
new file mode 100644
index 0000000000000..340d5994a0012
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala
@@ -0,0 +1,123 @@
+/*
+ * 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.status
+
+import java.lang.{Integer => JInteger, Long => JLong}
+
+import com.fasterxml.jackson.annotation.JsonIgnore
+
+import org.apache.spark.status.KVUtils._
+import org.apache.spark.status.api.v1._
+import org.apache.spark.util.kvstore.KVIndex
+
+private[spark] case class AppStatusStoreMetadata(
+ val version: Long)
+
+private[spark] class ApplicationInfoWrapper(val info: ApplicationInfo) {
+
+ @JsonIgnore @KVIndex
+ def id: String = info.id
+
+}
+
+private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) {
+
+ @JsonIgnore @KVIndex
+ private[this] val id: String = info.id
+
+ @JsonIgnore @KVIndex("active")
+ private[this] val active: Boolean = info.isActive
+
+ @JsonIgnore @KVIndex("host")
+ val host: String = info.hostPort.split(":")(0)
+
+}
+
+/**
+ * Keep track of the existing stages when the job was submitted, and those that were
+ * completed during the job's execution. This allows a more accurate acounting of how
+ * many tasks were skipped for the job.
+ */
+private[spark] class JobDataWrapper(
+ val info: JobData,
+ val skippedStages: Set[Int]) {
+
+ @JsonIgnore @KVIndex
+ private[this] val id: Int = info.jobId
+
+}
+
+private[spark] class StageDataWrapper(
+ val info: StageData,
+ val jobIds: Set[Int]) {
+
+ @JsonIgnore @KVIndex
+ def id: Array[Int] = Array(info.stageId, info.attemptId)
+
+ @JsonIgnore @KVIndex("stageId")
+ def stageId: Int = info.stageId
+
+}
+
+/**
+ * The task information is always indexed with the stage ID, since that is how the UI and API
+ * consume it. That means every indexed value has the stage ID and attempt ID included, aside
+ * from the actual data being indexed.
+ */
+private[spark] class TaskDataWrapper(
+ val info: TaskData,
+ val stageId: Int,
+ val stageAttemptId: Int) {
+
+ @JsonIgnore @KVIndex
+ def id: Long = info.taskId
+
+ @JsonIgnore @KVIndex("stage")
+ def stage: Array[Int] = Array(stageId, stageAttemptId)
+
+ @JsonIgnore @KVIndex("runtime")
+ def runtime: Array[AnyRef] = {
+ val _runtime = info.taskMetrics.map(_.executorRunTime).getOrElse(-1L)
+ Array(stageId: JInteger, stageAttemptId: JInteger, _runtime: JLong)
+ }
+
+}
+
+private[spark] class RDDStorageInfoWrapper(val info: RDDStorageInfo) {
+
+ @JsonIgnore @KVIndex
+ def id: Int = info.id
+
+ @JsonIgnore @KVIndex("cached")
+ def cached: Boolean = info.numCachedPartitions > 0
+
+}
+
+private[spark] class ExecutorStageSummaryWrapper(
+ val stageId: Int,
+ val stageAttemptId: Int,
+ val executorId: String,
+ val info: ExecutorStageSummary) {
+
+ @JsonIgnore @KVIndex
+ val id: Array[Any] = Array(stageId, stageAttemptId, executorId)
+
+ @JsonIgnore @KVIndex("stage")
+ private[this] val stage: Array[Int] = Array(stageId, stageAttemptId)
+
+}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
index 524f6970992a5..7ac2c71c18eb3 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala
@@ -19,6 +19,7 @@ package org.apache.spark.storage
import java.util.UUID
+import org.apache.spark.SparkException
import org.apache.spark.annotation.DeveloperApi
/**
@@ -41,11 +42,6 @@ sealed abstract class BlockId {
def isBroadcast: Boolean = isInstanceOf[BroadcastBlockId]
override def toString: String = name
- override def hashCode: Int = name.hashCode
- override def equals(other: Any): Boolean = other match {
- case o: BlockId => getClass == o.getClass && name.equals(o.name)
- case _ => false
- }
}
@DeveloperApi
@@ -100,6 +96,10 @@ private[spark] case class TestBlockId(id: String) extends BlockId {
override def name: String = "test_" + id
}
+@DeveloperApi
+class UnrecognizedBlockId(name: String)
+ extends SparkException(s"Failed to parse $name into a block ID")
+
@DeveloperApi
object BlockId {
val RDD = "rdd_([0-9]+)_([0-9]+)".r
@@ -109,10 +109,11 @@ object BlockId {
val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r
val TASKRESULT = "taskresult_([0-9]+)".r
val STREAM = "input-([0-9]+)-([0-9]+)".r
+ val TEMP_LOCAL = "temp_local_([-A-Fa-f0-9]+)".r
+ val TEMP_SHUFFLE = "temp_shuffle_([-A-Fa-f0-9]+)".r
val TEST = "test_(.*)".r
- /** Converts a BlockId "name" String back into a BlockId. */
- def apply(id: String): BlockId = id match {
+ def apply(name: String): BlockId = name match {
case RDD(rddId, splitIndex) =>
RDDBlockId(rddId.toInt, splitIndex.toInt)
case SHUFFLE(shuffleId, mapId, reduceId) =>
@@ -127,9 +128,13 @@ object BlockId {
TaskResultBlockId(taskId.toLong)
case STREAM(streamId, uniqueId) =>
StreamBlockId(streamId.toInt, uniqueId.toLong)
+ case TEMP_LOCAL(uuid) =>
+ TempLocalBlockId(UUID.fromString(uuid))
+ case TEMP_SHUFFLE(uuid) =>
+ TempShuffleBlockId(UUID.fromString(uuid))
case TEST(value) =>
TestBlockId(value)
case _ =>
- throw new IllegalStateException("Unrecognized BlockId: " + id)
+ throw new UnrecognizedBlockId(name)
}
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
index a98083df5bd84..e0276a4dc4224 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -18,8 +18,11 @@
package org.apache.spark.storage
import java.io._
+import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference}
import java.nio.ByteBuffer
import java.nio.channels.Channels
+import java.util.Collections
+import java.util.concurrent.ConcurrentHashMap
import scala.collection.mutable
import scala.collection.mutable.HashMap
@@ -39,7 +42,7 @@ import org.apache.spark.metrics.source.Source
import org.apache.spark.network._
import org.apache.spark.network.buffer.ManagedBuffer
import org.apache.spark.network.netty.SparkTransportConf
-import org.apache.spark.network.shuffle.ExternalShuffleClient
+import org.apache.spark.network.shuffle.{ExternalShuffleClient, TempFileManager}
import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
@@ -203,6 +206,13 @@ private[spark] class BlockManager(
private var blockReplicationPolicy: BlockReplicationPolicy = _
+ // A TempFileManager used to track all the files of remote blocks which above the
+ // specified memory threshold. Files will be deleted automatically based on weak reference.
+ // Exposed for test
+ private[storage] val remoteBlockTempFileManager =
+ new BlockManager.RemoteBlockTempFileManager(this)
+ private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM)
+
/**
* Initializes the BlockManager with the given appId. This is not performed in the constructor as
* the appId may not be known at BlockManager instantiation time (in particular for the driver,
@@ -632,8 +642,8 @@ private[spark] class BlockManager(
* Return a list of locations for the given block, prioritizing the local machine since
* multiple block managers can share the same host, followed by hosts on the same rack.
*/
- private def getLocations(blockId: BlockId): Seq[BlockManagerId] = {
- val locs = Random.shuffle(master.getLocations(blockId))
+ private def sortLocations(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = {
+ val locs = Random.shuffle(locations)
val (preferredLocs, otherLocs) = locs.partition { loc => blockManagerId.host == loc.host }
blockManagerId.topologyInfo match {
case None => preferredLocs ++ otherLocs
@@ -653,7 +663,25 @@ private[spark] class BlockManager(
require(blockId != null, "BlockId is null")
var runningFailureCount = 0
var totalFailureCount = 0
- val locations = getLocations(blockId)
+
+ // Because all the remote blocks are registered in driver, it is not necessary to ask
+ // all the slave executors to get block status.
+ val locationsAndStatus = master.getLocationsAndStatus(blockId)
+ val blockSize = locationsAndStatus.map { b =>
+ b.status.diskSize.max(b.status.memSize)
+ }.getOrElse(0L)
+ val blockLocations = locationsAndStatus.map(_.locations).getOrElse(Seq.empty)
+
+ // If the block size is above the threshold, we should pass our FileManger to
+ // BlockTransferService, which will leverage it to spill the block; if not, then passed-in
+ // null value means the block will be persisted in memory.
+ val tempFileManager = if (blockSize > maxRemoteBlockToMem) {
+ remoteBlockTempFileManager
+ } else {
+ null
+ }
+
+ val locations = sortLocations(blockLocations)
val maxFetchFailures = locations.size
var locationIterator = locations.iterator
while (locationIterator.hasNext) {
@@ -661,7 +689,7 @@ private[spark] class BlockManager(
logDebug(s"Getting remote block $blockId from $loc")
val data = try {
blockTransferService.fetchBlockSync(
- loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer()
+ loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager).nioByteBuffer()
} catch {
case NonFatal(e) =>
runningFailureCount += 1
@@ -684,7 +712,7 @@ private[spark] class BlockManager(
// take a significant amount of time. To get rid of these stale entries
// we refresh the block locations after a certain number of fetch failures
if (runningFailureCount >= maxFailuresBeforeLocationRefresh) {
- locationIterator = getLocations(blockId).iterator
+ locationIterator = sortLocations(master.getLocations(blockId)).iterator
logDebug(s"Refreshed locations from the driver " +
s"after ${runningFailureCount} fetch failures.")
runningFailureCount = 0
@@ -1512,6 +1540,7 @@ private[spark] class BlockManager(
// Closing should be idempotent, but maybe not for the NioBlockTransferService.
shuffleClient.close()
}
+ remoteBlockTempFileManager.stop()
diskBlockManager.stop()
rpcEnv.stop(slaveEndpoint)
blockInfoManager.clear()
@@ -1552,4 +1581,65 @@ private[spark] object BlockManager {
override val metricRegistry = new MetricRegistry
metricRegistry.registerAll(metricSet)
}
+
+ class RemoteBlockTempFileManager(blockManager: BlockManager)
+ extends TempFileManager with Logging {
+
+ private class ReferenceWithCleanup(file: File, referenceQueue: JReferenceQueue[File])
+ extends WeakReference[File](file, referenceQueue) {
+ private val filePath = file.getAbsolutePath
+
+ def cleanUp(): Unit = {
+ logDebug(s"Clean up file $filePath")
+
+ if (!new File(filePath).delete()) {
+ logDebug(s"Fail to delete file $filePath")
+ }
+ }
+ }
+
+ private val referenceQueue = new JReferenceQueue[File]
+ private val referenceBuffer = Collections.newSetFromMap[ReferenceWithCleanup](
+ new ConcurrentHashMap)
+
+ private val POLL_TIMEOUT = 1000
+ @volatile private var stopped = false
+
+ private val cleaningThread = new Thread() { override def run() { keepCleaning() } }
+ cleaningThread.setDaemon(true)
+ cleaningThread.setName("RemoteBlock-temp-file-clean-thread")
+ cleaningThread.start()
+
+ override def createTempFile(): File = {
+ blockManager.diskBlockManager.createTempLocalBlock()._2
+ }
+
+ override def registerTempFileToClean(file: File): Boolean = {
+ referenceBuffer.add(new ReferenceWithCleanup(file, referenceQueue))
+ }
+
+ def stop(): Unit = {
+ stopped = true
+ cleaningThread.interrupt()
+ cleaningThread.join()
+ }
+
+ private def keepCleaning(): Unit = {
+ while (!stopped) {
+ try {
+ Option(referenceQueue.remove(POLL_TIMEOUT))
+ .map(_.asInstanceOf[ReferenceWithCleanup])
+ .foreach { ref =>
+ referenceBuffer.remove(ref)
+ ref.cleanUp()
+ }
+ } catch {
+ case _: InterruptedException =>
+ // no-op
+ case NonFatal(e) =>
+ logError("Error in cleaning thread", e)
+ }
+ }
+ }
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
index 8b1dc0ba6356a..d24421b962774 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala
@@ -84,6 +84,12 @@ class BlockManagerMaster(
driverEndpoint.askSync[Seq[BlockManagerId]](GetLocations(blockId))
}
+ /** Get locations as well as status of the blockId from the driver */
+ def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = {
+ driverEndpoint.askSync[Option[BlockLocationsAndStatus]](
+ GetLocationsAndStatus(blockId))
+ }
+
/** Get locations of multiple blockIds from the driver */
def getLocations(blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = {
driverEndpoint.askSync[IndexedSeq[Seq[BlockManagerId]]](
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
index df0a5f5e229fb..56d0266b8edad 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
@@ -82,6 +82,9 @@ class BlockManagerMasterEndpoint(
case GetLocations(blockId) =>
context.reply(getLocations(blockId))
+ case GetLocationsAndStatus(blockId) =>
+ context.reply(getLocationsAndStatus(blockId))
+
case GetLocationsMultipleBlockIds(blockIds) =>
context.reply(getLocationsMultipleBlockIds(blockIds))
@@ -422,6 +425,17 @@ class BlockManagerMasterEndpoint(
if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty
}
+ private def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = {
+ val locations = Option(blockLocations.get(blockId)).map(_.toSeq).getOrElse(Seq.empty)
+ val status = locations.headOption.flatMap { bmId => blockManagerInfo(bmId).getStatus(blockId) }
+
+ if (locations.nonEmpty && status.isDefined) {
+ Some(BlockLocationsAndStatus(locations, status.get))
+ } else {
+ None
+ }
+ }
+
private def getLocationsMultipleBlockIds(
blockIds: Array[BlockId]): IndexedSeq[Seq[BlockManagerId]] = {
blockIds.map(blockId => getLocations(blockId))
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index 0c0ff144596ac..1bbe7a5b39509 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
@@ -93,6 +93,13 @@ private[spark] object BlockManagerMessages {
case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster
+ case class GetLocationsAndStatus(blockId: BlockId) extends ToBlockManagerMaster
+
+ // The response message of `GetLocationsAndStatus` request.
+ case class BlockLocationsAndStatus(locations: Seq[BlockManagerId], status: BlockStatus) {
+ assert(locations.nonEmpty)
+ }
+
case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster
case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
index 3d43e3c367aac..a69bcc9259995 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
@@ -100,7 +100,16 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
/** List all the blocks currently stored on disk by the disk manager. */
def getAllBlocks(): Seq[BlockId] = {
- getAllFiles().map(f => BlockId(f.getName))
+ getAllFiles().flatMap { f =>
+ try {
+ Some(BlockId(f.getName))
+ } catch {
+ case _: UnrecognizedBlockId =>
+ // Skip files which do not correspond to blocks, for example temporary
+ // files created by [[SortShuffleWriter]].
+ None
+ }
+ }
}
/** Produces a unique block id and File suitable for storing local intermediate results. */
diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
index 3579acf8d83d9..97abd92d4b70f 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -47,9 +47,9 @@ private[spark] class DiskStore(
private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m")
private val maxMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests",
Int.MaxValue.toString)
- private val blockSizes = new ConcurrentHashMap[String, Long]()
+ private val blockSizes = new ConcurrentHashMap[BlockId, Long]()
- def getSize(blockId: BlockId): Long = blockSizes.get(blockId.name)
+ def getSize(blockId: BlockId): Long = blockSizes.get(blockId)
/**
* Invokes the provided callback function to write the specific block.
@@ -67,7 +67,7 @@ private[spark] class DiskStore(
var threwException: Boolean = true
try {
writeFunc(out)
- blockSizes.put(blockId.name, out.getCount)
+ blockSizes.put(blockId, out.getCount)
threwException = false
} finally {
try {
@@ -113,7 +113,7 @@ private[spark] class DiskStore(
}
def remove(blockId: BlockId): Boolean = {
- blockSizes.remove(blockId.name)
+ blockSizes.remove(blockId)
val file = diskManager.getFile(blockId.name)
if (file.exists()) {
val ret = file.delete()
diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
index 2d176b62f8b36..98b5a735a4529 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -28,7 +28,7 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue}
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempFileManager}
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util.Utils
import org.apache.spark.util.io.ChunkedByteBufferOutputStream
@@ -69,7 +69,7 @@ final class ShuffleBlockFetcherIterator(
maxBlocksInFlightPerAddress: Int,
maxReqSizeShuffleToMem: Long,
detectCorrupt: Boolean)
- extends Iterator[(BlockId, InputStream)] with TempShuffleFileManager with Logging {
+ extends Iterator[(BlockId, InputStream)] with TempFileManager with Logging {
import ShuffleBlockFetcherIterator._
@@ -162,11 +162,11 @@ final class ShuffleBlockFetcherIterator(
currentResult = null
}
- override def createTempShuffleFile(): File = {
+ override def createTempFile(): File = {
blockManager.diskBlockManager.createTempLocalBlock()._2
}
- override def registerTempShuffleFileToClean(file: File): Boolean = synchronized {
+ override def registerTempFileToClean(file: File): Boolean = synchronized {
if (isZombie) {
false
} else {
diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
index 651e9c7b2ab61..17f7a69ad6ba1 100644
--- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala
@@ -388,7 +388,13 @@ private[spark] class MemoryStore(
// perform one final call to attempt to allocate additional memory if necessary.
if (keepUnrolling) {
serializationStream.close()
- reserveAdditionalMemoryIfNecessary()
+ if (bbos.size > unrollMemoryUsedByThisBlock) {
+ val amountToRequest = bbos.size - unrollMemoryUsedByThisBlock
+ keepUnrolling = reserveUnrollMemoryForThisTask(blockId, amountToRequest, memoryMode)
+ if (keepUnrolling) {
+ unrollMemoryUsedByThisBlock += amountToRequest
+ }
+ }
}
if (keepUnrolling) {
diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 5ee04dad6ed4d..0adeb4058b6e4 100644
--- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
@@ -39,6 +39,7 @@ import org.json4s.jackson.JsonMethods.{pretty, render}
import org.apache.spark.{SecurityManager, SparkConf, SSLOptions}
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config._
import org.apache.spark.util.Utils
/**
@@ -89,6 +90,14 @@ private[spark] object JettyUtils extends Logging {
val result = servletParams.responder(request)
response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate")
response.setHeader("X-Frame-Options", xFrameOptionsValue)
+ response.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION))
+ if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) {
+ response.setHeader("X-Content-Type-Options", "nosniff")
+ }
+ if (request.getScheme == "https") {
+ conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach(
+ response.setHeader("Strict-Transport-Security", _))
+ }
response.getWriter.print(servletParams.extractFn(result))
} else {
response.setStatus(HttpServletResponse.SC_FORBIDDEN)
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index 6e94073238a56..ee645f6bf8a7a 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -24,6 +24,7 @@ import scala.collection.JavaConverters._
import org.apache.spark.{SecurityManager, SparkConf, SparkContext}
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
+import org.apache.spark.status.AppStatusStore
import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationAttemptInfo, ApplicationInfo,
UIRoot}
import org.apache.spark.storage.StorageStatusListener
@@ -39,6 +40,7 @@ import org.apache.spark.util.Utils
* Top level user interface for a Spark application.
*/
private[spark] class SparkUI private (
+ val store: AppStatusStore,
val sc: Option[SparkContext],
val conf: SparkConf,
securityManager: SecurityManager,
@@ -51,7 +53,8 @@ private[spark] class SparkUI private (
var appName: String,
val basePath: String,
val lastUpdateTime: Option[Long] = None,
- val startTime: Long)
+ val startTime: Long,
+ val appSparkVersion: String)
extends WebUI(securityManager, securityManager.getSSLOptions("ui"), SparkUI.getUIPort(conf),
conf, basePath, "SparkUI")
with Logging
@@ -61,8 +64,6 @@ private[spark] class SparkUI private (
var appId: String = _
- var appSparkVersion = org.apache.spark.SPARK_VERSION
-
private var streamingJobProgressListener: Option[SparkListener] = None
/** Initialize all components of the server. */
@@ -104,8 +105,12 @@ private[spark] class SparkUI private (
logInfo(s"Stopped Spark web UI at $webUrl")
}
- def getSparkUI(appId: String): Option[SparkUI] = {
- if (appId == this.appId) Some(this) else None
+ override def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T = {
+ if (appId == this.appId) {
+ fn(this)
+ } else {
+ throw new NoSuchElementException()
+ }
}
def getApplicationInfoList: Iterator[ApplicationInfo] = {
@@ -159,63 +164,26 @@ private[spark] object SparkUI {
conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT)
}
- def createLiveUI(
- sc: SparkContext,
- conf: SparkConf,
- jobProgressListener: JobProgressListener,
- securityManager: SecurityManager,
- appName: String,
- startTime: Long): SparkUI = {
- create(Some(sc), conf,
- sc.listenerBus.addToStatusQueue,
- securityManager, appName, jobProgressListener = Some(jobProgressListener),
- startTime = startTime)
- }
-
- def createHistoryUI(
- conf: SparkConf,
- listenerBus: SparkListenerBus,
- securityManager: SecurityManager,
- appName: String,
- basePath: String,
- lastUpdateTime: Option[Long],
- startTime: Long): SparkUI = {
- val sparkUI = create(None, conf, listenerBus.addListener, securityManager, appName, basePath,
- lastUpdateTime = lastUpdateTime, startTime = startTime)
-
- val listenerFactories = ServiceLoader.load(classOf[SparkHistoryListenerFactory],
- Utils.getContextOrSparkClassLoader).asScala
- listenerFactories.foreach { listenerFactory =>
- val listeners = listenerFactory.createListeners(conf, sparkUI)
- listeners.foreach(listenerBus.addListener)
- }
- sparkUI
- }
-
/**
- * Create a new Spark UI.
- *
- * @param sc optional SparkContext; this can be None when reconstituting a UI from event logs.
- * @param jobProgressListener if supplied, this JobProgressListener will be used; otherwise, the
- * web UI will create and register its own JobProgressListener.
+ * Create a new UI backed by an AppStatusStore.
*/
- private def create(
+ def create(
sc: Option[SparkContext],
+ store: AppStatusStore,
conf: SparkConf,
addListenerFn: SparkListenerInterface => Unit,
securityManager: SecurityManager,
appName: String,
- basePath: String = "",
- jobProgressListener: Option[JobProgressListener] = None,
+ basePath: String,
+ startTime: Long,
lastUpdateTime: Option[Long] = None,
- startTime: Long): SparkUI = {
+ appSparkVersion: String = org.apache.spark.SPARK_VERSION): SparkUI = {
- val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse {
+ val jobProgressListener = sc.map(_.jobProgressListener).getOrElse {
val listener = new JobProgressListener(conf)
addListenerFn(listener)
listener
}
-
val environmentListener = new EnvironmentListener
val storageStatusListener = new StorageStatusListener(conf)
val executorsListener = new ExecutorsListener(storageStatusListener, conf)
@@ -228,8 +196,9 @@ private[spark] object SparkUI {
addListenerFn(storageListener)
addListenerFn(operationGraphListener)
- new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener,
- executorsListener, _jobProgressListener, storageListener, operationGraphListener,
- appName, basePath, lastUpdateTime, startTime)
+ new SparkUI(store, sc, conf, securityManager, environmentListener, storageStatusListener,
+ executorsListener, jobProgressListener, storageListener, operationGraphListener,
+ appName, basePath, lastUpdateTime, startTime, appSparkVersion)
}
+
}
diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
index d63381c78bc3b..7b2767f0be3cd 100644
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -82,7 +82,7 @@ private[ui] class ExecutorsPage(
++
- ++
+
++
++
++
diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
index 8406826a228db..5e60218c5740b 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -98,8 +98,8 @@ private[spark] object JsonProtocol {
logStartToJson(logStart)
case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
executorMetricsUpdateToJson(metricsUpdate)
- case blockUpdated: SparkListenerBlockUpdated =>
- throw new MatchError(blockUpdated) // TODO(ekl) implement this
+ case blockUpdate: SparkListenerBlockUpdated =>
+ blockUpdateToJson(blockUpdate)
case _ => parse(mapper.writeValueAsString(event))
}
}
@@ -246,6 +246,12 @@ private[spark] object JsonProtocol {
})
}
+ def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = {
+ val blockUpdatedInfo = blockUpdatedInfoToJson(blockUpdate.blockUpdatedInfo)
+ ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.blockUpdate) ~
+ ("Block Updated Info" -> blockUpdatedInfo)
+ }
+
/** ------------------------------------------------------------------- *
* JSON serialization methods for classes SparkListenerEvents depend on |
* -------------------------------------------------------------------- */
@@ -458,6 +464,14 @@ private[spark] object JsonProtocol {
("Log Urls" -> mapToJson(executorInfo.logUrlMap))
}
+ def blockUpdatedInfoToJson(blockUpdatedInfo: BlockUpdatedInfo): JValue = {
+ ("Block Manager ID" -> blockManagerIdToJson(blockUpdatedInfo.blockManagerId)) ~
+ ("Block ID" -> blockUpdatedInfo.blockId.toString) ~
+ ("Storage Level" -> storageLevelToJson(blockUpdatedInfo.storageLevel)) ~
+ ("Memory Size" -> blockUpdatedInfo.memSize) ~
+ ("Disk Size" -> blockUpdatedInfo.diskSize)
+ }
+
/** ------------------------------ *
* Util JSON serialization methods |
* ------------------------------- */
@@ -515,6 +529,7 @@ private[spark] object JsonProtocol {
val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved)
val logStart = Utils.getFormattedClassName(SparkListenerLogStart)
val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate)
+ val blockUpdate = Utils.getFormattedClassName(SparkListenerBlockUpdated)
}
def sparkEventFromJson(json: JValue): SparkListenerEvent = {
@@ -538,6 +553,7 @@ private[spark] object JsonProtocol {
case `executorRemoved` => executorRemovedFromJson(json)
case `logStart` => logStartFromJson(json)
case `metricsUpdate` => executorMetricsUpdateFromJson(json)
+ case `blockUpdate` => blockUpdateFromJson(json)
case other => mapper.readValue(compact(render(json)), Utils.classForName(other))
.asInstanceOf[SparkListenerEvent]
}
@@ -676,6 +692,11 @@ private[spark] object JsonProtocol {
SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates)
}
+ def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = {
+ val blockUpdatedInfo = blockUpdatedInfoFromJson(json \ "Block Updated Info")
+ SparkListenerBlockUpdated(blockUpdatedInfo)
+ }
+
/** --------------------------------------------------------------------- *
* JSON deserialization methods for classes SparkListenerEvents depend on |
* ---------------------------------------------------------------------- */
@@ -989,6 +1010,15 @@ private[spark] object JsonProtocol {
new ExecutorInfo(executorHost, totalCores, logUrls)
}
+ def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = {
+ val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID")
+ val blockId = BlockId((json \ "Block ID").extract[String])
+ val storageLevel = storageLevelFromJson(json \ "Storage Level")
+ val memorySize = (json \ "Memory Size").extract[Long]
+ val diskSize = (json \ "Disk Size").extract[Long]
+ BlockUpdatedInfo(blockManagerId, blockId, storageLevel, memorySize, diskSize)
+ }
+
/** -------------------------------- *
* Util JSON deserialization methods |
* --------------------------------- */
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 836e33c36d9a1..930e09d90c2f5 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -19,6 +19,7 @@ package org.apache.spark.util
import java.io._
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
+import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
import java.net._
import java.nio.ByteBuffer
@@ -37,7 +38,7 @@ import scala.collection.Map
import scala.collection.mutable.ArrayBuffer
import scala.io.Source
import scala.reflect.ClassTag
-import scala.util.Try
+import scala.util.{Failure, Success, Try}
import scala.util.control.{ControlThrowable, NonFatal}
import scala.util.matching.Regex
@@ -2687,6 +2688,60 @@ private[spark] object Utils extends Logging {
def stringToSeq(str: String): Seq[String] = {
str.split(",").map(_.trim()).filter(_.nonEmpty)
}
+
+ /**
+ * Create instances of extension classes.
+ *
+ * The classes in the given list must:
+ * - Be sub-classes of the given base class.
+ * - Provide either a no-arg constructor, or a 1-arg constructor that takes a SparkConf.
+ *
+ * The constructors are allowed to throw "UnsupportedOperationException" if the extension does not
+ * want to be registered; this allows the implementations to check the Spark configuration (or
+ * other state) and decide they do not need to be added. A log message is printed in that case.
+ * Other exceptions are bubbled up.
+ */
+ def loadExtensions[T](extClass: Class[T], classes: Seq[String], conf: SparkConf): Seq[T] = {
+ classes.flatMap { name =>
+ try {
+ val klass = classForName(name)
+ require(extClass.isAssignableFrom(klass),
+ s"$name is not a subclass of ${extClass.getName()}.")
+
+ val ext = Try(klass.getConstructor(classOf[SparkConf])) match {
+ case Success(ctor) =>
+ ctor.newInstance(conf)
+
+ case Failure(_) =>
+ klass.getConstructor().newInstance()
+ }
+
+ Some(ext.asInstanceOf[T])
+ } catch {
+ case _: NoSuchMethodException =>
+ throw new SparkException(
+ s"$name did not have a zero-argument constructor or a" +
+ " single-argument constructor that accepts SparkConf. Note: if the class is" +
+ " defined inside of another Scala class, then its constructors may accept an" +
+ " implicit parameter that references the enclosing class; in this case, you must" +
+ " define the class as a top-level class in order to prevent this extra" +
+ " parameter from breaking Spark's ability to find a valid constructor.")
+
+ case e: InvocationTargetException =>
+ e.getCause() match {
+ case uoe: UnsupportedOperationException =>
+ logDebug(s"Extension $name not being initialized.", uoe)
+ logInfo(s"Extension $name not being initialized.")
+ None
+
+ case null => throw e
+
+ case cause => throw cause
+ }
+ }
+ }
+ }
+
}
private[util] object CallerContext extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
index b755e5da51684..e17a9de97e335 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala
@@ -19,6 +19,8 @@ package org.apache.spark.util.collection
import java.util.Comparator
+import org.apache.spark.unsafe.Platform
+import org.apache.spark.unsafe.array.ByteArrayMethods
import org.apache.spark.util.collection.WritablePartitionedPairCollection._
/**
@@ -96,7 +98,5 @@ private[spark] class PartitionedPairBuffer[K, V](initialCapacity: Int = 64)
}
private object PartitionedPairBuffer {
- // Some JVMs can't allocate arrays of length Integer.MAX_VALUE; actual max is somewhat
- // smaller. Be conservative and lower the cap a little.
- val MAXIMUM_CAPACITY: Int = (Int.MaxValue - 8) / 2
+ val MAXIMUM_CAPACITY: Int = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 2
}
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
index 5330a688e63e3..d0d0334add0bf 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java
@@ -23,6 +23,7 @@
import java.util.LinkedList;
import java.util.UUID;
+import org.hamcrest.Matchers;
import scala.Tuple2$;
import org.junit.After;
@@ -503,6 +504,41 @@ public void testGetIterator() throws Exception {
verifyIntIterator(sorter.getIterator(279), 279, 300);
}
+ @Test
+ public void testOOMDuringSpill() throws Exception {
+ final UnsafeExternalSorter sorter = newSorter();
+ // we assume that given default configuration,
+ // the size of the data we insert to the sorter (ints)
+ // and assuming we shouldn't spill before pointers array is exhausted
+ // (memory manager is not configured to throw at this point)
+ // - so this loop runs a reasonable number of iterations (<2000).
+ // test indeed completed within <30ms (on a quad i7 laptop).
+ for (int i = 0; sorter.hasSpaceForAnotherRecord(); ++i) {
+ insertNumber(sorter, i);
+ }
+ // we expect the next insert to attempt growing the pointerssArray first
+ // allocation is expected to fail, then a spill is triggered which
+ // attempts another allocation which also fails and we expect to see this
+ // OOM here. the original code messed with a released array within the
+ // spill code and ended up with a failed assertion. we also expect the
+ // location of the OOM to be
+ // org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset
+ memoryManager.markconsequentOOM(2);
+ try {
+ insertNumber(sorter, 1024);
+ fail("expected OutOfMmoryError but it seems operation surprisingly succeeded");
+ }
+ // we expect an OutOfMemoryError here, anything else (i.e the original NPE is a failure)
+ catch (OutOfMemoryError oom){
+ String oomStackTrace = Utils.exceptionString(oom);
+ assertThat("expected OutOfMemoryError in " +
+ "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset",
+ oomStackTrace,
+ Matchers.containsString(
+ "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset"));
+ }
+ }
+
private void verifyIntIterator(UnsafeSorterIterator iter, int start, int end)
throws IOException {
for (int i = start; i < end; i++) {
diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
index bd89085aa9a14..594f07dd780f9 100644
--- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
+++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java
@@ -35,6 +35,7 @@
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.isIn;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
import static org.mockito.Mockito.mock;
public class UnsafeInMemorySorterSuite {
@@ -139,4 +140,50 @@ public int compare(
}
assertEquals(dataToSort.length, iterLength);
}
+
+ @Test
+ public void freeAfterOOM() {
+ final SparkConf sparkConf = new SparkConf();
+ sparkConf.set("spark.memory.offHeap.enabled", "false");
+
+ final TestMemoryManager testMemoryManager =
+ new TestMemoryManager(sparkConf);
+ final TaskMemoryManager memoryManager = new TaskMemoryManager(
+ testMemoryManager, 0);
+ final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager);
+ final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer);
+ final Object baseObject = dataPage.getBaseObject();
+ // Write the records into the data page:
+ long position = dataPage.getBaseOffset();
+
+ final HashPartitioner hashPartitioner = new HashPartitioner(4);
+ // Use integer comparison for comparing prefixes (which are partition ids, in this case)
+ final PrefixComparator prefixComparator = PrefixComparators.LONG;
+ final RecordComparator recordComparator = new RecordComparator() {
+ @Override
+ public int compare(
+ Object leftBaseObject,
+ long leftBaseOffset,
+ Object rightBaseObject,
+ long rightBaseOffset) {
+ return 0;
+ }
+ };
+ UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager,
+ recordComparator, prefixComparator, 100, shouldUseRadixSort());
+
+ testMemoryManager.markExecutionAsOutOfMemoryOnce();
+ try {
+ sorter.reset();
+ fail("expected OutOfMmoryError but it seems operation surprisingly succeeded");
+ } catch (OutOfMemoryError oom) {
+ // as expected
+ }
+ // [SPARK-21907] this failed on NPE at
+ // org.apache.spark.memory.MemoryConsumer.freeArray(MemoryConsumer.java:108)
+ sorter.free();
+ // simulate a 'back to back' free.
+ sorter.free();
+ }
+
}
diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
index bea67b71a5a12..f8005610f7e4f 100644
--- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala
@@ -171,7 +171,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex
val serializerManager = SparkEnv.get.serializerManager
blockManager.master.getLocations(blockId).foreach { cmId =>
val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId,
- blockId.toString)
+ blockId.toString, null)
val deserialized = serializerManager.dataDeserializeStream(blockId,
new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList
assert(deserialized === (1 to 100).toList)
diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala
index 02728180ac82d..e9539dc73f6fa 100644
--- a/core/src/test/scala/org/apache/spark/FileSuite.scala
+++ b/core/src/test/scala/org/apache/spark/FileSuite.scala
@@ -31,7 +31,7 @@ import org.apache.hadoop.mapreduce.Job
import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat}
import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat}
-import org.apache.spark.internal.config.IGNORE_CORRUPT_FILES
+import org.apache.spark.internal.config._
import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
@@ -347,10 +347,10 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
}
}
- test ("allow user to disable the output directory existence checking (old Hadoop API") {
- val sf = new SparkConf()
- sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
- sc = new SparkContext(sf)
+ test ("allow user to disable the output directory existence checking (old Hadoop API)") {
+ val conf = new SparkConf()
+ conf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
+ sc = new SparkContext(conf)
val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1)
randomRDD.saveAsTextFile(tempDir.getPath + "/output")
assert(new File(tempDir.getPath + "/output/part-00000").exists() === true)
@@ -380,9 +380,9 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
}
test ("allow user to disable the output directory existence checking (new Hadoop API") {
- val sf = new SparkConf()
- sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
- sc = new SparkContext(sf)
+ val conf = new SparkConf()
+ conf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false")
+ sc = new SparkContext(conf)
val randomRDD = sc.parallelize(
Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1)
randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](
@@ -510,4 +510,87 @@ class FileSuite extends SparkFunSuite with LocalSparkContext {
}
}
+ test("spark.hadoopRDD.ignoreEmptySplits work correctly (old Hadoop API)") {
+ val conf = new SparkConf()
+ .setAppName("test")
+ .setMaster("local")
+ .set(HADOOP_RDD_IGNORE_EMPTY_SPLITS, true)
+ sc = new SparkContext(conf)
+
+ def testIgnoreEmptySplits(
+ data: Array[Tuple2[String, String]],
+ actualPartitionNum: Int,
+ expectedPartitionNum: Int): Unit = {
+ val output = new File(tempDir, "output")
+ sc.parallelize(data, actualPartitionNum)
+ .saveAsHadoopFile[TextOutputFormat[String, String]](output.getPath)
+ for (i <- 0 until actualPartitionNum) {
+ assert(new File(output, s"part-0000$i").exists() === true)
+ }
+ val hadoopRDD = sc.textFile(new File(output, "part-*").getPath)
+ assert(hadoopRDD.partitions.length === expectedPartitionNum)
+ Utils.deleteRecursively(output)
+ }
+
+ // Ensure that if all of the splits are empty, we remove the splits correctly
+ testIgnoreEmptySplits(
+ data = Array.empty[Tuple2[String, String]],
+ actualPartitionNum = 1,
+ expectedPartitionNum = 0)
+
+ // Ensure that if no split is empty, we don't lose any splits
+ testIgnoreEmptySplits(
+ data = Array(("key1", "a"), ("key2", "a"), ("key3", "b")),
+ actualPartitionNum = 2,
+ expectedPartitionNum = 2)
+
+ // Ensure that if part of the splits are empty, we remove the splits correctly
+ testIgnoreEmptySplits(
+ data = Array(("key1", "a"), ("key2", "a")),
+ actualPartitionNum = 5,
+ expectedPartitionNum = 2)
+ }
+
+ test("spark.hadoopRDD.ignoreEmptySplits work correctly (new Hadoop API)") {
+ val conf = new SparkConf()
+ .setAppName("test")
+ .setMaster("local")
+ .set(HADOOP_RDD_IGNORE_EMPTY_SPLITS, true)
+ sc = new SparkContext(conf)
+
+ def testIgnoreEmptySplits(
+ data: Array[Tuple2[String, String]],
+ actualPartitionNum: Int,
+ expectedPartitionNum: Int): Unit = {
+ val output = new File(tempDir, "output")
+ sc.parallelize(data, actualPartitionNum)
+ .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](output.getPath)
+ for (i <- 0 until actualPartitionNum) {
+ assert(new File(output, s"part-r-0000$i").exists() === true)
+ }
+ val hadoopRDD = sc.newAPIHadoopFile(new File(output, "part-r-*").getPath,
+ classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text])
+ .asInstanceOf[NewHadoopRDD[_, _]]
+ assert(hadoopRDD.partitions.length === expectedPartitionNum)
+ Utils.deleteRecursively(output)
+ }
+
+ // Ensure that if all of the splits are empty, we remove the splits correctly
+ testIgnoreEmptySplits(
+ data = Array.empty[Tuple2[String, String]],
+ actualPartitionNum = 1,
+ expectedPartitionNum = 0)
+
+ // Ensure that if no split is empty, we don't lose any splits
+ testIgnoreEmptySplits(
+ data = Array(("1", "a"), ("2", "a"), ("3", "b")),
+ actualPartitionNum = 2,
+ expectedPartitionNum = 2)
+
+ // Ensure that if part of the splits are empty, we remove the splits correctly
+ testIgnoreEmptySplits(
+ data = Array(("1", "a"), ("2", "b")),
+ actualPartitionNum = 5,
+ expectedPartitionNum = 2)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
index 6aedcb1271ff6..1aa1c421d792e 100644
--- a/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
+++ b/core/src/test/scala/org/apache/spark/SharedSparkContext.scala
@@ -29,10 +29,23 @@ trait SharedSparkContext extends BeforeAndAfterAll with BeforeAndAfterEach { sel
var conf = new SparkConf(false)
+ /**
+ * Initialize the [[SparkContext]]. Generally, this is just called from beforeAll; however, in
+ * test using styles other than FunSuite, there is often code that relies on the session between
+ * test group constructs and the actual tests, which may need this session. It is purely a
+ * semantic difference, but semantically, it makes more sense to call 'initializeContext' between
+ * a 'describe' and an 'it' call than it does to call 'beforeAll'.
+ */
+ protected def initializeContext(): Unit = {
+ if (null == _sc) {
+ _sc = new SparkContext(
+ "local[4]", "test", conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName))
+ }
+ }
+
override def beforeAll() {
super.beforeAll()
- _sc = new SparkContext(
- "local[4]", "test", conf.set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName))
+ initializeContext()
}
override def afterAll() {
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index ad801bf8519a6..cfbf56fb8c369 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -100,6 +100,8 @@ class SparkSubmitSuite
with TimeLimits
with TestPrematureExit {
+ import SparkSubmitSuite._
+
override def beforeEach() {
super.beforeEach()
System.setProperty("spark.testing", "true")
@@ -174,10 +176,10 @@ class SparkSubmitSuite
"thejar.jar"
)
val appArgs = new SparkSubmitArguments(clArgs)
- val (_, _, sysProps, _) = prepareSubmitEnvironment(appArgs)
+ val (_, _, conf, _) = prepareSubmitEnvironment(appArgs)
appArgs.deployMode should be ("client")
- sysProps("spark.submit.deployMode") should be ("client")
+ conf.get("spark.submit.deployMode") should be ("client")
// Both cmd line and configuration are specified, cmdline option takes the priority
val clArgs1 = Seq(
@@ -188,10 +190,10 @@ class SparkSubmitSuite
"thejar.jar"
)
val appArgs1 = new SparkSubmitArguments(clArgs1)
- val (_, _, sysProps1, _) = prepareSubmitEnvironment(appArgs1)
+ val (_, _, conf1, _) = prepareSubmitEnvironment(appArgs1)
appArgs1.deployMode should be ("cluster")
- sysProps1("spark.submit.deployMode") should be ("cluster")
+ conf1.get("spark.submit.deployMode") should be ("cluster")
// Neither cmdline nor configuration are specified, client mode is the default choice
val clArgs2 = Seq(
@@ -202,9 +204,9 @@ class SparkSubmitSuite
val appArgs2 = new SparkSubmitArguments(clArgs2)
appArgs2.deployMode should be (null)
- val (_, _, sysProps2, _) = prepareSubmitEnvironment(appArgs2)
+ val (_, _, conf2, _) = prepareSubmitEnvironment(appArgs2)
appArgs2.deployMode should be ("client")
- sysProps2("spark.submit.deployMode") should be ("client")
+ conf2.get("spark.submit.deployMode") should be ("client")
}
test("handles YARN cluster mode") {
@@ -225,7 +227,7 @@ class SparkSubmitSuite
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
+ val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
val childArgsStr = childArgs.mkString(" ")
childArgsStr should include ("--class org.SomeClass")
childArgsStr should include ("--arg arg1 --arg arg2")
@@ -238,16 +240,16 @@ class SparkSubmitSuite
classpath(2) should endWith ("two.jar")
classpath(3) should endWith ("three.jar")
- sysProps("spark.executor.memory") should be ("5g")
- sysProps("spark.driver.memory") should be ("4g")
- sysProps("spark.executor.cores") should be ("5")
- sysProps("spark.yarn.queue") should be ("thequeue")
- sysProps("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar")
- sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
- sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
- sysProps("spark.app.name") should be ("beauty")
- sysProps("spark.ui.enabled") should be ("false")
- sysProps("SPARK_SUBMIT") should be ("true")
+ conf.get("spark.executor.memory") should be ("5g")
+ conf.get("spark.driver.memory") should be ("4g")
+ conf.get("spark.executor.cores") should be ("5")
+ conf.get("spark.yarn.queue") should be ("thequeue")
+ conf.get("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar")
+ conf.get("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
+ conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
+ conf.get("spark.app.name") should be ("beauty")
+ conf.get("spark.ui.enabled") should be ("false")
+ sys.props("SPARK_SUBMIT") should be ("true")
}
test("handles YARN client mode") {
@@ -268,7 +270,7 @@ class SparkSubmitSuite
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
+ val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (4)
@@ -276,17 +278,17 @@ class SparkSubmitSuite
classpath(1) should endWith ("one.jar")
classpath(2) should endWith ("two.jar")
classpath(3) should endWith ("three.jar")
- sysProps("spark.app.name") should be ("trill")
- sysProps("spark.executor.memory") should be ("5g")
- sysProps("spark.executor.cores") should be ("5")
- sysProps("spark.yarn.queue") should be ("thequeue")
- sysProps("spark.executor.instances") should be ("6")
- sysProps("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
- sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
- sysProps("spark.yarn.dist.jars") should include
+ conf.get("spark.app.name") should be ("trill")
+ conf.get("spark.executor.memory") should be ("5g")
+ conf.get("spark.executor.cores") should be ("5")
+ conf.get("spark.yarn.queue") should be ("thequeue")
+ conf.get("spark.executor.instances") should be ("6")
+ conf.get("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt")
+ conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt")
+ conf.get("spark.yarn.dist.jars") should include
regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar")
- sysProps("SPARK_SUBMIT") should be ("true")
- sysProps("spark.ui.enabled") should be ("false")
+ conf.get("spark.ui.enabled") should be ("false")
+ sys.props("SPARK_SUBMIT") should be ("true")
}
test("handles standalone cluster mode") {
@@ -314,7 +316,7 @@ class SparkSubmitSuite
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
appArgs.useRest = useRest
- val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
+ val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
val childArgsStr = childArgs.mkString(" ")
if (useRest) {
childArgsStr should endWith ("thejar.jar org.SomeClass arg1 arg2")
@@ -325,17 +327,18 @@ class SparkSubmitSuite
mainClass should be ("org.apache.spark.deploy.Client")
}
classpath should have size 0
- sysProps should have size 9
- sysProps.keys should contain ("SPARK_SUBMIT")
- sysProps.keys should contain ("spark.master")
- sysProps.keys should contain ("spark.app.name")
- sysProps.keys should contain ("spark.jars")
- sysProps.keys should contain ("spark.driver.memory")
- sysProps.keys should contain ("spark.driver.cores")
- sysProps.keys should contain ("spark.driver.supervise")
- sysProps.keys should contain ("spark.ui.enabled")
- sysProps.keys should contain ("spark.submit.deployMode")
- sysProps("spark.ui.enabled") should be ("false")
+ sys.props("SPARK_SUBMIT") should be ("true")
+
+ val confMap = conf.getAll.toMap
+ confMap.keys should contain ("spark.master")
+ confMap.keys should contain ("spark.app.name")
+ confMap.keys should contain ("spark.jars")
+ confMap.keys should contain ("spark.driver.memory")
+ confMap.keys should contain ("spark.driver.cores")
+ confMap.keys should contain ("spark.driver.supervise")
+ confMap.keys should contain ("spark.ui.enabled")
+ confMap.keys should contain ("spark.submit.deployMode")
+ conf.get("spark.ui.enabled") should be ("false")
}
test("handles standalone client mode") {
@@ -350,14 +353,14 @@ class SparkSubmitSuite
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
+ val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (1)
classpath(0) should endWith ("thejar.jar")
- sysProps("spark.executor.memory") should be ("5g")
- sysProps("spark.cores.max") should be ("5")
- sysProps("spark.ui.enabled") should be ("false")
+ conf.get("spark.executor.memory") should be ("5g")
+ conf.get("spark.cores.max") should be ("5")
+ conf.get("spark.ui.enabled") should be ("false")
}
test("handles mesos client mode") {
@@ -372,14 +375,14 @@ class SparkSubmitSuite
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (childArgs, classpath, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
+ val (childArgs, classpath, conf, mainClass) = prepareSubmitEnvironment(appArgs)
childArgs.mkString(" ") should be ("arg1 arg2")
mainClass should be ("org.SomeClass")
classpath should have length (1)
classpath(0) should endWith ("thejar.jar")
- sysProps("spark.executor.memory") should be ("5g")
- sysProps("spark.cores.max") should be ("5")
- sysProps("spark.ui.enabled") should be ("false")
+ conf.get("spark.executor.memory") should be ("5g")
+ conf.get("spark.cores.max") should be ("5")
+ conf.get("spark.ui.enabled") should be ("false")
}
test("handles confs with flag equivalents") {
@@ -392,13 +395,28 @@ class SparkSubmitSuite
"thejar.jar",
"arg1", "arg2")
val appArgs = new SparkSubmitArguments(clArgs)
- val (_, _, sysProps, mainClass) = prepareSubmitEnvironment(appArgs)
- sysProps("spark.executor.memory") should be ("5g")
- sysProps("spark.master") should be ("yarn")
- sysProps("spark.submit.deployMode") should be ("cluster")
+ val (_, _, conf, mainClass) = prepareSubmitEnvironment(appArgs)
+ conf.get("spark.executor.memory") should be ("5g")
+ conf.get("spark.master") should be ("yarn")
+ conf.get("spark.submit.deployMode") should be ("cluster")
mainClass should be ("org.apache.spark.deploy.yarn.Client")
}
+ test("SPARK-21568 ConsoleProgressBar should be enabled only in shells") {
+ // Unset from system properties since this config is defined in the root pom's test config.
+ sys.props -= UI_SHOW_CONSOLE_PROGRESS.key
+
+ val clArgs1 = Seq("--class", "org.apache.spark.repl.Main", "spark-shell")
+ val appArgs1 = new SparkSubmitArguments(clArgs1)
+ val (_, _, conf1, _) = prepareSubmitEnvironment(appArgs1)
+ conf1.get(UI_SHOW_CONSOLE_PROGRESS) should be (true)
+
+ val clArgs2 = Seq("--class", "org.SomeClass", "thejar.jar")
+ val appArgs2 = new SparkSubmitArguments(clArgs2)
+ val (_, _, conf2, _) = prepareSubmitEnvironment(appArgs2)
+ assert(!conf2.contains(UI_SHOW_CONSOLE_PROGRESS))
+ }
+
test("launch simple application with spark-submit") {
val unusedJar = TestUtils.createJarWithClasses(Seq.empty)
val args = Seq(
@@ -571,11 +589,11 @@ class SparkSubmitSuite
"--files", files,
"thejar.jar")
val appArgs = new SparkSubmitArguments(clArgs)
- val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3
+ val (_, _, conf, _) = SparkSubmit.prepareSubmitEnvironment(appArgs)
appArgs.jars should be (Utils.resolveURIs(jars))
appArgs.files should be (Utils.resolveURIs(files))
- sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar"))
- sysProps("spark.files") should be (Utils.resolveURIs(files))
+ conf.get("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar"))
+ conf.get("spark.files") should be (Utils.resolveURIs(files))
// Test files and archives (Yarn)
val clArgs2 = Seq(
@@ -586,11 +604,11 @@ class SparkSubmitSuite
"thejar.jar"
)
val appArgs2 = new SparkSubmitArguments(clArgs2)
- val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3
+ val (_, _, conf2, _) = SparkSubmit.prepareSubmitEnvironment(appArgs2)
appArgs2.files should be (Utils.resolveURIs(files))
appArgs2.archives should be (Utils.resolveURIs(archives))
- sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files))
- sysProps2("spark.yarn.dist.archives") should be (Utils.resolveURIs(archives))
+ conf2.get("spark.yarn.dist.files") should be (Utils.resolveURIs(files))
+ conf2.get("spark.yarn.dist.archives") should be (Utils.resolveURIs(archives))
// Test python files
val clArgs3 = Seq(
@@ -601,12 +619,12 @@ class SparkSubmitSuite
"mister.py"
)
val appArgs3 = new SparkSubmitArguments(clArgs3)
- val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3
+ val (_, _, conf3, _) = SparkSubmit.prepareSubmitEnvironment(appArgs3)
appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles))
- sysProps3("spark.submit.pyFiles") should be (
+ conf3.get("spark.submit.pyFiles") should be (
PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(","))
- sysProps3(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4")
- sysProps3(PYSPARK_PYTHON.key) should be ("python3.5")
+ conf3.get(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4")
+ conf3.get(PYSPARK_PYTHON.key) should be ("python3.5")
}
test("resolves config paths correctly") {
@@ -630,9 +648,9 @@ class SparkSubmitSuite
"thejar.jar"
)
val appArgs = new SparkSubmitArguments(clArgs)
- val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3
- sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar"))
- sysProps("spark.files") should be(Utils.resolveURIs(files))
+ val (_, _, conf, _) = SparkSubmit.prepareSubmitEnvironment(appArgs)
+ conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar"))
+ conf.get("spark.files") should be(Utils.resolveURIs(files))
// Test files and archives (Yarn)
val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir)
@@ -647,9 +665,9 @@ class SparkSubmitSuite
"thejar.jar"
)
val appArgs2 = new SparkSubmitArguments(clArgs2)
- val sysProps2 = SparkSubmit.prepareSubmitEnvironment(appArgs2)._3
- sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files))
- sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives))
+ val (_, _, conf2, _) = SparkSubmit.prepareSubmitEnvironment(appArgs2)
+ conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files))
+ conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives))
// Test python files
val f3 = File.createTempFile("test-submit-python-files", "", tmpDir)
@@ -662,8 +680,8 @@ class SparkSubmitSuite
"mister.py"
)
val appArgs3 = new SparkSubmitArguments(clArgs3)
- val sysProps3 = SparkSubmit.prepareSubmitEnvironment(appArgs3)._3
- sysProps3("spark.submit.pyFiles") should be(
+ val (_, _, conf3, _) = SparkSubmit.prepareSubmitEnvironment(appArgs3)
+ conf3.get("spark.submit.pyFiles") should be(
PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(","))
// Test remote python files
@@ -679,11 +697,9 @@ class SparkSubmitSuite
"hdfs:///tmp/mister.py"
)
val appArgs4 = new SparkSubmitArguments(clArgs4)
- val sysProps4 = SparkSubmit.prepareSubmitEnvironment(appArgs4)._3
+ val (_, _, conf4, _) = SparkSubmit.prepareSubmitEnvironment(appArgs4)
// Should not format python path for yarn cluster mode
- sysProps4("spark.submit.pyFiles") should be(
- Utils.resolveURIs(remotePyFiles)
- )
+ conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles))
}
test("user classpath first in driver") {
@@ -757,14 +773,14 @@ class SparkSubmitSuite
jar2.toString)
val appArgs = new SparkSubmitArguments(args)
- val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs)._3
- sysProps("spark.yarn.dist.jars").split(",").toSet should be
+ val (_, _, conf, _) = SparkSubmit.prepareSubmitEnvironment(appArgs)
+ conf.get("spark.yarn.dist.jars").split(",").toSet should be
(Set(jar1.toURI.toString, jar2.toURI.toString))
- sysProps("spark.yarn.dist.files").split(",").toSet should be
+ conf.get("spark.yarn.dist.files").split(",").toSet should be
(Set(file1.toURI.toString, file2.toURI.toString))
- sysProps("spark.yarn.dist.pyFiles").split(",").toSet should be
+ conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be
(Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath))
- sysProps("spark.yarn.dist.archives").split(",").toSet should be
+ conf.get("spark.yarn.dist.archives").split(",").toSet should be
(Set(archive1.toURI.toString, archive2.toURI.toString))
}
@@ -883,18 +899,18 @@ class SparkSubmitSuite
)
val appArgs = new SparkSubmitArguments(args)
- val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))._3
+ val (_, _, conf, _) = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))
// All the resources should still be remote paths, so that YARN client will not upload again.
- sysProps("spark.yarn.dist.jars") should be (tmpJarPath)
- sysProps("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}")
- sysProps("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}")
+ conf.get("spark.yarn.dist.jars") should be (tmpJarPath)
+ conf.get("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}")
+ conf.get("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}")
// Local repl jars should be a local path.
- sysProps("spark.repl.local.jars") should (startWith("file:"))
+ conf.get("spark.repl.local.jars") should (startWith("file:"))
// local py files should not be a URI format.
- sysProps("spark.submit.pyFiles") should (startWith("/"))
+ conf.get("spark.submit.pyFiles") should (startWith("/"))
}
test("download remote resource if it is not supported by yarn service") {
@@ -941,9 +957,9 @@ class SparkSubmitSuite
)
val appArgs = new SparkSubmitArguments(args)
- val sysProps = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))._3
+ val (_, _, conf, _) = SparkSubmit.prepareSubmitEnvironment(appArgs, Some(hadoopConf))
- val jars = sysProps("spark.yarn.dist.jars").split(",").toSet
+ val jars = conf.get("spark.yarn.dist.jars").split(",").toSet
// The URI of remote S3 resource should still be remote.
assert(jars.contains(tmpS3JarPath))
@@ -962,30 +978,6 @@ class SparkSubmitSuite
}
}
- // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
- private def runSparkSubmit(args: Seq[String]): Unit = {
- val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
- val sparkSubmitFile = if (Utils.isWindows) {
- new File("..\\bin\\spark-submit.cmd")
- } else {
- new File("../bin/spark-submit")
- }
- val process = Utils.executeCommand(
- Seq(sparkSubmitFile.getCanonicalPath) ++ args,
- new File(sparkHome),
- Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
-
- try {
- val exitCode = failAfter(60 seconds) { process.waitFor() }
- if (exitCode != 0) {
- fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.")
- }
- } finally {
- // Ensure we still kill the process in case it timed out
- process.destroy()
- }
- }
-
private def forConfDir(defaults: Map[String, String]) (f: String => Unit) = {
val tmpDir = Utils.createTempDir()
@@ -1006,6 +998,47 @@ class SparkSubmitSuite
conf.set("fs.s3a.impl", classOf[TestFileSystem].getCanonicalName)
conf.set("fs.s3a.impl.disable.cache", "true")
}
+
+ test("start SparkApplication without modifying system properties") {
+ val args = Array(
+ "--class", classOf[TestSparkApplication].getName(),
+ "--master", "local",
+ "--conf", "spark.test.hello=world",
+ "spark-internal",
+ "hello")
+
+ val exception = intercept[SparkException] {
+ SparkSubmit.main(args)
+ }
+
+ assert(exception.getMessage() === "hello")
+ }
+}
+
+object SparkSubmitSuite extends SparkFunSuite with TimeLimits {
+ // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly.
+ def runSparkSubmit(args: Seq[String], root: String = ".."): Unit = {
+ val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+ val sparkSubmitFile = if (Utils.isWindows) {
+ new File(s"$root\\bin\\spark-submit.cmd")
+ } else {
+ new File(s"$root/bin/spark-submit")
+ }
+ val process = Utils.executeCommand(
+ Seq(sparkSubmitFile.getCanonicalPath) ++ args,
+ new File(sparkHome),
+ Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+
+ try {
+ val exitCode = failAfter(60 seconds) { process.waitFor() }
+ if (exitCode != 0) {
+ fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.")
+ }
+ } finally {
+ // Ensure we still kill the process in case it timed out
+ process.destroy()
+ }
+ }
}
object JarCreationTest extends Logging {
@@ -1099,3 +1132,17 @@ class TestFileSystem extends org.apache.hadoop.fs.LocalFileSystem {
override def open(path: Path): FSDataInputStream = super.open(local(path))
}
+
+class TestSparkApplication extends SparkApplication with Matchers {
+
+ override def start(args: Array[String], conf: SparkConf): Unit = {
+ assert(args.size === 1)
+ assert(args(0) === "hello")
+ assert(conf.get("spark.test.hello") === "world")
+ assert(sys.props.get("spark.test.hello") === None)
+
+ // This is how the test verifies the application was actually run.
+ throw new SparkException(args(0))
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala
index 6e50e84549047..44f9c566a380d 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala
@@ -18,15 +18,11 @@
package org.apache.spark.deploy.history
import java.util.{Date, NoSuchElementException}
-import javax.servlet.Filter
import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
import scala.collection.mutable
-import scala.collection.mutable.ListBuffer
import com.codahale.metrics.Counter
-import com.google.common.cache.LoadingCache
-import com.google.common.util.concurrent.UncheckedExecutionException
import org.eclipse.jetty.servlet.ServletContextHandler
import org.mockito.Matchers._
import org.mockito.Mockito._
@@ -39,23 +35,10 @@ import org.apache.spark.SparkFunSuite
import org.apache.spark.internal.Logging
import org.apache.spark.status.api.v1.{ApplicationAttemptInfo => AttemptInfo, ApplicationInfo}
import org.apache.spark.ui.SparkUI
-import org.apache.spark.util.{Clock, ManualClock, Utils}
+import org.apache.spark.util.ManualClock
class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar with Matchers {
- /**
- * subclass with access to the cache internals
- * @param retainedApplications number of retained applications
- */
- class TestApplicationCache(
- operations: ApplicationCacheOperations = new StubCacheOperations(),
- retainedApplications: Int,
- clock: Clock = new ManualClock(0))
- extends ApplicationCache(operations, retainedApplications, clock) {
-
- def cache(): LoadingCache[CacheKey, CacheEntry] = appCache
- }
-
/**
* Stub cache operations.
* The state is kept in a map of [[CacheKey]] to [[CacheEntry]],
@@ -77,8 +60,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = {
logDebug(s"getAppUI($appId, $attemptId)")
getAppUICount += 1
- instances.get(CacheKey(appId, attemptId)).map( e =>
- LoadedAppUI(e.ui, () => updateProbe(appId, attemptId, e.probeTime)))
+ instances.get(CacheKey(appId, attemptId)).map { e => e.loadedUI }
}
override def attachSparkUI(
@@ -96,10 +78,9 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
attemptId: Option[String],
completed: Boolean,
started: Long,
- ended: Long,
- timestamp: Long): SparkUI = {
- val ui = putAppUI(appId, attemptId, completed, started, ended, timestamp)
- attachSparkUI(appId, attemptId, ui, completed)
+ ended: Long): LoadedAppUI = {
+ val ui = putAppUI(appId, attemptId, completed, started, ended)
+ attachSparkUI(appId, attemptId, ui.ui, completed)
ui
}
@@ -108,23 +89,12 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
attemptId: Option[String],
completed: Boolean,
started: Long,
- ended: Long,
- timestamp: Long): SparkUI = {
- val ui = newUI(appId, attemptId, completed, started, ended)
- putInstance(appId, attemptId, ui, completed, timestamp)
+ ended: Long): LoadedAppUI = {
+ val ui = LoadedAppUI(newUI(appId, attemptId, completed, started, ended))
+ instances(CacheKey(appId, attemptId)) = new CacheEntry(ui, completed)
ui
}
- def putInstance(
- appId: String,
- attemptId: Option[String],
- ui: SparkUI,
- completed: Boolean,
- timestamp: Long): Unit = {
- instances += (CacheKey(appId, attemptId) ->
- new CacheEntry(ui, completed, () => updateProbe(appId, attemptId, timestamp), timestamp))
- }
-
/**
* Detach a reconstructed UI
*
@@ -146,23 +116,6 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
attached.get(CacheKey(appId, attemptId))
}
- /**
- * The update probe.
- * @param appId application to probe
- * @param attemptId attempt to probe
- * @param updateTime timestamp of this UI load
- */
- private[history] def updateProbe(
- appId: String,
- attemptId: Option[String],
- updateTime: Long)(): Boolean = {
- updateProbeCount += 1
- logDebug(s"isUpdated($appId, $attemptId, ${updateTime})")
- val entry = instances.get(CacheKey(appId, attemptId)).get
- val updated = entry.probeTime > updateTime
- logDebug(s"entry = $entry; updated = $updated")
- updated
- }
}
/**
@@ -210,15 +163,13 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
val now = clock.getTimeMillis()
// add the entry
- operations.putAppUI(app1, None, true, now, now, now)
+ operations.putAppUI(app1, None, true, now, now)
// make sure its local
operations.getAppUI(app1, None).get
operations.getAppUICount = 0
// now expect it to be found
- val cacheEntry = cache.lookupCacheEntry(app1, None)
- assert(1 === cacheEntry.probeTime)
- assert(cacheEntry.completed)
+ cache.withSparkUI(app1, None) { _ => }
// assert about queries made of the operations
assert(1 === operations.getAppUICount, "getAppUICount")
assert(1 === operations.attachCount, "attachCount")
@@ -236,8 +187,8 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
assert(0 === operations.detachCount, "attachCount")
// evict the entry
- operations.putAndAttach("2", None, true, time2, time2, time2)
- operations.putAndAttach("3", None, true, time2, time2, time2)
+ operations.putAndAttach("2", None, true, time2, time2)
+ operations.putAndAttach("3", None, true, time2, time2)
cache.get("2")
cache.get("3")
@@ -248,7 +199,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
val appId = "app1"
val attemptId = Some("_01")
val time3 = clock.getTimeMillis()
- operations.putAppUI(appId, attemptId, false, time3, 0, time3)
+ operations.putAppUI(appId, attemptId, false, time3, 0)
// expect an error here
assertNotFound(appId, None)
}
@@ -256,10 +207,11 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
test("Test that if an attempt ID is set, it must be used in lookups") {
val operations = new StubCacheOperations()
val clock = new ManualClock(1)
- implicit val cache = new ApplicationCache(operations, retainedApplications = 10, clock = clock)
+ implicit val cache = new ApplicationCache(operations, retainedApplications = 10,
+ clock = clock)
val appId = "app1"
val attemptId = Some("_01")
- operations.putAppUI(appId, attemptId, false, clock.getTimeMillis(), 0, 0)
+ operations.putAppUI(appId, attemptId, false, clock.getTimeMillis(), 0)
assertNotFound(appId, None)
}
@@ -271,50 +223,29 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
test("Incomplete apps refreshed") {
val operations = new StubCacheOperations()
val clock = new ManualClock(50)
- val window = 500
- implicit val cache = new ApplicationCache(operations, retainedApplications = 5, clock = clock)
+ implicit val cache = new ApplicationCache(operations, 5, clock)
val metrics = cache.metrics
// add the incomplete app
// add the entry
val started = clock.getTimeMillis()
val appId = "app1"
val attemptId = Some("001")
- operations.putAppUI(appId, attemptId, false, started, 0, started)
- val firstEntry = cache.lookupCacheEntry(appId, attemptId)
- assert(started === firstEntry.probeTime, s"timestamp in $firstEntry")
- assert(!firstEntry.completed, s"entry is complete: $firstEntry")
- assertMetric("lookupCount", metrics.lookupCount, 1)
+ val initialUI = operations.putAndAttach(appId, attemptId, false, started, 0)
+ val firstUI = cache.withSparkUI(appId, attemptId) { ui => ui }
+ assertMetric("lookupCount", metrics.lookupCount, 1)
assert(0 === operations.updateProbeCount, "expected no update probe on that first get")
- val checkTime = window * 2
- clock.setTime(checkTime)
- val entry3 = cache.lookupCacheEntry(appId, attemptId)
- assert(firstEntry !== entry3, s"updated entry test from $cache")
+ // Invalidate the first entry to trigger a re-load.
+ initialUI.invalidate()
+
+ // Update the UI in the stub so that a new one is provided to the cache.
+ operations.putAppUI(appId, attemptId, true, started, started + 10)
+
+ val updatedUI = cache.withSparkUI(appId, attemptId) { ui => ui }
+ assert(firstUI !== updatedUI, s"expected updated UI")
assertMetric("lookupCount", metrics.lookupCount, 2)
- assertMetric("updateProbeCount", metrics.updateProbeCount, 1)
- assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 0)
- assert(1 === operations.updateProbeCount, s"refresh count in $cache")
- assert(0 === operations.detachCount, s"detach count")
- assert(entry3.probeTime === checkTime)
-
- val updateTime = window * 3
- // update the cached value
- val updatedApp = operations.putAppUI(appId, attemptId, true, started, updateTime, updateTime)
- val endTime = window * 10
- clock.setTime(endTime)
- logDebug(s"Before operation = $cache")
- val entry5 = cache.lookupCacheEntry(appId, attemptId)
- assertMetric("lookupCount", metrics.lookupCount, 3)
- assertMetric("updateProbeCount", metrics.updateProbeCount, 2)
- // the update was triggered
- assertMetric("updateTriggeredCount", metrics.updateTriggeredCount, 1)
- assert(updatedApp === entry5.ui, s"UI {$updatedApp} did not match entry {$entry5} in $cache")
-
- // at which point, the refreshes stop
- clock.setTime(window * 20)
- assertCacheEntryEquals(appId, attemptId, entry5)
- assertMetric("updateProbeCount", metrics.updateProbeCount, 2)
+ assert(1 === operations.detachCount, s"detach count")
}
/**
@@ -337,27 +268,6 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
}
}
- /**
- * Look up the cache entry and assert that it matches in the expected value.
- * This assertion works if the two CacheEntries are different -it looks at the fields.
- * UI are compared on object equality; the timestamp and completed flags directly.
- * @param appId application ID
- * @param attemptId attempt ID
- * @param expected expected value
- * @param cache app cache
- */
- def assertCacheEntryEquals(
- appId: String,
- attemptId: Option[String],
- expected: CacheEntry)
- (implicit cache: ApplicationCache): Unit = {
- val actual = cache.lookupCacheEntry(appId, attemptId)
- val errorText = s"Expected get($appId, $attemptId) -> $expected, but got $actual from $cache"
- assert(expected.ui === actual.ui, errorText + " SparkUI reference")
- assert(expected.completed === actual.completed, errorText + " -completed flag")
- assert(expected.probeTime === actual.probeTime, errorText + " -timestamp")
- }
-
/**
* Assert that a key wasn't found in cache or loaded.
*
@@ -370,14 +280,9 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
appId: String,
attemptId: Option[String])
(implicit cache: ApplicationCache): Unit = {
- val ex = intercept[UncheckedExecutionException] {
+ val ex = intercept[NoSuchElementException] {
cache.get(appId, attemptId)
}
- var cause = ex.getCause
- assert(cause !== null)
- if (!cause.isInstanceOf[NoSuchElementException]) {
- throw cause
- }
}
test("Large Scale Application Eviction") {
@@ -385,12 +290,12 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
val clock = new ManualClock(0)
val size = 5
// only two entries are retained, so we expect evictions to occur on lookups
- implicit val cache: ApplicationCache = new TestApplicationCache(operations,
- retainedApplications = size, clock = clock)
+ implicit val cache = new ApplicationCache(operations, retainedApplications = size,
+ clock = clock)
val attempt1 = Some("01")
- val ids = new ListBuffer[String]()
+ val ids = new mutable.ListBuffer[String]()
// build a list of applications
val count = 100
for (i <- 1 to count ) {
@@ -398,7 +303,7 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
ids += appId
clock.advance(10)
val t = clock.getTimeMillis()
- operations.putAppUI(appId, attempt1, true, t, t, t)
+ operations.putAppUI(appId, attempt1, true, t, t)
}
// now go through them in sequence reading them, expect evictions
ids.foreach { id =>
@@ -413,20 +318,19 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
test("Attempts are Evicted") {
val operations = new StubCacheOperations()
- implicit val cache: ApplicationCache = new TestApplicationCache(operations,
- retainedApplications = 4)
+ implicit val cache = new ApplicationCache(operations, 4, new ManualClock())
val metrics = cache.metrics
val appId = "app1"
val attempt1 = Some("01")
val attempt2 = Some("02")
val attempt3 = Some("03")
- operations.putAppUI(appId, attempt1, true, 100, 110, 110)
- operations.putAppUI(appId, attempt2, true, 200, 210, 210)
- operations.putAppUI(appId, attempt3, true, 300, 310, 310)
+ operations.putAppUI(appId, attempt1, true, 100, 110)
+ operations.putAppUI(appId, attempt2, true, 200, 210)
+ operations.putAppUI(appId, attempt3, true, 300, 310)
val attempt4 = Some("04")
- operations.putAppUI(appId, attempt4, true, 400, 410, 410)
+ operations.putAppUI(appId, attempt4, true, 400, 410)
val attempt5 = Some("05")
- operations.putAppUI(appId, attempt5, true, 500, 510, 510)
+ operations.putAppUI(appId, attempt5, true, 500, 510)
def expectLoadAndEvictionCounts(expectedLoad: Int, expectedEvictionCount: Int): Unit = {
assertMetric("loadCount", metrics.loadCount, expectedLoad)
@@ -457,20 +361,14 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar
}
- test("Instantiate Filter") {
- // this is a regression test on the filter being constructable
- val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME)
- val instance = clazz.newInstance()
- instance shouldBe a [Filter]
- }
-
test("redirect includes query params") {
- val clazz = Utils.classForName(ApplicationCacheCheckFilterRelay.FILTER_NAME)
- val filter = clazz.newInstance().asInstanceOf[ApplicationCacheCheckFilter]
- filter.appId = "local-123"
+ val operations = new StubCacheOperations()
+ val ui = operations.putAndAttach("foo", None, true, 0, 10)
val cache = mock[ApplicationCache]
- when(cache.checkForUpdates(any(), any())).thenReturn(true)
- ApplicationCacheCheckFilterRelay.setApplicationCache(cache)
+ when(cache.operations).thenReturn(operations)
+ val filter = new ApplicationCacheCheckFilter(new CacheKey("foo", None), ui, cache)
+ ui.invalidate()
+
val request = mock[HttpServletRequest]
when(request.getMethod()).thenReturn("GET")
when(request.getRequestURI()).thenReturn("http://localhost:18080/history/local-123/jobs/job/")
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 7109146ece371..86c8cdf43258c 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -36,10 +36,12 @@ import org.scalatest.Matchers
import org.scalatest.concurrent.Eventually._
import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
+import org.apache.spark.deploy.history.config._
import org.apache.spark.internal.Logging
import org.apache.spark.io._
import org.apache.spark.scheduler._
import org.apache.spark.security.GroupMappingServiceProvider
+import org.apache.spark.status.AppStatusStore
import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils}
class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging {
@@ -66,9 +68,15 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
new File(logPath)
}
- test("Parse application logs") {
+ Seq(true, false).foreach { inMemory =>
+ test(s"Parse application logs (inMemory = $inMemory)") {
+ testAppLogParsing(inMemory)
+ }
+ }
+
+ private def testAppLogParsing(inMemory: Boolean) {
val clock = new ManualClock(12345678)
- val provider = new FsHistoryProvider(createTestConf(), clock)
+ val provider = new FsHistoryProvider(createTestConf(inMemory = inMemory), clock)
// Write a new-style application log.
val newAppComplete = newLogFile("new1", None, inProgress = false)
@@ -172,20 +180,18 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
)
updateAndCheck(provider) { list =>
list.size should be (1)
- list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should
- endWith(EventLoggingListener.IN_PROGRESS)
+ provider.getAttempt("app1", None).logPath should endWith(EventLoggingListener.IN_PROGRESS)
}
logFile1.renameTo(newLogFile("app1", None, inProgress = false))
updateAndCheck(provider) { list =>
list.size should be (1)
- list.head.attempts.head.asInstanceOf[FsApplicationAttemptInfo].logPath should not
- endWith(EventLoggingListener.IN_PROGRESS)
+ provider.getAttempt("app1", None).logPath should not endWith(EventLoggingListener.IN_PROGRESS)
}
}
test("Parse logs that application is not started") {
- val provider = new FsHistoryProvider((createTestConf()))
+ val provider = new FsHistoryProvider(createTestConf())
val logFile1 = newLogFile("app1", None, inProgress = true)
writeFile(logFile1, true, None,
@@ -342,17 +348,23 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
provider.checkForLogs()
// This should not trigger any cleanup
- updateAndCheck(provider)(list => list.size should be(2))
+ updateAndCheck(provider) { list =>
+ list.size should be(2)
+ }
// Should trigger cleanup for first file but not second one
clock.setTime(firstFileModifiedTime + maxAge + 1)
- updateAndCheck(provider)(list => list.size should be(1))
+ updateAndCheck(provider) { list =>
+ list.size should be(1)
+ }
assert(!log1.exists())
assert(log2.exists())
// Should cleanup the second file as well.
clock.setTime(secondFileModifiedTime + maxAge + 1)
- updateAndCheck(provider)(list => list.size should be(0))
+ updateAndCheck(provider) { list =>
+ list.size should be(0)
+ }
assert(!log1.exists())
assert(!log2.exists())
}
@@ -580,7 +592,71 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
securityManager.checkUIViewPermissions("user4") should be (false)
securityManager.checkUIViewPermissions("user5") should be (false)
}
- }
+ }
+
+ test("mismatched version discards old listing") {
+ val conf = createTestConf()
+ val oldProvider = new FsHistoryProvider(conf)
+
+ val logFile1 = newLogFile("app1", None, inProgress = false)
+ writeFile(logFile1, true, None,
+ SparkListenerLogStart("2.3"),
+ SparkListenerApplicationStart("test", Some("test"), 1L, "test", None),
+ SparkListenerApplicationEnd(5L)
+ )
+
+ updateAndCheck(oldProvider) { list =>
+ list.size should be (1)
+ }
+ assert(oldProvider.listing.count(classOf[ApplicationInfoWrapper]) === 1)
+
+ // Manually overwrite the version in the listing db; this should cause the new provider to
+ // discard all data because the versions don't match.
+ val meta = new FsHistoryProviderMetadata(FsHistoryProvider.CURRENT_LISTING_VERSION + 1,
+ AppStatusStore.CURRENT_VERSION, conf.get(LOCAL_STORE_DIR).get)
+ oldProvider.listing.setMetadata(meta)
+ oldProvider.stop()
+
+ val mistatchedVersionProvider = new FsHistoryProvider(conf)
+ assert(mistatchedVersionProvider.listing.count(classOf[ApplicationInfoWrapper]) === 0)
+ }
+
+ test("invalidate cached UI") {
+ val provider = new FsHistoryProvider(createTestConf())
+ val appId = "new1"
+
+ // Write an incomplete app log.
+ val appLog = newLogFile(appId, None, inProgress = true)
+ writeFile(appLog, true, None,
+ SparkListenerApplicationStart(appId, Some(appId), 1L, "test", None)
+ )
+ provider.checkForLogs()
+
+ // Load the app UI.
+ val oldUI = provider.getAppUI(appId, None)
+ assert(oldUI.isDefined)
+ intercept[NoSuchElementException] {
+ oldUI.get.ui.store.job(0)
+ }
+
+ // Add more info to the app log, and trigger the provider to update things.
+ writeFile(appLog, true, None,
+ SparkListenerApplicationStart(appId, Some(appId), 1L, "test", None),
+ SparkListenerJobStart(0, 1L, Nil, null),
+ SparkListenerApplicationEnd(5L)
+ )
+ provider.checkForLogs()
+
+ // Manually detach the old UI; ApplicationCache would do this automatically in a real SHS
+ // when the app's UI was requested.
+ provider.onUIDetached(appId, None, oldUI.get.ui)
+
+ // Load the UI again and make sure we can get the new info added to the logs.
+ val freshUI = provider.getAppUI(appId, None)
+ assert(freshUI.isDefined)
+ assert(freshUI != oldUI)
+ freshUI.get.ui.store.job(0)
+ }
/**
* Asks the provider to check for logs and calls a function to perform checks on the updated
@@ -623,8 +699,15 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
new FileOutputStream(file).close()
}
- private def createTestConf(): SparkConf = {
- new SparkConf().set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
+ private def createTestConf(inMemory: Boolean = false): SparkConf = {
+ val conf = new SparkConf()
+ .set("spark.history.fs.logDirectory", testDir.getAbsolutePath())
+
+ if (!inMemory) {
+ conf.set(LOCAL_STORE_DIR, Utils.createTempDir().getAbsolutePath())
+ }
+
+ conf
}
private class SafeModeTestProvider(conf: SparkConf, clock: Clock)
diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 18da8c18939ed..010a8dd004d4f 100644
--- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -43,6 +43,7 @@ import org.scalatest.mockito.MockitoSugar
import org.scalatest.selenium.WebBrowser
import org.apache.spark._
+import org.apache.spark.deploy.history.config._
import org.apache.spark.ui.SparkUI
import org.apache.spark.ui.jobs.UIData.JobUIData
import org.apache.spark.util.{ResetSystemProperties, Utils}
@@ -64,16 +65,20 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
private val logDir = getTestResourcePath("spark-events")
private val expRoot = getTestResourceFile("HistoryServerExpectations")
+ private val storeDir = Utils.createTempDir(namePrefix = "history")
private var provider: FsHistoryProvider = null
private var server: HistoryServer = null
private var port: Int = -1
def init(extraConf: (String, String)*): Unit = {
+ Utils.deleteRecursively(storeDir)
+ assert(storeDir.mkdir())
val conf = new SparkConf()
.set("spark.history.fs.logDirectory", logDir)
.set("spark.history.fs.update.interval", "0")
.set("spark.testing", "true")
+ .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath())
conf.setAll(extraConf)
provider = new FsHistoryProvider(conf)
provider.checkForLogs()
@@ -87,14 +92,13 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
def stop(): Unit = {
server.stop()
+ server = null
}
before {
- init()
- }
-
- after{
- stop()
+ if (server == null) {
+ init()
+ }
}
val cases = Seq(
@@ -290,20 +294,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
val uiRoot = "/testwebproxybase"
System.setProperty("spark.ui.proxyBase", uiRoot)
- server.stop()
-
- val conf = new SparkConf()
- .set("spark.history.fs.logDirectory", logDir)
- .set("spark.history.fs.update.interval", "0")
- .set("spark.testing", "true")
-
- provider = new FsHistoryProvider(conf)
- provider.checkForLogs()
- val securityManager = HistoryServer.createSecurityManager(conf)
-
- server = new HistoryServer(conf, provider, securityManager, 18080)
- server.initialize()
- server.bind()
+ stop()
+ init()
val port = server.boundPort
@@ -372,7 +364,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
}
test("incomplete apps get refreshed") {
-
implicit val webDriver: WebDriver = new HtmlUnitDriver
implicit val formats = org.json4s.DefaultFormats
@@ -382,12 +373,14 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
// a new conf is used with the background thread set and running at its fastest
// allowed refresh rate (1Hz)
+ stop()
val myConf = new SparkConf()
.set("spark.history.fs.logDirectory", logDir.getAbsolutePath)
.set("spark.eventLog.dir", logDir.getAbsolutePath)
.set("spark.history.fs.update.interval", "1s")
.set("spark.eventLog.enabled", "true")
.set("spark.history.cache.window", "250ms")
+ .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath())
.remove("spark.testing")
val provider = new FsHistoryProvider(myConf)
val securityManager = HistoryServer.createSecurityManager(myConf)
@@ -413,9 +406,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
}
}
- // stop the server with the old config, and start the new one
- server.stop()
- server = new HistoryServer(myConf, provider, securityManager, 18080)
+ server = new HistoryServer(myConf, provider, securityManager, 0)
server.initialize()
server.bind()
val port = server.boundPort
@@ -461,7 +452,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
rootAppPage should not be empty
def getAppUI: SparkUI = {
- provider.getAppUI(appId, None).get.ui
+ server.withSparkUI(appId, None) { ui => ui }
}
// selenium isn't that useful on failures...add our own reporting
@@ -516,7 +507,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers
getNumJobs("") should be (1)
getNumJobs("/jobs") should be (1)
getNumJobsRestful() should be (1)
- assert(metrics.lookupCount.getCount > 1, s"lookup count too low in $metrics")
+ assert(metrics.lookupCount.getCount > 0, s"lookup count too low in $metrics")
// dump state before the next bit of test, which is where update
// checking really gets stressed
diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
index 70887dc5dd97a..490baf040491f 100644
--- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala
@@ -445,9 +445,9 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach {
"--class", mainClass,
mainJar) ++ appArgs
val args = new SparkSubmitArguments(commandLineArgs)
- val (_, _, sparkProperties, _) = SparkSubmit.prepareSubmitEnvironment(args)
+ val (_, _, sparkConf, _) = SparkSubmit.prepareSubmitEnvironment(args)
new RestSubmissionClient("spark://host:port").constructSubmitRequest(
- mainJar, mainClass, appArgs, sparkProperties.toMap, Map.empty)
+ mainJar, mainClass, appArgs, sparkConf.getAll.toMap, Map.empty)
}
/** Return the response as a submit response, or fail with error otherwise. */
diff --git a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
index 5f699df8211de..c26945fa5fa31 100644
--- a/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
+++ b/core/src/test/scala/org/apache/spark/memory/TestMemoryManager.scala
@@ -27,8 +27,8 @@ class TestMemoryManager(conf: SparkConf)
numBytes: Long,
taskAttemptId: Long,
memoryMode: MemoryMode): Long = {
- if (oomOnce) {
- oomOnce = false
+ if (consequentOOM > 0) {
+ consequentOOM -= 1
0
} else if (available >= numBytes) {
available -= numBytes
@@ -58,11 +58,15 @@ class TestMemoryManager(conf: SparkConf)
override def maxOffHeapStorageMemory: Long = 0L
- private var oomOnce = false
+ private var consequentOOM = 0
private var available = Long.MaxValue
def markExecutionAsOutOfMemoryOnce(): Unit = {
- oomOnce = true
+ markconsequentOOM(1)
+ }
+
+ def markconsequentOOM(n : Int) : Unit = {
+ consequentOOM += n
}
def limit(avail: Long): Unit = {
diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
index 44dd955ce8690..0a248b6064ee8 100644
--- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala
@@ -26,7 +26,7 @@ import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistr
import org.apache.hadoop.conf.{Configurable, Configuration}
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.mapred._
-import org.apache.hadoop.mapreduce.{JobContext => NewJobContext,
+import org.apache.hadoop.mapreduce.{Job => NewJob, JobContext => NewJobContext,
OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat,
RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext}
import org.apache.hadoop.util.Progressable
@@ -568,6 +568,50 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext {
assert(FakeWriterWithCallback.exception.getMessage contains "failed to write")
}
+ test("saveAsNewAPIHadoopDataset should support invalid output paths when " +
+ "there are no files to be committed to an absolute output location") {
+ val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
+
+ def saveRddWithPath(path: String): Unit = {
+ val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration))
+ job.setOutputKeyClass(classOf[Integer])
+ job.setOutputValueClass(classOf[Integer])
+ job.setOutputFormatClass(classOf[NewFakeFormat])
+ if (null != path) {
+ job.getConfiguration.set("mapred.output.dir", path)
+ } else {
+ job.getConfiguration.unset("mapred.output.dir")
+ }
+ val jobConfiguration = job.getConfiguration
+
+ // just test that the job does not fail with java.lang.IllegalArgumentException.
+ pairs.saveAsNewAPIHadoopDataset(jobConfiguration)
+ }
+
+ saveRddWithPath(null)
+ saveRddWithPath("")
+ saveRddWithPath("::invalid::")
+ }
+
+ // In spark 2.1, only null was supported - not other invalid paths.
+ // org.apache.hadoop.mapred.FileOutputFormat.getOutputPath fails with IllegalArgumentException
+ // for non-null invalid paths.
+ test("saveAsHadoopDataset should respect empty output directory when " +
+ "there are no files to be committed to an absolute output location") {
+ val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1)
+
+ val conf = new JobConf()
+ conf.setOutputKeyClass(classOf[Integer])
+ conf.setOutputValueClass(classOf[Integer])
+ conf.setOutputFormat(classOf[FakeOutputFormat])
+ conf.setOutputCommitter(classOf[FakeOutputCommitter])
+
+ FakeOutputCommitter.ran = false
+ pairs.saveAsHadoopDataset(conf)
+
+ assert(FakeOutputCommitter.ran, "OutputCommitter was never called")
+ }
+
test("lookup") {
val pairs = sc.parallelize(Array((1, 2), (3, 4), (5, 6), (5, 7)))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
index f6015cd51c2bd..d3bbfd11d406d 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala
@@ -115,8 +115,9 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM
withBackend(runBackend _) {
val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray)
awaitJobTermination(jobFuture, duration)
- val pattern = ("Aborting TaskSet 0.0 because task .* " +
- "cannot run anywhere due to node and executor blacklist").r
+ val pattern = (
+ s"""|Aborting TaskSet 0.0 because task .*
+ |cannot run anywhere due to node and executor blacklist""".stripMargin).r
assert(pattern.findFirstIn(failure.getMessage).isDefined,
s"Couldn't find $pattern in ${failure.getMessage()}")
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
index a136d69b36d6c..cd1b7a9e5ab18 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala
@@ -110,7 +110,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
val taskSetBlacklist = createTaskSetBlacklist(stageId)
if (stageId % 2 == 0) {
// fail one task in every other taskset
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
failuresSoFar += 1
}
blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures)
@@ -132,7 +133,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// for many different stages, executor 1 fails a task, and then the taskSet fails.
(0 until failuresUntilBlacklisted * 10).foreach { stage =>
val taskSetBlacklist = createTaskSetBlacklist(stage)
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
}
assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
}
@@ -147,7 +149,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
val numFailures = math.max(conf.get(config.MAX_FAILURES_PER_EXEC),
conf.get(config.MAX_FAILURES_PER_EXEC_STAGE))
(0 until numFailures).foreach { index =>
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = index)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = index, failureReason = "testing")
}
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set())
@@ -170,7 +173,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole
// application.
(0 until 4).foreach { partition =>
- taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ taskSetBlacklist0.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures)
assert(blacklist.nodeBlacklist() === Set())
@@ -183,7 +187,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// application. Since that's the second executor that is blacklisted on the same node, we also
// blacklist that node.
(0 until 4).foreach { partition =>
- taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ taskSetBlacklist1.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures)
assert(blacklist.nodeBlacklist() === Set("hostA"))
@@ -207,7 +212,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Fail one more task, but executor isn't put back into blacklist since the count of failures
// on that executor should have been reset to 0.
val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2)
- taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ taskSetBlacklist2.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
blacklist.updateBlacklistForSuccessfulTaskSet(2, 0, taskSetBlacklist2.execToFailures)
assert(blacklist.nodeBlacklist() === Set())
assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
@@ -221,7 +227,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Lets say that executor 1 dies completely. We get some task failures, but
// the taskset then finishes successfully (elsewhere).
(0 until 4).foreach { partition =>
- taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ taskSetBlacklist0.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = partition, failureReason = "testing")
}
blacklist.handleRemovedExecutor("1")
blacklist.updateBlacklistForSuccessfulTaskSet(
@@ -236,7 +243,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Now another executor gets spun up on that host, but it also dies.
val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
(0 until 4).foreach { partition =>
- taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ taskSetBlacklist1.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = partition, failureReason = "testing")
}
blacklist.handleRemovedExecutor("2")
blacklist.updateBlacklistForSuccessfulTaskSet(
@@ -279,7 +287,7 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
def failOneTaskInTaskSet(exec: String): Unit = {
val taskSetBlacklist = createTaskSetBlacklist(stageId = stageId)
- taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0)
+ taskSetBlacklist.updateBlacklistForFailedTask("host-" + exec, exec, 0, "testing")
blacklist.updateBlacklistForSuccessfulTaskSet(stageId, 0, taskSetBlacklist.execToFailures)
stageId += 1
}
@@ -354,12 +362,12 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 2)
// Taskset1 has one failure immediately
- taskSetBlacklist1.updateBlacklistForFailedTask("host-1", "1", 0)
+ taskSetBlacklist1.updateBlacklistForFailedTask("host-1", "1", 0, "testing")
// Then we have a *long* delay, much longer than the timeout, before any other failures or
// taskset completion
clock.advance(blacklist.BLACKLIST_TIMEOUT_MILLIS * 5)
// After the long delay, we have one failure on taskset 2, on the same executor
- taskSetBlacklist2.updateBlacklistForFailedTask("host-1", "1", 0)
+ taskSetBlacklist2.updateBlacklistForFailedTask("host-1", "1", 0, "testing")
// Finally, we complete both tasksets. Its important here to complete taskset2 *first*. We
// want to make sure that when taskset 1 finishes, even though we've now got two task failures,
// we realize that the task failure we just added was well before the timeout.
@@ -377,16 +385,20 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// we blacklist executors on two different hosts -- make sure that doesn't lead to any
// node blacklisting
val taskSetBlacklist0 = createTaskSetBlacklist(stageId = 0)
- taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
- taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = 1)
+ taskSetBlacklist0.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
+ taskSetBlacklist0.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 1, failureReason = "testing")
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures)
assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1"))
verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "1", 2))
assertEquivalentToSet(blacklist.isNodeBlacklisted(_), Set())
val taskSetBlacklist1 = createTaskSetBlacklist(stageId = 1)
- taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 0)
- taskSetBlacklist1.updateBlacklistForFailedTask("hostB", exec = "2", index = 1)
+ taskSetBlacklist1.updateBlacklistForFailedTask(
+ "hostB", exec = "2", index = 0, failureReason = "testing")
+ taskSetBlacklist1.updateBlacklistForFailedTask(
+ "hostB", exec = "2", index = 1, failureReason = "testing")
blacklist.updateBlacklistForSuccessfulTaskSet(1, 0, taskSetBlacklist1.execToFailures)
assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2"))
verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "2", 2))
@@ -395,8 +407,10 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Finally, blacklist another executor on the same node as the original blacklisted executor,
// and make sure this time we *do* blacklist the node.
val taskSetBlacklist2 = createTaskSetBlacklist(stageId = 0)
- taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 0)
- taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "3", index = 1)
+ taskSetBlacklist2.updateBlacklistForFailedTask(
+ "hostA", exec = "3", index = 0, failureReason = "testing")
+ taskSetBlacklist2.updateBlacklistForFailedTask(
+ "hostA", exec = "3", index = 1, failureReason = "testing")
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures)
assertEquivalentToSet(blacklist.isExecutorBlacklisted(_), Set("1", "2", "3"))
verify(listenerBusMock).post(SparkListenerExecutorBlacklisted(0, "3", 2))
@@ -486,7 +500,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole
// application.
(0 until 4).foreach { partition =>
- taskSetBlacklist0.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ taskSetBlacklist0.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist0.execToFailures)
@@ -497,7 +512,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// application. Since that's the second executor that is blacklisted on the same node, we also
// blacklist that node.
(0 until 4).foreach { partition =>
- taskSetBlacklist1.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ taskSetBlacklist1.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist1.execToFailures)
@@ -512,7 +528,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// Fail 4 tasks in one task set on executor 1, so that executor gets blacklisted for the whole
// application.
(0 until 4).foreach { partition =>
- taskSetBlacklist2.updateBlacklistForFailedTask("hostA", exec = "1", index = partition)
+ taskSetBlacklist2.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist2.execToFailures)
@@ -523,7 +540,8 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M
// application. Since that's the second executor that is blacklisted on the same node, we also
// blacklist that node.
(0 until 4).foreach { partition =>
- taskSetBlacklist3.updateBlacklistForFailedTask("hostA", exec = "2", index = partition)
+ taskSetBlacklist3.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = partition, failureReason = "testing")
}
blacklist.updateBlacklistForSuccessfulTaskSet(0, 0, taskSetBlacklist3.execToFailures)
diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index 6b42775ccb0f6..a9e92fa07b9dd 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -228,6 +228,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
SparkListenerStageCompleted,
SparkListenerTaskStart,
SparkListenerTaskEnd,
+ SparkListenerBlockUpdated,
SparkListenerApplicationEnd).map(Utils.getFormattedClassName)
Utils.tryWithSafeFinally {
val logStart = SparkListenerLogStart(SPARK_VERSION)
@@ -291,6 +292,7 @@ object EventLoggingListenerSuite {
def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = {
val conf = new SparkConf
conf.set("spark.eventLog.enabled", "true")
+ conf.set("spark.eventLog.logBlockUpdates.enabled", "true")
conf.set("spark.eventLog.testing", "true")
conf.set("spark.eventLog.dir", logDir.toString)
compressionCodec.foreach { codec =>
diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
index fe6de2bd98850..109d4a0a870b8 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala
@@ -19,8 +19,7 @@ package org.apache.spark.scheduler
import java.util.Properties
-import org.apache.spark.SparkEnv
-import org.apache.spark.TaskContext
+import org.apache.spark.{Partition, SparkEnv, TaskContext}
import org.apache.spark.executor.TaskMetrics
class FakeTask(
@@ -58,4 +57,21 @@ object FakeTask {
}
new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null)
}
+
+ def createShuffleMapTaskSet(
+ numTasks: Int,
+ stageId: Int,
+ stageAttemptId: Int,
+ prefLocs: Seq[TaskLocation]*): TaskSet = {
+ if (prefLocs.size != 0 && prefLocs.size != numTasks) {
+ throw new IllegalArgumentException("Wrong number of task locations")
+ }
+ val tasks = Array.tabulate[Task[_]](numTasks) { i =>
+ new ShuffleMapTask(stageId, stageAttemptId, null, new Partition {
+ override def index: Int = i
+ }, prefLocs(i), new Properties,
+ SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array())
+ }
+ new TaskSet(tasks, stageId, stageAttemptId, priority = 0, null)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
index d061c7845f4a6..1beb36afa95f0 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
@@ -27,7 +27,7 @@ import org.scalatest.Matchers
import org.apache.spark._
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.internal.config.LISTENER_BUS_EVENT_QUEUE_CAPACITY
+import org.apache.spark.internal.config._
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.util.{ResetSystemProperties, RpcUtils}
@@ -446,13 +446,13 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match
classOf[FirehoseListenerThatAcceptsSparkConf],
classOf[BasicJobCounter])
val conf = new SparkConf().setMaster("local").setAppName("test")
- .set("spark.extraListeners", listeners.map(_.getName).mkString(","))
+ .set(EXTRA_LISTENERS, listeners.map(_.getName))
sc = new SparkContext(conf)
sc.listenerBus.listeners.asScala.count(_.isInstanceOf[BasicJobCounter]) should be (1)
sc.listenerBus.listeners.asScala
.count(_.isInstanceOf[ListenerThatAcceptsSparkConf]) should be (1)
sc.listenerBus.listeners.asScala
- .count(_.isInstanceOf[FirehoseListenerThatAcceptsSparkConf]) should be (1)
+ .count(_.isInstanceOf[FirehoseListenerThatAcceptsSparkConf]) should be (1)
}
test("add and remove listeners to/from LiveListenerBus queues") {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
index b8626bf777598..6003899bb7bef 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
@@ -660,9 +660,14 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B
assert(tsm.isZombie)
assert(failedTaskSet)
val idx = failedTask.index
- assert(failedTaskSetReason === s"Aborting TaskSet 0.0 because task $idx (partition $idx) " +
- s"cannot run anywhere due to node and executor blacklist. Blacklisting behavior can be " +
- s"configured via spark.blacklist.*.")
+ assert(failedTaskSetReason === s"""
+ |Aborting $taskSet because task $idx (partition $idx)
+ |cannot run anywhere due to node and executor blacklist.
+ |Most recent failure:
+ |${tsm.taskSetBlacklistHelperOpt.get.getLatestFailureReason}
+ |
+ |Blacklisting behavior can be configured via spark.blacklist.*.
+ |""".stripMargin)
}
test("don't abort if there is an executor available, though it hasn't had scheduled tasks yet") {
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
index f1392e9db6bfd..18981d5be2f94 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala
@@ -37,7 +37,8 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
// First, mark task 0 as failed on exec1.
// task 0 should be blacklisted on exec1, and nowhere else
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "exec1", index = 0, failureReason = "testing")
for {
executor <- (1 to 4).map(_.toString)
index <- 0 until 10
@@ -49,17 +50,20 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Mark task 1 failed on exec1 -- this pushes the executor into the blacklist
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec1", index = 1)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "exec1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Mark one task as failed on exec2 -- not enough for any further blacklisting yet.
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "exec2", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Mark another task as failed on exec2 -- now we blacklist exec2, which also leads to
// blacklisting the entire node.
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "exec2", index = 1)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "exec2", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec1"))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("exec2"))
assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
@@ -108,34 +112,41 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
.set(config.MAX_FAILED_EXEC_PER_NODE_STAGE, 3)
val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock())
// Fail a task twice on hostA, exec:1
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTask("1", 0))
assert(!taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Fail the same task once more on hostA, exec:2
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 0)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = 0, failureReason = "testing")
assert(taskSetBlacklist.isNodeBlacklistedForTask("hostA", 0))
assert(!taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Fail another task on hostA, exec:1. Now that executor has failures on two different tasks,
// so its blacklisted
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Fail a third task on hostA, exec:2, so that exec is blacklisted for the whole task set
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "2", index = 2)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "2", index = 2, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
// Fail a fourth & fifth task on hostA, exec:3. Now we've got three executors that are
// blacklisted for the taskset, so blacklist the whole node.
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 3)
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "3", index = 4)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "3", index = 3, failureReason = "testing")
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "3", index = 4, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("3"))
assert(taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
}
@@ -147,13 +158,17 @@ class TaskSetBlacklistSuite extends SparkFunSuite {
val conf = new SparkConf().setAppName("test").setMaster("local")
.set(config.BLACKLIST_ENABLED.key, "true")
val taskSetBlacklist = new TaskSetBlacklist(conf, stageId = 0, new SystemClock())
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 0)
- taskSetBlacklist.updateBlacklistForFailedTask("hostA", exec = "1", index = 1)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 0, failureReason = "testing")
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostA", exec = "1", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
- taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 0)
- taskSetBlacklist.updateBlacklistForFailedTask("hostB", exec = "2", index = 1)
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostB", exec = "2", index = 0, failureReason = "testing")
+ taskSetBlacklist.updateBlacklistForFailedTask(
+ "hostB", exec = "2", index = 1, failureReason = "testing")
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("1"))
assert(taskSetBlacklist.isExecutorBlacklistedForTaskSet("2"))
assert(!taskSetBlacklist.isNodeBlacklistedForTaskSet("hostA"))
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index ae43f4cadc037..2ce81ae27daf6 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -744,6 +744,113 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
assert(resubmittedTasks === 0)
}
+
+ test("[SPARK-22074] Task killed by other attempt task should not be resubmitted") {
+ val conf = new SparkConf().set("spark.speculation", "true")
+ sc = new SparkContext("local", "test", conf)
+ // Set the speculation multiplier to be 0 so speculative tasks are launched immediately
+ sc.conf.set("spark.speculation.multiplier", "0.0")
+ sc.conf.set("spark.speculation.quantile", "0.5")
+ sc.conf.set("spark.speculation", "true")
+
+ var killTaskCalled = false
+ val sched = new FakeTaskScheduler(sc, ("exec1", "host1"),
+ ("exec2", "host2"), ("exec3", "host3"))
+ sched.initialize(new FakeSchedulerBackend() {
+ override def killTask(
+ taskId: Long,
+ executorId: String,
+ interruptThread: Boolean,
+ reason: String): Unit = {
+ // Check the only one killTask event in this case, which triggered by
+ // task 2.1 completed.
+ assert(taskId === 2)
+ assert(executorId === "exec3")
+ assert(interruptThread)
+ assert(reason === "another attempt succeeded")
+ killTaskCalled = true
+ }
+ })
+
+ // Keep track of the number of tasks that are resubmitted,
+ // so that the test can check that no tasks were resubmitted.
+ var resubmittedTasks = 0
+ val dagScheduler = new FakeDAGScheduler(sc, sched) {
+ override def taskEnded(
+ task: Task[_],
+ reason: TaskEndReason,
+ result: Any,
+ accumUpdates: Seq[AccumulatorV2[_, _]],
+ taskInfo: TaskInfo): Unit = {
+ super.taskEnded(task, reason, result, accumUpdates, taskInfo)
+ reason match {
+ case Resubmitted => resubmittedTasks += 1
+ case _ =>
+ }
+ }
+ }
+ sched.setDAGScheduler(dagScheduler)
+
+ val taskSet = FakeTask.createShuffleMapTaskSet(4, 0, 0,
+ Seq(TaskLocation("host1", "exec1")),
+ Seq(TaskLocation("host1", "exec1")),
+ Seq(TaskLocation("host3", "exec3")),
+ Seq(TaskLocation("host2", "exec2")))
+
+ val clock = new ManualClock()
+ val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock)
+ val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task =>
+ task.metrics.internalAccums
+ }
+ // Offer resources for 4 tasks to start
+ for ((exec, host) <- Seq(
+ "exec1" -> "host1",
+ "exec1" -> "host1",
+ "exec3" -> "host3",
+ "exec2" -> "host2")) {
+ val taskOption = manager.resourceOffer(exec, host, NO_PREF)
+ assert(taskOption.isDefined)
+ val task = taskOption.get
+ assert(task.executorId === exec)
+ // Add an extra assert to make sure task 2.0 is running on exec3
+ if (task.index == 2) {
+ assert(task.attemptNumber === 0)
+ assert(task.executorId === "exec3")
+ }
+ }
+ assert(sched.startedTasks.toSet === Set(0, 1, 2, 3))
+ clock.advance(1)
+ // Complete the 2 tasks and leave 2 task in running
+ for (id <- Set(0, 1)) {
+ manager.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id)))
+ assert(sched.endedTasks(id) === Success)
+ }
+
+ // checkSpeculatableTasks checks that the task runtime is greater than the threshold for
+ // speculating. Since we use a threshold of 0 for speculation, tasks need to be running for
+ // > 0ms, so advance the clock by 1ms here.
+ clock.advance(1)
+ assert(manager.checkSpeculatableTasks(0))
+ assert(sched.speculativeTasks.toSet === Set(2, 3))
+
+ // Offer resource to start the speculative attempt for the running task 2.0
+ val taskOption = manager.resourceOffer("exec2", "host2", ANY)
+ assert(taskOption.isDefined)
+ val task4 = taskOption.get
+ assert(task4.index === 2)
+ assert(task4.taskId === 4)
+ assert(task4.executorId === "exec2")
+ assert(task4.attemptNumber === 1)
+ // Complete the speculative attempt for the running task
+ manager.handleSuccessfulTask(4, createTaskResult(2, accumUpdatesByTask(2)))
+ // Make sure schedBackend.killTask(2, "exec3", true, "another attempt succeeded") gets called
+ assert(killTaskCalled)
+ // Host 3 Losts, there's only task 2.0 on it, which killed by task 2.1
+ manager.executorLost("exec3", "host3", SlaveLost())
+ // Check the resubmittedTasks
+ assert(resubmittedTasks === 0)
+ }
+
test("speculative and noPref task should be scheduled after node-local") {
sc = new SparkContext("local", "test")
sched = new FakeTaskScheduler(
@@ -1146,7 +1253,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg
// Make sure that the blacklist ignored all of the task failures above, since they aren't
// the fault of the executor where the task was running.
verify(blacklist, never())
- .updateBlacklistForFailedTask(anyString(), anyString(), anyInt())
+ .updateBlacklistForFailedTask(anyString(), anyString(), anyInt(), anyString())
}
test("update application blacklist for shuffle-fetch") {
diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
new file mode 100644
index 0000000000000..7ac1ce19f8ddf
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -0,0 +1,703 @@
+/*
+ * 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.status
+
+import java.io.File
+import java.lang.{Integer => JInteger, Long => JLong}
+import java.util.{Arrays, Date, Properties}
+
+import scala.collection.JavaConverters._
+import scala.reflect.{classTag, ClassTag}
+
+import org.scalatest.BeforeAndAfter
+
+import org.apache.spark._
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.scheduler._
+import org.apache.spark.scheduler.cluster._
+import org.apache.spark.status.api.v1
+import org.apache.spark.storage._
+import org.apache.spark.util.Utils
+import org.apache.spark.util.kvstore._
+
+class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
+
+ import config._
+
+ private val conf = new SparkConf().set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
+
+ private var time: Long = _
+ private var testDir: File = _
+ private var store: KVStore = _
+
+ before {
+ time = 0L
+ testDir = Utils.createTempDir()
+ store = KVUtils.open(testDir, getClass().getName())
+ }
+
+ after {
+ store.close()
+ Utils.deleteRecursively(testDir)
+ }
+
+ test("scheduler events") {
+ val listener = new AppStatusListener(store, conf, true)
+
+ // Start the application.
+ time += 1
+ listener.onApplicationStart(SparkListenerApplicationStart(
+ "name",
+ Some("id"),
+ time,
+ "user",
+ Some("attempt"),
+ None))
+
+ check[ApplicationInfoWrapper]("id") { app =>
+ assert(app.info.name === "name")
+ assert(app.info.id === "id")
+ assert(app.info.attempts.size === 1)
+
+ val attempt = app.info.attempts.head
+ assert(attempt.attemptId === Some("attempt"))
+ assert(attempt.startTime === new Date(time))
+ assert(attempt.lastUpdated === new Date(time))
+ assert(attempt.endTime.getTime() === -1L)
+ assert(attempt.sparkUser === "user")
+ assert(!attempt.completed)
+ }
+
+ // Start a couple of executors.
+ time += 1
+ val execIds = Array("1", "2")
+
+ execIds.foreach { id =>
+ listener.onExecutorAdded(SparkListenerExecutorAdded(time, id,
+ new ExecutorInfo(s"$id.example.com", 1, Map())))
+ }
+
+ execIds.foreach { id =>
+ check[ExecutorSummaryWrapper](id) { exec =>
+ assert(exec.info.id === id)
+ assert(exec.info.hostPort === s"$id.example.com")
+ assert(exec.info.isActive)
+ }
+ }
+
+ // Start a job with 2 stages / 4 tasks each
+ time += 1
+ val stages = Seq(
+ new StageInfo(1, 0, "stage1", 4, Nil, Nil, "details1"),
+ new StageInfo(2, 0, "stage2", 4, Nil, Seq(1), "details2"))
+
+ val jobProps = new Properties()
+ jobProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup")
+ jobProps.setProperty("spark.scheduler.pool", "schedPool")
+
+ listener.onJobStart(SparkListenerJobStart(1, time, stages, jobProps))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.jobId === 1)
+ assert(job.info.name === stages.last.name)
+ assert(job.info.description === None)
+ assert(job.info.status === JobExecutionStatus.RUNNING)
+ assert(job.info.submissionTime === Some(new Date(time)))
+ assert(job.info.jobGroup === Some("jobGroup"))
+ }
+
+ stages.foreach { info =>
+ check[StageDataWrapper](key(info)) { stage =>
+ assert(stage.info.status === v1.StageStatus.PENDING)
+ assert(stage.jobIds === Set(1))
+ }
+ }
+
+ // Submit stage 1
+ time += 1
+ stages.head.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stages.head, jobProps))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numActiveStages === 1)
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.status === v1.StageStatus.ACTIVE)
+ assert(stage.info.submissionTime === Some(new Date(stages.head.submissionTime.get)))
+ assert(stage.info.schedulingPool === "schedPool")
+ }
+
+ // Start tasks from stage 1
+ time += 1
+ var _taskIdTracker = -1L
+ def nextTaskId(): Long = {
+ _taskIdTracker += 1
+ _taskIdTracker
+ }
+
+ def createTasks(count: Int, time: Long): Seq[TaskInfo] = {
+ (1 to count).map { id =>
+ val exec = execIds(id.toInt % execIds.length)
+ val taskId = nextTaskId()
+ new TaskInfo(taskId, taskId.toInt, 1, time, exec, s"$exec.example.com",
+ TaskLocality.PROCESS_LOCAL, id % 2 == 0)
+ }
+ }
+
+ val s1Tasks = createTasks(4, time)
+ s1Tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId, task))
+ }
+
+ assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size)
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numActiveTasks === s1Tasks.size)
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.numActiveTasks === s1Tasks.size)
+ assert(stage.info.firstTaskLaunchedTime === Some(new Date(s1Tasks.head.launchTime)))
+ }
+
+ s1Tasks.foreach { task =>
+ check[TaskDataWrapper](task.taskId) { wrapper =>
+ assert(wrapper.info.taskId === task.taskId)
+ assert(wrapper.stageId === stages.head.stageId)
+ assert(wrapper.stageAttemptId === stages.head.attemptId)
+ assert(Arrays.equals(wrapper.stage, Array(stages.head.stageId, stages.head.attemptId)))
+
+ val runtime = Array[AnyRef](stages.head.stageId: JInteger, stages.head.attemptId: JInteger,
+ -1L: JLong)
+ assert(Arrays.equals(wrapper.runtime, runtime))
+
+ assert(wrapper.info.index === task.index)
+ assert(wrapper.info.attempt === task.attemptNumber)
+ assert(wrapper.info.launchTime === new Date(task.launchTime))
+ assert(wrapper.info.executorId === task.executorId)
+ assert(wrapper.info.host === task.host)
+ assert(wrapper.info.status === task.status)
+ assert(wrapper.info.taskLocality === task.taskLocality.toString())
+ assert(wrapper.info.speculative === task.speculative)
+ }
+ }
+
+ // Send executor metrics update. Only update one metric to avoid a lot of boilerplate code.
+ s1Tasks.foreach { task =>
+ val accum = new AccumulableInfo(1L, Some(InternalAccumulator.MEMORY_BYTES_SPILLED),
+ Some(1L), None, true, false, None)
+ listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(
+ task.executorId,
+ Seq((task.taskId, stages.head.stageId, stages.head.attemptId, Seq(accum)))))
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.memoryBytesSpilled === s1Tasks.size)
+ }
+
+ val execs = store.view(classOf[ExecutorStageSummaryWrapper]).index("stage")
+ .first(key(stages.head)).last(key(stages.head)).asScala.toSeq
+ assert(execs.size > 0)
+ execs.foreach { exec =>
+ assert(exec.info.memoryBytesSpilled === s1Tasks.size / 2)
+ }
+
+ // Fail one of the tasks, re-start it.
+ time += 1
+ s1Tasks.head.markFinished(TaskState.FAILED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ "taskType", TaskResultLost, s1Tasks.head, null))
+
+ time += 1
+ val reattempt = {
+ val orig = s1Tasks.head
+ // Task reattempts have a different ID, but the same index as the original.
+ new TaskInfo(nextTaskId(), orig.index, orig.attemptNumber + 1, time, orig.executorId,
+ s"${orig.executorId}.example.com", TaskLocality.PROCESS_LOCAL, orig.speculative)
+ }
+ listener.onTaskStart(SparkListenerTaskStart(stages.head.stageId, stages.head.attemptId,
+ reattempt))
+
+ assert(store.count(classOf[TaskDataWrapper]) === s1Tasks.size + 1)
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numFailedTasks === 1)
+ assert(job.info.numActiveTasks === s1Tasks.size)
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.numFailedTasks === 1)
+ assert(stage.info.numActiveTasks === s1Tasks.size)
+ }
+
+ check[TaskDataWrapper](s1Tasks.head.taskId) { task =>
+ assert(task.info.status === s1Tasks.head.status)
+ assert(task.info.duration === Some(s1Tasks.head.duration))
+ assert(task.info.errorMessage == Some(TaskResultLost.toErrorString))
+ }
+
+ check[TaskDataWrapper](reattempt.taskId) { task =>
+ assert(task.info.index === s1Tasks.head.index)
+ assert(task.info.attempt === reattempt.attemptNumber)
+ }
+
+ // Succeed all tasks in stage 1.
+ val pending = s1Tasks.drop(1) ++ Seq(reattempt)
+
+ val s1Metrics = TaskMetrics.empty
+ s1Metrics.setExecutorCpuTime(2L)
+ s1Metrics.setExecutorRunTime(4L)
+
+ time += 1
+ pending.foreach { task =>
+ task.markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.head.stageId, stages.head.attemptId,
+ "taskType", Success, task, s1Metrics))
+ }
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numFailedTasks === 1)
+ assert(job.info.numActiveTasks === 0)
+ assert(job.info.numCompletedTasks === pending.size)
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.numFailedTasks === 1)
+ assert(stage.info.numActiveTasks === 0)
+ assert(stage.info.numCompleteTasks === pending.size)
+ }
+
+ pending.foreach { task =>
+ check[TaskDataWrapper](task.taskId) { wrapper =>
+ assert(wrapper.info.errorMessage === None)
+ assert(wrapper.info.taskMetrics.get.executorCpuTime === 2L)
+ assert(wrapper.info.taskMetrics.get.executorRunTime === 4L)
+ }
+ }
+
+ assert(store.count(classOf[TaskDataWrapper]) === pending.size + 1)
+
+ // End stage 1.
+ time += 1
+ stages.head.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(stages.head))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numActiveStages === 0)
+ assert(job.info.numCompletedStages === 1)
+ }
+
+ check[StageDataWrapper](key(stages.head)) { stage =>
+ assert(stage.info.status === v1.StageStatus.COMPLETE)
+ assert(stage.info.numFailedTasks === 1)
+ assert(stage.info.numActiveTasks === 0)
+ assert(stage.info.numCompleteTasks === pending.size)
+ }
+
+ // Submit stage 2.
+ time += 1
+ stages.last.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stages.last, jobProps))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numActiveStages === 1)
+ }
+
+ check[StageDataWrapper](key(stages.last)) { stage =>
+ assert(stage.info.status === v1.StageStatus.ACTIVE)
+ assert(stage.info.submissionTime === Some(new Date(stages.last.submissionTime.get)))
+ }
+
+ // Start and fail all tasks of stage 2.
+ time += 1
+ val s2Tasks = createTasks(4, time)
+ s2Tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(stages.last.stageId, stages.last.attemptId, task))
+ }
+
+ time += 1
+ s2Tasks.foreach { task =>
+ task.markFinished(TaskState.FAILED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(stages.last.stageId, stages.last.attemptId,
+ "taskType", TaskResultLost, task, null))
+ }
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numFailedTasks === 1 + s2Tasks.size)
+ assert(job.info.numActiveTasks === 0)
+ }
+
+ check[StageDataWrapper](key(stages.last)) { stage =>
+ assert(stage.info.numFailedTasks === s2Tasks.size)
+ assert(stage.info.numActiveTasks === 0)
+ }
+
+ // Fail stage 2.
+ time += 1
+ stages.last.completionTime = Some(time)
+ stages.last.failureReason = Some("uh oh")
+ listener.onStageCompleted(SparkListenerStageCompleted(stages.last))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numCompletedStages === 1)
+ assert(job.info.numFailedStages === 1)
+ }
+
+ check[StageDataWrapper](key(stages.last)) { stage =>
+ assert(stage.info.status === v1.StageStatus.FAILED)
+ assert(stage.info.numFailedTasks === s2Tasks.size)
+ assert(stage.info.numActiveTasks === 0)
+ assert(stage.info.numCompleteTasks === 0)
+ }
+
+ // - Re-submit stage 2, all tasks, and succeed them and the stage.
+ val oldS2 = stages.last
+ val newS2 = new StageInfo(oldS2.stageId, oldS2.attemptId + 1, oldS2.name, oldS2.numTasks,
+ oldS2.rddInfos, oldS2.parentIds, oldS2.details, oldS2.taskMetrics)
+
+ time += 1
+ newS2.submissionTime = Some(time)
+ listener.onStageSubmitted(SparkListenerStageSubmitted(newS2, jobProps))
+ assert(store.count(classOf[StageDataWrapper]) === 3)
+
+ val newS2Tasks = createTasks(4, time)
+
+ newS2Tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(newS2.stageId, newS2.attemptId, task))
+ }
+
+ time += 1
+ newS2Tasks.foreach { task =>
+ task.markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(newS2.stageId, newS2.attemptId, "taskType", Success,
+ task, null))
+ }
+
+ time += 1
+ newS2.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(newS2))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numActiveStages === 0)
+ assert(job.info.numFailedStages === 1)
+ assert(job.info.numCompletedStages === 2)
+ }
+
+ check[StageDataWrapper](key(newS2)) { stage =>
+ assert(stage.info.status === v1.StageStatus.COMPLETE)
+ assert(stage.info.numActiveTasks === 0)
+ assert(stage.info.numCompleteTasks === newS2Tasks.size)
+ }
+
+ // End job.
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(1, time, JobSucceeded))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.status === JobExecutionStatus.SUCCEEDED)
+ }
+
+ // Submit a second job that re-uses stage 1 and stage 2. Stage 1 won't be re-run, but
+ // stage 2 will. In any case, the DAGScheduler creates new info structures that are copies
+ // of the old stages, so mimic that behavior here. The "new" stage 1 is submitted without
+ // a submission time, which means it is "skipped", and the stage 2 re-execution should not
+ // change the stats of the already finished job.
+ time += 1
+ val j2Stages = Seq(
+ new StageInfo(3, 0, "stage1", 4, Nil, Nil, "details1"),
+ new StageInfo(4, 0, "stage2", 4, Nil, Seq(3), "details2"))
+ j2Stages.last.submissionTime = Some(time)
+ listener.onJobStart(SparkListenerJobStart(2, time, j2Stages, null))
+ assert(store.count(classOf[JobDataWrapper]) === 2)
+
+ listener.onStageSubmitted(SparkListenerStageSubmitted(j2Stages.head, jobProps))
+ listener.onStageCompleted(SparkListenerStageCompleted(j2Stages.head))
+ listener.onStageSubmitted(SparkListenerStageSubmitted(j2Stages.last, jobProps))
+ assert(store.count(classOf[StageDataWrapper]) === 5)
+
+ time += 1
+ val j2s2Tasks = createTasks(4, time)
+
+ j2s2Tasks.foreach { task =>
+ listener.onTaskStart(SparkListenerTaskStart(j2Stages.last.stageId, j2Stages.last.attemptId,
+ task))
+ }
+
+ time += 1
+ j2s2Tasks.foreach { task =>
+ task.markFinished(TaskState.FINISHED, time)
+ listener.onTaskEnd(SparkListenerTaskEnd(j2Stages.last.stageId, j2Stages.last.attemptId,
+ "taskType", Success, task, null))
+ }
+
+ time += 1
+ j2Stages.last.completionTime = Some(time)
+ listener.onStageCompleted(SparkListenerStageCompleted(j2Stages.last))
+
+ time += 1
+ listener.onJobEnd(SparkListenerJobEnd(2, time, JobSucceeded))
+
+ check[JobDataWrapper](1) { job =>
+ assert(job.info.numCompletedStages === 2)
+ assert(job.info.numCompletedTasks === s1Tasks.size + s2Tasks.size)
+ }
+
+ check[JobDataWrapper](2) { job =>
+ assert(job.info.status === JobExecutionStatus.SUCCEEDED)
+ assert(job.info.numCompletedStages === 1)
+ assert(job.info.numCompletedTasks === j2s2Tasks.size)
+ assert(job.info.numSkippedStages === 1)
+ assert(job.info.numSkippedTasks === s1Tasks.size)
+ }
+
+ // Blacklist an executor.
+ time += 1
+ listener.onExecutorBlacklisted(SparkListenerExecutorBlacklisted(time, "1", 42))
+ check[ExecutorSummaryWrapper]("1") { exec =>
+ assert(exec.info.isBlacklisted)
+ }
+
+ time += 1
+ listener.onExecutorUnblacklisted(SparkListenerExecutorUnblacklisted(time, "1"))
+ check[ExecutorSummaryWrapper]("1") { exec =>
+ assert(!exec.info.isBlacklisted)
+ }
+
+ // Blacklist a node.
+ time += 1
+ listener.onNodeBlacklisted(SparkListenerNodeBlacklisted(time, "1.example.com", 2))
+ check[ExecutorSummaryWrapper]("1") { exec =>
+ assert(exec.info.isBlacklisted)
+ }
+
+ time += 1
+ listener.onNodeUnblacklisted(SparkListenerNodeUnblacklisted(time, "1.example.com"))
+ check[ExecutorSummaryWrapper]("1") { exec =>
+ assert(!exec.info.isBlacklisted)
+ }
+
+ // Stop executors.
+ listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "1", "Test"))
+ listener.onExecutorRemoved(SparkListenerExecutorRemoved(41L, "2", "Test"))
+
+ Seq("1", "2").foreach { id =>
+ check[ExecutorSummaryWrapper](id) { exec =>
+ assert(exec.info.id === id)
+ assert(!exec.info.isActive)
+ }
+ }
+
+ // End the application.
+ listener.onApplicationEnd(SparkListenerApplicationEnd(42L))
+
+ check[ApplicationInfoWrapper]("id") { app =>
+ assert(app.info.name === "name")
+ assert(app.info.id === "id")
+ assert(app.info.attempts.size === 1)
+
+ val attempt = app.info.attempts.head
+ assert(attempt.attemptId === Some("attempt"))
+ assert(attempt.startTime === new Date(1L))
+ assert(attempt.lastUpdated === new Date(42L))
+ assert(attempt.endTime === new Date(42L))
+ assert(attempt.duration === 41L)
+ assert(attempt.sparkUser === "user")
+ assert(attempt.completed)
+ }
+ }
+
+ test("storage events") {
+ val listener = new AppStatusListener(store, conf, true)
+ val maxMemory = 42L
+
+ // Register a couple of block managers.
+ val bm1 = BlockManagerId("1", "1.example.com", 42)
+ val bm2 = BlockManagerId("2", "2.example.com", 84)
+ Seq(bm1, bm2).foreach { bm =>
+ listener.onExecutorAdded(SparkListenerExecutorAdded(1L, bm.executorId,
+ new ExecutorInfo(bm.host, 1, Map())))
+ listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm, maxMemory))
+ check[ExecutorSummaryWrapper](bm.executorId) { exec =>
+ assert(exec.info.maxMemory === maxMemory)
+ }
+ }
+
+ val rdd1b1 = RDDBlockId(1, 1)
+ val level = StorageLevel.MEMORY_AND_DISK
+
+ // Submit a stage and make sure the RDD is recorded.
+ val rddInfo = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, Nil)
+ val stage = new StageInfo(1, 0, "stage1", 4, Seq(rddInfo), Nil, "details1")
+ listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties()))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.name === rddInfo.name)
+ assert(wrapper.info.numPartitions === rddInfo.numPartitions)
+ assert(wrapper.info.storageLevel === rddInfo.storageLevel.description)
+ }
+
+ // Add partition 1 replicated on two block managers.
+ listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b1, level, 1L, 1L)))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.memoryUsed === 1L)
+ assert(wrapper.info.diskUsed === 1L)
+
+ assert(wrapper.info.dataDistribution.isDefined)
+ assert(wrapper.info.dataDistribution.get.size === 1)
+
+ val dist = wrapper.info.dataDistribution.get.head
+ assert(dist.address === bm1.hostPort)
+ assert(dist.memoryUsed === 1L)
+ assert(dist.diskUsed === 1L)
+ assert(dist.memoryRemaining === maxMemory - dist.memoryUsed)
+
+ assert(wrapper.info.partitions.isDefined)
+ assert(wrapper.info.partitions.get.size === 1)
+
+ val part = wrapper.info.partitions.get.head
+ assert(part.blockName === rdd1b1.name)
+ assert(part.storageLevel === level.description)
+ assert(part.memoryUsed === 1L)
+ assert(part.diskUsed === 1L)
+ assert(part.executors === Seq(bm1.executorId))
+ }
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 1L)
+ assert(exec.info.memoryUsed === 1L)
+ assert(exec.info.diskUsed === 1L)
+ }
+
+ listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm2, rdd1b1, level, 1L, 1L)))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.memoryUsed === 2L)
+ assert(wrapper.info.diskUsed === 2L)
+ assert(wrapper.info.dataDistribution.get.size === 2L)
+ assert(wrapper.info.partitions.get.size === 1L)
+
+ val dist = wrapper.info.dataDistribution.get.find(_.address == bm2.hostPort).get
+ assert(dist.memoryUsed === 1L)
+ assert(dist.diskUsed === 1L)
+ assert(dist.memoryRemaining === maxMemory - dist.memoryUsed)
+
+ val part = wrapper.info.partitions.get(0)
+ assert(part.memoryUsed === 2L)
+ assert(part.diskUsed === 2L)
+ assert(part.executors === Seq(bm1.executorId, bm2.executorId))
+ }
+
+ check[ExecutorSummaryWrapper](bm2.executorId) { exec =>
+ assert(exec.info.rddBlocks === 1L)
+ assert(exec.info.memoryUsed === 1L)
+ assert(exec.info.diskUsed === 1L)
+ }
+
+ // Add a second partition only to bm 1.
+ val rdd1b2 = RDDBlockId(1, 2)
+ listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b2, level,
+ 3L, 3L)))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.memoryUsed === 5L)
+ assert(wrapper.info.diskUsed === 5L)
+ assert(wrapper.info.dataDistribution.get.size === 2L)
+ assert(wrapper.info.partitions.get.size === 2L)
+
+ val dist = wrapper.info.dataDistribution.get.find(_.address == bm1.hostPort).get
+ assert(dist.memoryUsed === 4L)
+ assert(dist.diskUsed === 4L)
+ assert(dist.memoryRemaining === maxMemory - dist.memoryUsed)
+
+ val part = wrapper.info.partitions.get.find(_.blockName === rdd1b2.name).get
+ assert(part.storageLevel === level.description)
+ assert(part.memoryUsed === 3L)
+ assert(part.diskUsed === 3L)
+ assert(part.executors === Seq(bm1.executorId))
+ }
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 2L)
+ assert(exec.info.memoryUsed === 4L)
+ assert(exec.info.diskUsed === 4L)
+ }
+
+ // Remove block 1 from bm 1.
+ listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm1, rdd1b1,
+ StorageLevel.NONE, 1L, 1L)))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.memoryUsed === 4L)
+ assert(wrapper.info.diskUsed === 4L)
+ assert(wrapper.info.dataDistribution.get.size === 2L)
+ assert(wrapper.info.partitions.get.size === 2L)
+
+ val dist = wrapper.info.dataDistribution.get.find(_.address == bm1.hostPort).get
+ assert(dist.memoryUsed === 3L)
+ assert(dist.diskUsed === 3L)
+ assert(dist.memoryRemaining === maxMemory - dist.memoryUsed)
+
+ val part = wrapper.info.partitions.get.find(_.blockName === rdd1b1.name).get
+ assert(part.storageLevel === level.description)
+ assert(part.memoryUsed === 1L)
+ assert(part.diskUsed === 1L)
+ assert(part.executors === Seq(bm2.executorId))
+ }
+
+ check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+ assert(exec.info.rddBlocks === 1L)
+ assert(exec.info.memoryUsed === 3L)
+ assert(exec.info.diskUsed === 3L)
+ }
+
+ // Remove block 2 from bm 2. This should leave only block 2 info in the store.
+ listener.onBlockUpdated(SparkListenerBlockUpdated(BlockUpdatedInfo(bm2, rdd1b1,
+ StorageLevel.NONE, 1L, 1L)))
+
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper =>
+ assert(wrapper.info.memoryUsed === 3L)
+ assert(wrapper.info.diskUsed === 3L)
+ assert(wrapper.info.dataDistribution.get.size === 1L)
+ assert(wrapper.info.partitions.get.size === 1L)
+ assert(wrapper.info.partitions.get(0).blockName === rdd1b2.name)
+ }
+
+ check[ExecutorSummaryWrapper](bm2.executorId) { exec =>
+ assert(exec.info.rddBlocks === 0L)
+ assert(exec.info.memoryUsed === 0L)
+ assert(exec.info.diskUsed === 0L)
+ }
+
+ // Unpersist RDD1.
+ listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd1b1.rddId))
+ intercept[NoSuchElementException] {
+ check[RDDStorageInfoWrapper](rdd1b1.rddId) { _ => () }
+ }
+
+ }
+
+ private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptId)
+
+ private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = {
+ val value = store.read(classTag[T].runtimeClass, key).asInstanceOf[T]
+ fn(value)
+ }
+
+}
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
index f0c521b00b583..ff4755833a916 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockIdSuite.scala
@@ -35,13 +35,8 @@ class BlockIdSuite extends SparkFunSuite {
}
test("test-bad-deserialization") {
- try {
- // Try to deserialize an invalid block id.
+ intercept[UnrecognizedBlockId] {
BlockId("myblock")
- fail()
- } catch {
- case e: IllegalStateException => // OK
- case _: Throwable => fail()
}
}
@@ -139,6 +134,7 @@ class BlockIdSuite extends SparkFunSuite {
assert(id.id.getMostSignificantBits() === 5)
assert(id.id.getLeastSignificantBits() === 2)
assert(!id.isShuffle)
+ assertSame(id, BlockId(id.toString))
}
test("temp shuffle") {
@@ -151,6 +147,7 @@ class BlockIdSuite extends SparkFunSuite {
assert(id.id.getMostSignificantBits() === 1)
assert(id.id.getLeastSignificantBits() === 2)
assert(!id.isShuffle)
+ assertSame(id, BlockId(id.toString))
}
test("test") {
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
index dd61dcd11bcda..c2101ba828553 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala
@@ -198,55 +198,6 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite
}
}
- test("block replication - deterministic node selection") {
- val blockSize = 1000
- val storeSize = 10000
- val stores = (1 to 5).map {
- i => makeBlockManager(storeSize, s"store$i")
- }
- val storageLevel2x = StorageLevel.MEMORY_AND_DISK_2
- val storageLevel3x = StorageLevel(true, true, false, true, 3)
- val storageLevel4x = StorageLevel(true, true, false, true, 4)
-
- def putBlockAndGetLocations(blockId: String, level: StorageLevel): Set[BlockManagerId] = {
- stores.head.putSingle(blockId, new Array[Byte](blockSize), level)
- val locations = master.getLocations(blockId).sortBy { _.executorId }.toSet
- stores.foreach { _.removeBlock(blockId) }
- master.removeBlock(blockId)
- locations
- }
-
- // Test if two attempts to 2x replication returns same set of locations
- val a1Locs = putBlockAndGetLocations("a1", storageLevel2x)
- assert(putBlockAndGetLocations("a1", storageLevel2x) === a1Locs,
- "Inserting a 2x replicated block second time gave different locations from the first")
-
- // Test if two attempts to 3x replication returns same set of locations
- val a2Locs3x = putBlockAndGetLocations("a2", storageLevel3x)
- assert(putBlockAndGetLocations("a2", storageLevel3x) === a2Locs3x,
- "Inserting a 3x replicated block second time gave different locations from the first")
-
- // Test if 2x replication of a2 returns a strict subset of the locations of 3x replication
- val a2Locs2x = putBlockAndGetLocations("a2", storageLevel2x)
- assert(
- a2Locs2x.subsetOf(a2Locs3x),
- "Inserting a with 2x replication gave locations that are not a subset of locations" +
- s" with 3x replication [3x: ${a2Locs3x.mkString(",")}; 2x: ${a2Locs2x.mkString(",")}"
- )
-
- // Test if 4x replication of a2 returns a strict superset of the locations of 3x replication
- val a2Locs4x = putBlockAndGetLocations("a2", storageLevel4x)
- assert(
- a2Locs3x.subsetOf(a2Locs4x),
- "Inserting a with 4x replication gave locations that are not a superset of locations " +
- s"with 3x replication [3x: ${a2Locs3x.mkString(",")}; 4x: ${a2Locs4x.mkString(",")}"
- )
-
- // Test if 3x replication of two different blocks gives two different sets of locations
- val a3Locs3x = putBlockAndGetLocations("a3", storageLevel3x)
- assert(a3Locs3x !== a2Locs3x, "Two blocks gave same locations with 3x replication")
- }
-
test("block replication - replication failures") {
/*
Create a system of three block managers / stores. One of them (say, failableStore)
diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
index cfe89fde63f88..d45c194d31adc 100644
--- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.storage
-import java.io.File
import java.nio.ByteBuffer
import scala.collection.JavaConverters._
@@ -45,14 +44,14 @@ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
import org.apache.spark.network.client.{RpcResponseCallback, TransportClient}
import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf}
import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempShuffleFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempFileManager}
import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor}
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite}
import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, SerializerManager}
import org.apache.spark.shuffle.sort.SortShuffleManager
-import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
+import org.apache.spark.storage.BlockManagerMessages._
import org.apache.spark.util._
import org.apache.spark.util.io.ChunkedByteBuffer
@@ -512,8 +511,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
when(bmMaster.getLocations(mc.any[BlockId])).thenReturn(Seq(bmId1, bmId2, bmId3))
val blockManager = makeBlockManager(128, "exec", bmMaster)
- val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations)
- val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0))
+ val sortLocations = PrivateMethod[Seq[BlockManagerId]]('sortLocations)
+ val locations = blockManager invokePrivate sortLocations(bmMaster.getLocations("test"))
assert(locations.map(_.host) === Seq(localHost, localHost, otherHost))
}
@@ -535,8 +534,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
val blockManager = makeBlockManager(128, "exec", bmMaster)
blockManager.blockManagerId =
BlockManagerId(SparkContext.DRIVER_IDENTIFIER, localHost, 1, Some(localRack))
- val getLocations = PrivateMethod[Seq[BlockManagerId]]('getLocations)
- val locations = blockManager invokePrivate getLocations(BroadcastBlockId(0))
+ val sortLocations = PrivateMethod[Seq[BlockManagerId]]('sortLocations)
+ val locations = blockManager invokePrivate sortLocations(bmMaster.getLocations("test"))
assert(locations.map(_.host) === Seq(localHost, localHost, otherHost, otherHost, otherHost))
assert(locations.flatMap(_.topologyInfo)
=== Seq(localRack, localRack, localRack, otherRack, otherRack))
@@ -1274,13 +1273,18 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
// so that we have a chance to do location refresh
val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh)
.map { i => BlockManagerId(s"id-$i", s"host-$i", i + 1) }
- when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockManagerIds)
+ when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId])).thenReturn(
+ Option(BlockLocationsAndStatus(blockManagerIds, BlockStatus.empty)))
+ when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(
+ blockManagerIds)
+
store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
transferService = Option(mockBlockTransferService))
val block = store.getRemoteBytes("item")
.asInstanceOf[Option[ByteBuffer]]
assert(block.isDefined)
- verify(mockBlockManagerMaster, times(2)).getLocations("item")
+ verify(mockBlockManagerMaster, times(1)).getLocationsAndStatus("item")
+ verify(mockBlockManagerMaster, times(1)).getLocations("item")
}
test("SPARK-17484: block status is properly updated following an exception in put()") {
@@ -1371,8 +1375,32 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
server.close()
}
+ test("fetch remote block to local disk if block size is larger than threshold") {
+ conf.set(MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM, 1000L)
+
+ val mockBlockManagerMaster = mock(classOf[BlockManagerMaster])
+ val mockBlockTransferService = new MockBlockTransferService(0)
+ val blockLocations = Seq(BlockManagerId("id-0", "host-0", 1))
+ val blockStatus = BlockStatus(StorageLevel.DISK_ONLY, 0L, 2000L)
+
+ when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId])).thenReturn(
+ Option(BlockLocationsAndStatus(blockLocations, blockStatus)))
+ when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations)
+
+ store = makeBlockManager(8000, "executor1", mockBlockManagerMaster,
+ transferService = Option(mockBlockTransferService))
+ val block = store.getRemoteBytes("item")
+ .asInstanceOf[Option[ByteBuffer]]
+
+ assert(block.isDefined)
+ assert(mockBlockTransferService.numCalls === 1)
+ // assert FileManager is not null if the block size is larger than threshold.
+ assert(mockBlockTransferService.tempFileManager === store.remoteBlockTempFileManager)
+ }
+
class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService {
var numCalls = 0
+ var tempFileManager: TempFileManager = null
override def init(blockDataManager: BlockDataManager): Unit = {}
@@ -1382,7 +1410,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
execId: String,
blockIds: Array[String],
listener: BlockFetchingListener,
- tempShuffleFileManager: TempShuffleFileManager): Unit = {
+ tempFileManager: TempFileManager): Unit = {
listener.onBlockFetchSuccess("mockBlockId", new NioManagedBuffer(ByteBuffer.allocate(1)))
}
@@ -1394,7 +1422,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
override def uploadBlock(
hostname: String,
- port: Int, execId: String,
+ port: Int,
+ execId: String,
blockId: BlockId,
blockData: ManagedBuffer,
level: StorageLevel,
@@ -1407,12 +1436,14 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE
host: String,
port: Int,
execId: String,
- blockId: String): ManagedBuffer = {
+ blockId: String,
+ tempFileManager: TempFileManager): ManagedBuffer = {
numCalls += 1
+ this.tempFileManager = tempFileManager
if (numCalls <= maxFailures) {
throw new RuntimeException("Failing block fetch in the mock block transfer service")
}
- super.fetchBlockSync(host, port, execId, blockId)
+ super.fetchBlockSync(host, port, execId, blockId, tempFileManager)
}
}
}
diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
index 7859b0bba2b48..0c4f3c48ef802 100644
--- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.storage
import java.io.{File, FileWriter}
+import java.util.UUID
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
@@ -79,6 +80,12 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
assert(diskBlockManager.getAllBlocks.toSet === ids.toSet)
}
+ test("SPARK-22227: non-block files are skipped") {
+ val file = diskBlockManager.getFile("unmanaged_file")
+ writeToFile(file, 10)
+ assert(diskBlockManager.getAllBlocks().isEmpty)
+ }
+
def writeToFile(file: File, numBytes: Int) {
val writer = new FileWriter(file, true)
for (i <- 0 until numBytes) writer.write(i)
diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
index c371cbcf8dff5..5bfe9905ff17b 100644
--- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
+++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala
@@ -33,7 +33,7 @@ import org.scalatest.PrivateMethodTester
import org.apache.spark.{SparkFunSuite, TaskContext}
import org.apache.spark.network._
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
-import org.apache.spark.network.shuffle.{BlockFetchingListener, TempShuffleFileManager}
+import org.apache.spark.network.shuffle.{BlockFetchingListener, TempFileManager}
import org.apache.spark.network.util.LimitedInputStream
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util.Utils
@@ -437,12 +437,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
val remoteBlocks = Map[BlockId, ManagedBuffer](
ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer())
val transfer = mock(classOf[BlockTransferService])
- var tempShuffleFileManager: TempShuffleFileManager = null
+ var tempFileManager: TempFileManager = null
when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
.thenAnswer(new Answer[Unit] {
override def answer(invocation: InvocationOnMock): Unit = {
val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
- tempShuffleFileManager = invocation.getArguments()(5).asInstanceOf[TempShuffleFileManager]
+ tempFileManager = invocation.getArguments()(5).asInstanceOf[TempFileManager]
Future {
listener.onBlockFetchSuccess(
ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0)))
@@ -472,13 +472,13 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
fetchShuffleBlock(blocksByAddress1)
// `maxReqSizeShuffleToMem` is 200, which is greater than the block size 100, so don't fetch
// shuffle block to disk.
- assert(tempShuffleFileManager == null)
+ assert(tempFileManager == null)
val blocksByAddress2 = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
(remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 300L)).toSeq))
fetchShuffleBlock(blocksByAddress2)
// `maxReqSizeShuffleToMem` is 200, which is smaller than the block size 300, so fetch
// shuffle block to disk.
- assert(tempShuffleFileManager != null)
+ assert(tempFileManager != null)
}
}
diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
index a1a858765a7d4..4abbb8e7894f5 100644
--- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala
@@ -96,6 +96,9 @@ class JsonProtocolSuite extends SparkFunSuite {
.zipWithIndex.map { case (a, i) => a.copy(id = i) }
SparkListenerExecutorMetricsUpdate("exec3", Seq((1L, 2, 3, accumUpdates)))
}
+ val blockUpdated =
+ SparkListenerBlockUpdated(BlockUpdatedInfo(BlockManagerId("Stars",
+ "In your multitude...", 300), RDDBlockId(0, 0), StorageLevel.MEMORY_ONLY, 100L, 0L))
testEvent(stageSubmitted, stageSubmittedJsonString)
testEvent(stageCompleted, stageCompletedJsonString)
@@ -120,6 +123,7 @@ class JsonProtocolSuite extends SparkFunSuite {
testEvent(nodeBlacklisted, nodeBlacklistedJsonString)
testEvent(nodeUnblacklisted, nodeUnblacklistedJsonString)
testEvent(executorMetricsUpdate, executorMetricsUpdateJsonString)
+ testEvent(blockUpdated, blockUpdatedJsonString)
}
test("Dependent Classes") {
@@ -2007,6 +2011,29 @@ private[spark] object JsonProtocolSuite extends Assertions {
|}
""".stripMargin
+ private val blockUpdatedJsonString =
+ """
+ |{
+ | "Event": "SparkListenerBlockUpdated",
+ | "Block Updated Info": {
+ | "Block Manager ID": {
+ | "Executor ID": "Stars",
+ | "Host": "In your multitude...",
+ | "Port": 300
+ | },
+ | "Block ID": "rdd_0_0",
+ | "Storage Level": {
+ | "Use Disk": false,
+ | "Use Memory": true,
+ | "Deserialized": true,
+ | "Replication": 1
+ | },
+ | "Memory Size": 100,
+ | "Disk Size": 0
+ | }
+ |}
+ """.stripMargin
+
private val executorBlacklistedJsonString =
s"""
|{
diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
index 2b16cc4852ba8..4d3adeb968e84 100644
--- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
+++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
@@ -38,9 +38,10 @@ import org.apache.commons.math3.stat.inference.ChiSquareTest
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext}
+import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.network.util.ByteUnit
+import org.apache.spark.scheduler.SparkListener
class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
@@ -1110,4 +1111,57 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
Utils.tryWithSafeFinallyAndFailureCallbacks {}(catchBlock = {}, finallyBlock = {})
TaskContext.unset
}
+
+ test("load extensions") {
+ val extensions = Seq(
+ classOf[SimpleExtension],
+ classOf[ExtensionWithConf],
+ classOf[UnregisterableExtension]).map(_.getName())
+
+ val conf = new SparkConf(false)
+ val instances = Utils.loadExtensions(classOf[Object], extensions, conf)
+ assert(instances.size === 2)
+ assert(instances.count(_.isInstanceOf[SimpleExtension]) === 1)
+
+ val extWithConf = instances.find(_.isInstanceOf[ExtensionWithConf])
+ .map(_.asInstanceOf[ExtensionWithConf])
+ .get
+ assert(extWithConf.conf eq conf)
+
+ class NestedExtension { }
+
+ val invalid = Seq(classOf[NestedExtension].getName())
+ intercept[SparkException] {
+ Utils.loadExtensions(classOf[Object], invalid, conf)
+ }
+
+ val error = Seq(classOf[ExtensionWithError].getName())
+ intercept[IllegalArgumentException] {
+ Utils.loadExtensions(classOf[Object], error, conf)
+ }
+
+ val wrongType = Seq(classOf[ListenerImpl].getName())
+ intercept[IllegalArgumentException] {
+ Utils.loadExtensions(classOf[Seq[_]], wrongType, conf)
+ }
+ }
+
+}
+
+private class SimpleExtension
+
+private class ExtensionWithConf(val conf: SparkConf)
+
+private class UnregisterableExtension {
+
+ throw new UnsupportedOperationException()
+
+}
+
+private class ExtensionWithError {
+
+ throw new IllegalArgumentException()
+
}
+
+private class ListenerImpl extends SparkListener
diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml
index 6e15f6955984e..bbda824dd13b4 100644
--- a/dev/checkstyle-suppressions.xml
+++ b/dev/checkstyle-suppressions.xml
@@ -40,8 +40,6 @@
files="src/main/java/org/apache/hive/service/*"/>
-
-
diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh
index 8de1d6a37dc25..7e8d5c7075195 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -74,7 +74,7 @@ GIT_REF=${GIT_REF:-master}
# Destination directory parent on remote server
REMOTE_PARENT_DIR=${REMOTE_PARENT_DIR:-/home/$ASF_USERNAME/public_html}
-GPG="gpg --no-tty --batch"
+GPG="gpg -u $GPG_KEY --no-tty --batch"
NEXUS_ROOT=https://repository.apache.org/service/local/staging
NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads
BASE_DIR=$(pwd)
@@ -84,9 +84,9 @@ MVN="build/mvn --force"
# Hive-specific profiles for some builds
HIVE_PROFILES="-Phive -Phive-thriftserver"
# Profiles for publishing snapshots and release to Maven Central
-PUBLISH_PROFILES="-Pmesos -Pyarn $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
+PUBLISH_PROFILES="-Pmesos -Pyarn -Pflume $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl"
# Profiles for building binary releases
-BASE_RELEASE_PROFILES="-Pmesos -Pyarn -Psparkr"
+BASE_RELEASE_PROFILES="-Pmesos -Pyarn -Pflume -Psparkr"
# Scala 2.11 only profiles for some builds
SCALA_2_11_PROFILES="-Pkafka-0-8"
# Scala 2.12 only profiles for some builds
@@ -125,7 +125,7 @@ else
echo "Please set JAVA_HOME correctly."
exit 1
else
- JAVA_HOME="$JAVA_7_HOME"
+ export JAVA_HOME="$JAVA_7_HOME"
fi
fi
fi
@@ -140,7 +140,7 @@ DEST_DIR_NAME="spark-$SPARK_PACKAGE_VERSION"
function LFTP {
SSH="ssh -o ConnectTimeout=300 -o StrictHostKeyChecking=no -i $ASF_RSA_KEY"
COMMANDS=$(cat <= (2, 7):
- subprocess_check_output = subprocess.check_output
- subprocess_check_call = subprocess.check_call
-else:
- # SPARK-8763
- # backported from subprocess module in Python 2.7
- def subprocess_check_output(*popenargs, **kwargs):
- if 'stdout' in kwargs:
- raise ValueError('stdout argument not allowed, it will be overridden.')
- process = subprocess.Popen(stdout=subprocess.PIPE, *popenargs, **kwargs)
- output, unused_err = process.communicate()
- retcode = process.poll()
- if retcode:
- cmd = kwargs.get("args")
- if cmd is None:
- cmd = popenargs[0]
- raise subprocess.CalledProcessError(retcode, cmd, output=output)
- return output
-
- # backported from subprocess module in Python 2.7
- def subprocess_check_call(*popenargs, **kwargs):
- retcode = call(*popenargs, **kwargs)
- if retcode:
- cmd = kwargs.get("args")
- if cmd is None:
- cmd = popenargs[0]
- raise CalledProcessError(retcode, cmd)
- return 0
+subprocess_check_output = subprocess.check_output
+subprocess_check_call = subprocess.check_call
def exit_from_command_with_retcode(cmd, retcode):
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index c7714578bd005..58b295d4f6e00 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -29,7 +29,7 @@ export LC_ALL=C
# TODO: This would be much nicer to do in SBT, once SBT supports Maven-style resolution.
# NOTE: These should match those in the release publishing script
-HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pyarn -Phive"
+HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pyarn -Pflume -Phive"
MVN="build/mvn"
HADOOP_PROFILES=(
hadoop-2.6
diff --git a/docs/building-spark.md b/docs/building-spark.md
index 57baa503259c1..98f7df155456f 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -100,6 +100,13 @@ Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.
Kafka 0.10 support is still automatically built.
+## Building with Flume support
+
+Apache Flume support must be explicitly enabled with the `flume` profile.
+Note: Flume support is deprecated as of Spark 2.3.0.
+
+ ./build/mvn -Pflume -DskipTests clean package
+
## Building submodules individually
It's possible to build Spark sub-modules using the `mvn -pl` option.
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index a2ad958959a50..c42bb4bb8377e 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -58,6 +58,9 @@ for providing container-centric infrastructure. Kubernetes support is being acti
developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization.
For documentation, refer to that project's README.
+A third-party project (not supported by the Spark project) exists to add support for
+[Nomad](https://github.com/hashicorp/nomad-spark) as a cluster manager.
+
# Submitting Applications
Applications can be submitted to a cluster of any type using the `spark-submit` script.
diff --git a/docs/configuration.md b/docs/configuration.md
index 6e9fe591b70a3..d3c358bb74173 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -547,13 +547,14 @@ Apart from these, the following properties are also available, and may be useful
- spark.reducer.maxReqSizeShuffleToMem |
+ spark.maxRemoteBlockSizeFetchToMem |
Long.MaxValue |
- The blocks of a shuffle request will be fetched to disk when size of the request is above
- this threshold. This is to avoid a giant request takes too much memory. We can enable this
- config by setting a specific value(e.g. 200m). Note that this config can be enabled only when
- the shuffle shuffle service is newer than Spark-2.2 or the shuffle service is disabled.
+ The remote block will be fetched to disk when size of the block is above this threshold.
+ This is to avoid a giant request takes too much memory. We can enable this config by setting
+ a specific value(e.g. 200m). Note this configuration will affect both shuffle fetch
+ and block manager remote block fetch. For users who enabled external shuffle service,
+ this feature can only be worked when external shuffle service is newer than Spark 2.2.
|
@@ -713,6 +714,14 @@ Apart from these, the following properties are also available, and may be useful
| Property Name | Default | Meaning |
+
+ spark.eventLog.logBlockUpdates.enabled |
+ false |
+
+ Whether to log events for every block update, if spark.eventLog.enabled is true.
+ *Warning*: This will increase the size of the event log considerably.
+ |
+
spark.eventLog.compress |
false |
@@ -739,6 +748,20 @@ Apart from these, the following properties are also available, and may be useful
finished.
+
+ spark.eventLog.overwrite |
+ false |
+
+ Whether to overwrite any existing files.
+ |
+
+
+ spark.eventLog.buffer.kb |
+ 100k |
+
+ Buffer size in KB to use when writing to output streams.
+ |
+
spark.ui.enabled |
true |
@@ -1015,7 +1038,7 @@ Apart from these, the following properties are also available, and may be useful
0.5 |
Amount of storage memory immune to eviction, expressed as a fraction of the size of the
- region set aside by s​park.memory.fraction. The higher this is, the less
+ region set aside by spark.memory.fraction. The higher this is, the less
working memory may be available to execution and tasks may spill to disk more often.
Leaving this at the default value is recommended. For more detail, see
this description.
@@ -1041,7 +1064,7 @@ Apart from these, the following properties are also available, and may be useful
| spark.memory.useLegacyMode |
false |
- ​Whether to enable the legacy memory management mode used in Spark 1.5 and before.
+ Whether to enable the legacy memory management mode used in Spark 1.5 and before.
The legacy mode rigidly partitions the heap space into fixed-size regions,
potentially leading to excessive spilling if the application was not tuned.
The following deprecated memory fraction configurations are not read unless this is enabled:
@@ -1115,11 +1138,8 @@ Apart from these, the following properties are also available, and may be useful
|
The number of cores to use on each executor.
- In standalone and Mesos coarse-grained modes, setting this
- parameter allows an application to run multiple executors on the
- same worker, provided that there are enough cores on that
- worker. Otherwise, only one executor per application will run on
- each worker.
+ In standalone and Mesos coarse-grained modes, for more detail, see
+ this description.
|
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 51084a25983ea..1ae43185d22f8 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -224,6 +224,15 @@ The history server can be configured as follows:
Number of threads that will be used by history server to process event logs.
+
+ | spark.history.store.path |
+ (none) |
+
+ Local directory where to cache application history data. If set, the history
+ server will store application data on disk instead of keeping it in memory. The data
+ written to disk will be re-used in the event of a history server restart.
+ |
+
Note that in all of these UIs, the tables are sortable by clicking their headers,
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index 432639588cc2b..9599d40c545b2 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -401,6 +401,15 @@ To use a custom metrics.properties for the application master and executors, upd
Principal to be used to login to KDC, while running on secure HDFS. (Works also with the "local" master)
+
+ spark.yarn.kerberos.relogin.period |
+ 1m |
+
+ How often to check whether the kerberos TGT should be renewed. This should be set to a value
+ that is shorter than the TGT renewal period (or the TGT lifetime if TGT renewal is not enabled).
+ The default value should be enough for most deployments.
+ |
+
spark.yarn.config.gatewayPath |
(none) |
diff --git a/docs/security.md b/docs/security.md
index 1d004003f9a32..15aadf07cf873 100644
--- a/docs/security.md
+++ b/docs/security.md
@@ -186,7 +186,54 @@ configure those ports.
+### HTTP Security Headers
+
+Apache Spark can be configured to include HTTP Headers which aids in preventing Cross
+Site Scripting (XSS), Cross-Frame Scripting (XFS), MIME-Sniffing and also enforces HTTP
+Strict Transport Security.
+
+
+| Property Name | Default | Meaning |
+
+ spark.ui.xXssProtection |
+ 1; mode=block |
+
+ Value for HTTP X-XSS-Protection response header. You can choose appropriate value
+ from below:
+
+ 0 (Disables XSS filtering)
+ 1 (Enables XSS filtering. If a cross-site scripting attack is detected,
+ the browser will sanitize the page.)
+ 1; mode=block (Enables XSS filtering. The browser will prevent rendering
+ of the page if an attack is detected.)
+
+ |
+
+
+ spark.ui.xContentTypeOptions.enabled |
+ true |
+
+ When value is set to "true", X-Content-Type-Options HTTP response header will be set
+ to "nosniff". Set "false" to disable.
+ |
+
+
+ spark.ui.strictTransportSecurity |
+ None |
+
+ Value for HTTP Strict Transport Security (HSTS) Response Header. You can choose appropriate
+ value from below and set expire-time accordingly, when Spark is SSL/TLS enabled.
+
+ max-age=<expire-time>
+ max-age=<expire-time>; includeSubDomains
+ max-age=<expire-time>; preload
+
+ |
+
+
+
See the [configuration page](configuration.html) for more details on the security configuration
parameters, and
org.apache.spark.SecurityManager for implementation details about security.
+
diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md
index 1095386c31ab8..f51c5cc38f4de 100644
--- a/docs/spark-standalone.md
+++ b/docs/spark-standalone.md
@@ -328,6 +328,14 @@ export SPARK_MASTER_OPTS="-Dspark.deploy.defaultCores="
This is useful on shared clusters where users might not have configured a maximum number of cores
individually.
+# Executors Scheduling
+
+The number of cores assigned to each executor is configurable. When `spark.executor.cores` is
+explicitly set, multiple executors from the same application may be launched on the same worker
+if the worker has enough cores and memory. Otherwise, each executor grabs all the cores available
+on the worker by default, in which case only one executor per application may be launched on each
+worker during one single schedule iteration.
+
# Monitoring and Logging
Spark's standalone mode offers a web-based user interface to monitor the cluster. The master and each worker has its own web UI that shows cluster and job statistics. By default you can access the web UI for the master at port 8080. The port can be changed either in the configuration file or via command-line options.
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index a095263bfa619..639a8ea7bb8ad 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -461,6 +461,8 @@ name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can al
names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data
source type can be converted into other types using this syntax.
+To load a JSON file you can use:
+
{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
@@ -479,6 +481,26 @@ source type can be converted into other types using this syntax.
+To load a CSV file you can use:
+
+
+
+{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %}
+
+
+
+{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %}
+
+
+
+{% include_example manual_load_options_csv python/sql/datasource.py %}
+
+
+
+{% include_example manual_load_options_csv r/RSparkSQLExample.R %}
+
+
+
### Run SQL on files directly
Instead of using read API to load a file into DataFrame and query it, you can also query that
@@ -573,7 +595,7 @@ Note that partition information is not gathered by default when creating externa
### Bucketing, Sorting and Partitioning
-For file-based data source, it is also possible to bucket and sort or partition the output.
+For file-based data source, it is also possible to bucket and sort or partition the output.
Bucketing and sorting are applicable only to persistent tables:
@@ -598,7 +620,7 @@ CREATE TABLE users_bucketed_by_name(
name STRING,
favorite_color STRING,
favorite_numbers array
-) USING parquet
+) USING parquet
CLUSTERED BY(name) INTO 42 BUCKETS;
{% endhighlight %}
@@ -629,7 +651,7 @@ while partitioning can be used with both `save` and `saveAsTable` when using the
{% highlight sql %}
CREATE TABLE users_by_favorite_color(
- name STRING,
+ name STRING,
favorite_color STRING,
favorite_numbers array
) USING csv PARTITIONED BY(favorite_color);
@@ -664,7 +686,7 @@ CREATE TABLE users_bucketed_and_partitioned(
name STRING,
favorite_color STRING,
favorite_numbers array
-) USING parquet
+) USING parquet
PARTITIONED BY (favorite_color)
CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS;
@@ -675,7 +697,7 @@ CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS;
`partitionBy` creates a directory structure as described in the [Partition Discovery](#partition-discovery) section.
-Thus, it has limited applicability to columns with high cardinality. In contrast
+Thus, it has limited applicability to columns with high cardinality. In contrast
`bucketBy` distributes
data across a fixed number of buckets and can be used when a number of unique values is unbounded.
diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md
index a5d36da5b6de9..257a4f7d4f3ca 100644
--- a/docs/streaming-flume-integration.md
+++ b/docs/streaming-flume-integration.md
@@ -5,6 +5,8 @@ title: Spark Streaming + Flume Integration Guide
[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this.
+**Note: Flume support is deprecated as of Spark 2.3.0.**
+
## Approach 1: Flume-style Push-based Approach
Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps.
@@ -44,8 +46,7 @@ configuring Flume agents.
val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port])
- See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$)
- and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala).
+ See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$).
import org.apache.spark.streaming.flume.*;
@@ -53,8 +54,7 @@ configuring Flume agents.
JavaReceiverInputDStream flumeStream =
FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]);
- See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html)
- and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java).
+ See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html).
from pyspark.streaming.flume import FlumeUtils
@@ -62,8 +62,7 @@ configuring Flume agents.
flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port])
By default, the Python API will decode Flume event body as UTF8 encoded strings. You can specify your custom decoding function to decode the body byte arrays in Flume events to any arbitrary data type.
- See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils)
- and the [example]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/python/streaming/flume_wordcount.py).
+ See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils).
@@ -162,8 +161,6 @@ configuring Flume agents.
- See the Scala example [FlumePollingEventCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala).
-
Note that each input DStream can be configured to receive data from multiple sinks.
3. **Deploying:** This is same as the first approach.
diff --git a/examples/pom.xml b/examples/pom.xml
index 52a6764ae26a5..1791dbaad775e 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -34,7 +34,6 @@