- * The key and value must be word-aligned (that is, their sizes must multiples of 8).
+ * The key and value must be word-aligned (that is, their sizes must be a multiple of 8).
*
*
* After calling this method, calls to `get[Key|Value]Address()` and `get[Key|Value]Length`
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
index 4fc19b1721518..5056652a2420b 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java
@@ -402,7 +402,7 @@ public void insertRecord(
growPointerArrayIfNecessary();
int uaoSize = UnsafeAlignedOffset.getUaoSize();
- // Need 4 bytes to store the record length.
+ // Need 4 or 8 bytes to store the record length.
final int required = length + uaoSize;
acquireNewPageIfNecessary(required);
@@ -544,7 +544,7 @@ public long spill() throws IOException {
// is accessing the current record. We free this page in that caller's next loadNext()
// call.
for (MemoryBlock page : allocatedPages) {
- if (!loaded || page.getPageNumber() !=
+ if (!loaded || page.pageNumber !=
((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) {
released += page.size();
freePage(page);
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
index 717823ebbd320..75690ae264838 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java
@@ -26,6 +26,7 @@
import org.apache.spark.memory.MemoryConsumer;
import org.apache.spark.memory.SparkOutOfMemoryError;
import org.apache.spark.memory.TaskMemoryManager;
+import org.apache.spark.unsafe.Platform;
import org.apache.spark.unsafe.UnsafeAlignedOffset;
import org.apache.spark.unsafe.array.LongArray;
import org.apache.spark.unsafe.memory.MemoryBlock;
@@ -215,7 +216,12 @@ public void expandPointerArray(LongArray newArray) {
if (newArray.size() < array.size()) {
throw new SparkOutOfMemoryError("Not enough memory to grow pointer array");
}
- MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(), pos * 8L);
+ Platform.copyMemory(
+ array.getBaseObject(),
+ array.getBaseOffset(),
+ newArray.getBaseObject(),
+ newArray.getBaseOffset(),
+ pos * 8L);
consumer.freeArray(array);
array = newArray;
usableCapacity = getUsableCapacity();
@@ -342,7 +348,10 @@ public UnsafeSorterIterator getSortedIterator() {
array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7,
radixSortSupport.sortDescending(), radixSortSupport.sortSigned());
} else {
- MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L, (array.size() - pos) * 8L);
+ MemoryBlock unused = new MemoryBlock(
+ array.getBaseObject(),
+ array.getBaseOffset() + pos * 8L,
+ (array.size() - pos) * 8L);
LongArray buffer = new LongArray(unused);
Sorter sorter =
new Sorter<>(new UnsafeSortDataFormat(buffer));
diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
index ff0dcc259a4ad..ab800288dcb43 100644
--- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
+++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillMerger.java
@@ -51,7 +51,7 @@ public void addSpillIfNotEmpty(UnsafeSorterIterator spillReader) throws IOExcept
if (spillReader.hasNext()) {
// We only add the spillReader to the priorityQueue if it is not empty. We do this to
// make sure the hasNext method of UnsafeSorterIterator returned by getSortedIterator
- // does not return wrong result because hasNext will returns true
+ // does not return wrong result because hasNext will return true
// at least priorityQueue.size() times. If we allow n spillReaders in the
// priorityQueue, we will have n extra empty records in the result of UnsafeSorterIterator.
spillReader.loadNext();
diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala
new file mode 100644
index 0000000000000..6439ca5db06e9
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala
@@ -0,0 +1,236 @@
+/*
+ * 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
+
+import java.util.{Timer, TimerTask}
+import java.util.concurrent.ConcurrentHashMap
+import java.util.function.{Consumer, Function}
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint}
+import org.apache.spark.scheduler.{LiveListenerBus, SparkListener, SparkListenerStageCompleted}
+
+/**
+ * For each barrier stage attempt, only at most one barrier() call can be active at any time, thus
+ * we can use (stageId, stageAttemptId) to identify the stage attempt where the barrier() call is
+ * from.
+ */
+private case class ContextBarrierId(stageId: Int, stageAttemptId: Int) {
+ override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)"
+}
+
+/**
+ * A coordinator that handles all global sync requests from BarrierTaskContext. Each global sync
+ * request is generated by `BarrierTaskContext.barrier()`, and identified by
+ * stageId + stageAttemptId + barrierEpoch. Reply all the blocking global sync requests upon
+ * all the requests for a group of `barrier()` calls are received. If the coordinator is unable to
+ * collect enough global sync requests within a configured time, fail all the requests and return
+ * an Exception with timeout message.
+ */
+private[spark] class BarrierCoordinator(
+ timeoutInSecs: Long,
+ listenerBus: LiveListenerBus,
+ override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging {
+
+ // TODO SPARK-25030 Create a Timer() in the mainClass submitted to SparkSubmit makes it unable to
+ // fetch result, we shall fix the issue.
+ private lazy val timer = new Timer("BarrierCoordinator barrier epoch increment timer")
+
+ // Listen to StageCompleted event, clear corresponding ContextBarrierState.
+ private val listener = new SparkListener {
+ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
+ val stageInfo = stageCompleted.stageInfo
+ val barrierId = ContextBarrierId(stageInfo.stageId, stageInfo.attemptNumber)
+ // Clear ContextBarrierState from a finished stage attempt.
+ cleanupBarrierStage(barrierId)
+ }
+ }
+
+ // Record all active stage attempts that make barrier() call(s), and the corresponding internal
+ // state.
+ private val states = new ConcurrentHashMap[ContextBarrierId, ContextBarrierState]
+
+ override def onStart(): Unit = {
+ super.onStart()
+ listenerBus.addToStatusQueue(listener)
+ }
+
+ override def onStop(): Unit = {
+ try {
+ states.forEachValue(1, clearStateConsumer)
+ states.clear()
+ listenerBus.removeListener(listener)
+ } finally {
+ super.onStop()
+ }
+ }
+
+ /**
+ * Provide the current state of a barrier() call. A state is created when a new stage attempt
+ * sends out a barrier() call, and recycled on stage completed.
+ *
+ * @param barrierId Identifier of the barrier stage that make a barrier() call.
+ * @param numTasks Number of tasks of the barrier stage, all barrier() calls from the stage shall
+ * collect `numTasks` requests to succeed.
+ */
+ private class ContextBarrierState(
+ val barrierId: ContextBarrierId,
+ val numTasks: Int) {
+
+ // There may be multiple barrier() calls from a barrier stage attempt, `barrierEpoch` is used
+ // to identify each barrier() call. It shall get increased when a barrier() call succeeds, or
+ // reset when a barrier() call fails due to timeout.
+ private var barrierEpoch: Int = 0
+
+ // An array of RPCCallContexts for barrier tasks that are waiting for reply of a barrier()
+ // call.
+ private val requesters: ArrayBuffer[RpcCallContext] = new ArrayBuffer[RpcCallContext](numTasks)
+
+ // A timer task that ensures we may timeout for a barrier() call.
+ private var timerTask: TimerTask = null
+
+ // Init a TimerTask for a barrier() call.
+ private def initTimerTask(): Unit = {
+ timerTask = new TimerTask {
+ override def run(): Unit = synchronized {
+ // Timeout current barrier() call, fail all the sync requests.
+ requesters.foreach(_.sendFailure(new SparkException("The coordinator didn't get all " +
+ s"barrier sync requests for barrier epoch $barrierEpoch from $barrierId within " +
+ s"$timeoutInSecs second(s).")))
+ cleanupBarrierStage(barrierId)
+ }
+ }
+ }
+
+ // Cancel the current active TimerTask and release resources.
+ private def cancelTimerTask(): Unit = {
+ if (timerTask != null) {
+ timerTask.cancel()
+ timer.purge()
+ timerTask = null
+ }
+ }
+
+ // Process the global sync request. The barrier() call succeed if collected enough requests
+ // within a configured time, otherwise fail all the pending requests.
+ def handleRequest(requester: RpcCallContext, request: RequestToSync): Unit = synchronized {
+ val taskId = request.taskAttemptId
+ val epoch = request.barrierEpoch
+
+ // Require the number of tasks is correctly set from the BarrierTaskContext.
+ require(request.numTasks == numTasks, s"Number of tasks of $barrierId is " +
+ s"${request.numTasks} from Task $taskId, previously it was $numTasks.")
+
+ // Check whether the epoch from the barrier tasks matches current barrierEpoch.
+ logInfo(s"Current barrier epoch for $barrierId is $barrierEpoch.")
+ if (epoch != barrierEpoch) {
+ requester.sendFailure(new SparkException(s"The request to sync of $barrierId with " +
+ s"barrier epoch $barrierEpoch has already finished. Maybe task $taskId is not " +
+ "properly killed."))
+ } else {
+ // If this is the first sync message received for a barrier() call, start timer to ensure
+ // we may timeout for the sync.
+ if (requesters.isEmpty) {
+ initTimerTask()
+ timer.schedule(timerTask, timeoutInSecs * 1000)
+ }
+ // Add the requester to array of RPCCallContexts pending for reply.
+ requesters += requester
+ logInfo(s"Barrier sync epoch $barrierEpoch from $barrierId received update from Task " +
+ s"$taskId, current progress: ${requesters.size}/$numTasks.")
+ if (maybeFinishAllRequesters(requesters, numTasks)) {
+ // Finished current barrier() call successfully, clean up ContextBarrierState and
+ // increase the barrier epoch.
+ logInfo(s"Barrier sync epoch $barrierEpoch from $barrierId received all updates from " +
+ s"tasks, finished successfully.")
+ barrierEpoch += 1
+ requesters.clear()
+ cancelTimerTask()
+ }
+ }
+ }
+
+ // Finish all the blocking barrier sync requests from a stage attempt successfully if we
+ // have received all the sync requests.
+ private def maybeFinishAllRequesters(
+ requesters: ArrayBuffer[RpcCallContext],
+ numTasks: Int): Boolean = {
+ if (requesters.size == numTasks) {
+ requesters.foreach(_.reply(()))
+ true
+ } else {
+ false
+ }
+ }
+
+ // Cleanup the internal state of a barrier stage attempt.
+ def clear(): Unit = synchronized {
+ // The global sync fails so the stage is expected to retry another attempt, all sync
+ // messages come from current stage attempt shall fail.
+ barrierEpoch = -1
+ requesters.clear()
+ cancelTimerTask()
+ }
+ }
+
+ // Clean up the [[ContextBarrierState]] that correspond to a specific stage attempt.
+ private def cleanupBarrierStage(barrierId: ContextBarrierId): Unit = {
+ val barrierState = states.remove(barrierId)
+ if (barrierState != null) {
+ barrierState.clear()
+ }
+ }
+
+ override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = {
+ case request @ RequestToSync(numTasks, stageId, stageAttemptId, _, _) =>
+ // Get or init the ContextBarrierState correspond to the stage attempt.
+ val barrierId = ContextBarrierId(stageId, stageAttemptId)
+ states.computeIfAbsent(barrierId, new Function[ContextBarrierId, ContextBarrierState] {
+ override def apply(key: ContextBarrierId): ContextBarrierState =
+ new ContextBarrierState(key, numTasks)
+ })
+ val barrierState = states.get(barrierId)
+
+ barrierState.handleRequest(context, request)
+ }
+
+ private val clearStateConsumer = new Consumer[ContextBarrierState] {
+ override def accept(state: ContextBarrierState) = state.clear()
+ }
+}
+
+private[spark] sealed trait BarrierCoordinatorMessage extends Serializable
+
+/**
+ * A global sync request message from BarrierTaskContext, by `barrier()` call. Each request is
+ * identified by stageId + stageAttemptId + barrierEpoch.
+ *
+ * @param numTasks The number of global sync requests the BarrierCoordinator shall receive
+ * @param stageId ID of current stage
+ * @param stageAttemptId ID of current stage attempt
+ * @param taskAttemptId Unique ID of current task
+ * @param barrierEpoch ID of the `barrier()` call, a task may consist multiple `barrier()` calls.
+ */
+private[spark] case class RequestToSync(
+ numTasks: Int,
+ stageId: Int,
+ stageAttemptId: Int,
+ taskAttemptId: Long,
+ barrierEpoch: Int) extends BarrierCoordinatorMessage
diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
new file mode 100644
index 0000000000000..90a5c4130f799
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala
@@ -0,0 +1,237 @@
+/*
+ * 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
+
+import java.util.{Properties, Timer, TimerTask}
+
+import scala.concurrent.duration._
+import scala.language.postfixOps
+
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.internal.Logging
+import org.apache.spark.memory.TaskMemoryManager
+import org.apache.spark.metrics.source.Source
+import org.apache.spark.rpc.{RpcEndpointRef, RpcTimeout}
+import org.apache.spark.shuffle.FetchFailedException
+import org.apache.spark.util._
+
+/**
+ * :: Experimental ::
+ * A [[TaskContext]] with extra contextual info and tooling for tasks in a barrier stage.
+ * Use [[BarrierTaskContext#get]] to obtain the barrier context for a running barrier task.
+ */
+@Experimental
+@Since("2.4.0")
+class BarrierTaskContext private[spark] (
+ taskContext: TaskContext) extends TaskContext with Logging {
+
+ // Find the driver side RPCEndpointRef of the coordinator that handles all the barrier() calls.
+ private val barrierCoordinator: RpcEndpointRef = {
+ val env = SparkEnv.get
+ RpcUtils.makeDriverRef("barrierSync", env.conf, env.rpcEnv)
+ }
+
+ private val timer = new Timer("Barrier task timer for barrier() calls.")
+
+ // Local barrierEpoch that identify a barrier() call from current task, it shall be identical
+ // with the driver side epoch.
+ private var barrierEpoch = 0
+
+ // Number of tasks of the current barrier stage, a barrier() call must collect enough requests
+ // from different tasks within the same barrier stage attempt to succeed.
+ private lazy val numTasks = getTaskInfos().size
+
+ /**
+ * :: Experimental ::
+ * Sets a global barrier and waits until all tasks in this stage hit this barrier. Similar to
+ * MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same
+ * stage have reached this routine.
+ *
+ * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all
+ * possible code branches. Otherwise, you may get the job hanging or a SparkException after
+ * timeout. Some examples of '''misuses''' are listed below:
+ * 1. Only call barrier() function on a subset of all the tasks in the same barrier stage, it
+ * shall lead to timeout of the function call.
+ * {{{
+ * rdd.barrier().mapPartitions { iter =>
+ * val context = BarrierTaskContext.get()
+ * if (context.partitionId() == 0) {
+ * // Do nothing.
+ * } else {
+ * context.barrier()
+ * }
+ * iter
+ * }
+ * }}}
+ *
+ * 2. Include barrier() function in a try-catch code block, this may lead to timeout of the
+ * second function call.
+ * {{{
+ * rdd.barrier().mapPartitions { iter =>
+ * val context = BarrierTaskContext.get()
+ * try {
+ * // Do something that might throw an Exception.
+ * doSomething()
+ * context.barrier()
+ * } catch {
+ * case e: Exception => logWarning("...", e)
+ * }
+ * context.barrier()
+ * iter
+ * }
+ * }}}
+ */
+ @Experimental
+ @Since("2.4.0")
+ def barrier(): Unit = {
+ logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) has entered " +
+ s"the global sync, current barrier epoch is $barrierEpoch.")
+ logTrace("Current callSite: " + Utils.getCallSite())
+
+ val startTime = System.currentTimeMillis()
+ val timerTask = new TimerTask {
+ override def run(): Unit = {
+ logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) waiting " +
+ s"under the global sync since $startTime, has been waiting for " +
+ s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch " +
+ s"is $barrierEpoch.")
+ }
+ }
+ // Log the update of global sync every 60 seconds.
+ timer.schedule(timerTask, 60000, 60000)
+
+ try {
+ barrierCoordinator.askSync[Unit](
+ message = RequestToSync(numTasks, stageId, stageAttemptNumber, taskAttemptId,
+ barrierEpoch),
+ // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by
+ // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework.
+ timeout = new RpcTimeout(31536000 /* = 3600 * 24 * 365 */ seconds, "barrierTimeout"))
+ barrierEpoch += 1
+ logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) finished " +
+ "global sync successfully, waited for " +
+ s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch is " +
+ s"$barrierEpoch.")
+ } catch {
+ case e: SparkException =>
+ logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) failed " +
+ "to perform global sync, waited for " +
+ s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch " +
+ s"is $barrierEpoch.")
+ throw e
+ } finally {
+ timerTask.cancel()
+ timer.purge()
+ }
+ }
+
+ /**
+ * :: Experimental ::
+ * Returns [[BarrierTaskInfo]] for all tasks in this barrier stage, ordered by partition ID.
+ */
+ @Experimental
+ @Since("2.4.0")
+ def getTaskInfos(): Array[BarrierTaskInfo] = {
+ val addressesStr = Option(taskContext.getLocalProperty("addresses")).getOrElse("")
+ addressesStr.split(",").map(_.trim()).map(new BarrierTaskInfo(_))
+ }
+
+ // delegate methods
+
+ override def isCompleted(): Boolean = taskContext.isCompleted()
+
+ override def isInterrupted(): Boolean = taskContext.isInterrupted()
+
+ override def isRunningLocally(): Boolean = taskContext.isRunningLocally()
+
+ override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = {
+ taskContext.addTaskCompletionListener(listener)
+ this
+ }
+
+ override def addTaskFailureListener(listener: TaskFailureListener): this.type = {
+ taskContext.addTaskFailureListener(listener)
+ this
+ }
+
+ override def stageId(): Int = taskContext.stageId()
+
+ override def stageAttemptNumber(): Int = taskContext.stageAttemptNumber()
+
+ override def partitionId(): Int = taskContext.partitionId()
+
+ override def attemptNumber(): Int = taskContext.attemptNumber()
+
+ override def taskAttemptId(): Long = taskContext.taskAttemptId()
+
+ override def getLocalProperty(key: String): String = taskContext.getLocalProperty(key)
+
+ override def taskMetrics(): TaskMetrics = taskContext.taskMetrics()
+
+ override def getMetricsSources(sourceName: String): Seq[Source] = {
+ taskContext.getMetricsSources(sourceName)
+ }
+
+ override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted()
+
+ override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason()
+
+ override private[spark] def taskMemoryManager(): TaskMemoryManager = {
+ taskContext.taskMemoryManager()
+ }
+
+ override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {
+ taskContext.registerAccumulator(a)
+ }
+
+ override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {
+ taskContext.setFetchFailed(fetchFailed)
+ }
+
+ override private[spark] def markInterrupted(reason: String): Unit = {
+ taskContext.markInterrupted(reason)
+ }
+
+ override private[spark] def markTaskFailed(error: Throwable): Unit = {
+ taskContext.markTaskFailed(error)
+ }
+
+ override private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = {
+ taskContext.markTaskCompleted(error)
+ }
+
+ override private[spark] def fetchFailed: Option[FetchFailedException] = {
+ taskContext.fetchFailed
+ }
+
+ override private[spark] def getLocalProperties: Properties = taskContext.getLocalProperties
+}
+
+@Experimental
+@Since("2.4.0")
+object BarrierTaskContext {
+ /**
+ * :: Experimental ::
+ * Returns the currently active BarrierTaskContext. This can be called inside of user functions to
+ * access contextual information about running barrier tasks.
+ */
+ @Experimental
+ @Since("2.4.0")
+ def get(): BarrierTaskContext = TaskContext.get().asInstanceOf[BarrierTaskContext]
+}
diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskInfo.scala b/core/src/main/scala/org/apache/spark/BarrierTaskInfo.scala
new file mode 100644
index 0000000000000..347239b1d7db4
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/BarrierTaskInfo.scala
@@ -0,0 +1,31 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.{Experimental, Since}
+
+
+/**
+ * :: Experimental ::
+ * Carries all task infos of a barrier task.
+ *
+ * @param address the IPv4 address(host:port) of the executor that a barrier task is running on
+ */
+@Experimental
+@Since("2.4.0")
+class BarrierTaskInfo private[spark] (val address: String)
diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
index aa363eeffffb8..c3e5b96a55884 100644
--- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
+++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala
@@ -25,7 +25,7 @@ import scala.util.control.{ControlThrowable, NonFatal}
import com.codahale.metrics.{Gauge, MetricRegistry}
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
import org.apache.spark.metrics.source.Source
import org.apache.spark.scheduler._
@@ -212,7 +212,7 @@ private[spark] class ExecutorAllocationManager(
}
// Require external shuffle service for dynamic allocation
// Otherwise, we may lose shuffle files when killing executors
- if (!conf.getBoolean("spark.shuffle.service.enabled", false) && !testing) {
+ if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !testing) {
throw new SparkException("Dynamic allocation of executors requires the external " +
"shuffle service. You may enable this through spark.shuffle.service.enabled.")
}
@@ -488,9 +488,15 @@ private[spark] class ExecutorAllocationManager(
newExecutorTotal = numExistingExecutors
if (testing || executorsRemoved.nonEmpty) {
executorsRemoved.foreach { removedExecutorId =>
+ // If it is a cached block, it uses cachedExecutorIdleTimeoutS for timeout
+ val idleTimeout = if (blockManagerMaster.hasCachedBlocks(removedExecutorId)) {
+ cachedExecutorIdleTimeoutS
+ } else {
+ executorIdleTimeoutS
+ }
newExecutorTotal -= 1
logInfo(s"Removing executor $removedExecutorId because it has been idle for " +
- s"$executorIdleTimeoutS seconds (new desired total will be $newExecutorTotal)")
+ s"$idleTimeout seconds (new desired total will be $newExecutorTotal)")
executorsPendingToRemove.add(removedExecutorId)
}
executorsRemoved
diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
index bcbc8df0d5865..ab0ae55ed357d 100644
--- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
+++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
@@ -22,6 +22,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit}
import scala.collection.mutable
import scala.concurrent.Future
+import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint}
import org.apache.spark.scheduler._
@@ -37,7 +38,8 @@ import org.apache.spark.util._
private[spark] case class Heartbeat(
executorId: String,
accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates
- blockManagerId: BlockManagerId)
+ blockManagerId: BlockManagerId,
+ executorUpdates: ExecutorMetrics) // executor level updates
/**
* An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is
@@ -119,14 +121,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock)
context.reply(true)
// Messages received from executors
- case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId) =>
+ case heartbeat @ Heartbeat(executorId, accumUpdates, blockManagerId, executorMetrics) =>
if (scheduler != null) {
if (executorLastSeen.contains(executorId)) {
executorLastSeen(executorId) = clock.getTimeMillis()
eventLoopThread.submit(new Runnable {
override def run(): Unit = Utils.tryLogNonFatalError {
val unknownExecutor = !scheduler.executorHeartbeatReceived(
- executorId, accumUpdates, blockManagerId)
+ executorId, accumUpdates, blockManagerId, executorMetrics)
val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor)
context.reply(response)
}
diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala
new file mode 100644
index 0000000000000..5ba1b9b2d828e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala
@@ -0,0 +1,71 @@
+/*
+ * 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
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.executor.ExecutorMetrics
+import org.apache.spark.internal.Logging
+import org.apache.spark.memory.MemoryManager
+import org.apache.spark.metrics.ExecutorMetricType
+import org.apache.spark.util.{ThreadUtils, Utils}
+
+/**
+ * Creates a heartbeat thread which will call the specified reportHeartbeat function at
+ * intervals of intervalMs.
+ *
+ * @param memoryManager the memory manager for execution and storage memory.
+ * @param reportHeartbeat the heartbeat reporting function to call.
+ * @param name the thread name for the heartbeater.
+ * @param intervalMs the interval between heartbeats.
+ */
+private[spark] class Heartbeater(
+ memoryManager: MemoryManager,
+ reportHeartbeat: () => Unit,
+ name: String,
+ intervalMs: Long) extends Logging {
+ // Executor for the heartbeat task
+ private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name)
+
+ /** Schedules a task to report a heartbeat. */
+ def start(): Unit = {
+ // Wait a random interval so the heartbeats don't end up in sync
+ val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int]
+
+ val heartbeatTask = new Runnable() {
+ override def run(): Unit = Utils.logUncaughtExceptions(reportHeartbeat())
+ }
+ heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS)
+ }
+
+ /** Stops the heartbeat thread. */
+ def stop(): Unit = {
+ heartbeater.shutdown()
+ heartbeater.awaitTermination(10, TimeUnit.SECONDS)
+ }
+
+ /**
+ * Get the current executor level metrics. These are returned as an array, with the index
+ * determined by MetricGetter.values
+ */
+ def getCurrentMetrics(): ExecutorMetrics = {
+ val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray
+ new ExecutorMetrics(metrics)
+ }
+}
+
diff --git a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala
index f8a6f1d0d8cbb..ff85e11409e35 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala
@@ -23,5 +23,9 @@ package org.apache.spark
* @param shuffleId ID of the shuffle
* @param bytesByPartitionId approximate number of output bytes for each map output partition
* (may be inexact due to use of compressed map statuses)
+ * @param recordsByPartitionId number of output records for each map output partition
*/
-private[spark] class MapOutputStatistics(val shuffleId: Int, val bytesByPartitionId: Array[Long])
+private[spark] class MapOutputStatistics(
+ val shuffleId: Int,
+ val bytesByPartitionId: Array[Long],
+ val recordsByPartitionId: Array[Long])
diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 73646051f264c..41575ce4e6e3d 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -434,6 +434,18 @@ private[spark] class MapOutputTrackerMaster(
}
}
+ /** Unregister all map output information of the given shuffle. */
+ def unregisterAllMapOutput(shuffleId: Int) {
+ shuffleStatuses.get(shuffleId) match {
+ case Some(shuffleStatus) =>
+ shuffleStatus.removeOutputsByFilter(x => true)
+ incrementEpoch()
+ case None =>
+ throw new SparkException(
+ s"unregisterAllMapOutput called for nonexistent shuffle ID $shuffleId.")
+ }
+ }
+
/** Unregister shuffle data */
def unregisterShuffle(shuffleId: Int) {
shuffleStatuses.remove(shuffleId).foreach { shuffleStatus =>
@@ -510,16 +522,19 @@ private[spark] class MapOutputTrackerMaster(
def getStatistics(dep: ShuffleDependency[_, _, _]): MapOutputStatistics = {
shuffleStatuses(dep.shuffleId).withMapStatuses { statuses =>
val totalSizes = new Array[Long](dep.partitioner.numPartitions)
+ val recordsByMapTask = new Array[Long](statuses.length)
+
val parallelAggThreshold = conf.get(
SHUFFLE_MAP_OUTPUT_PARALLEL_AGGREGATION_THRESHOLD)
val parallelism = math.min(
Runtime.getRuntime.availableProcessors(),
statuses.length.toLong * totalSizes.length / parallelAggThreshold + 1).toInt
if (parallelism <= 1) {
- for (s <- statuses) {
+ statuses.zipWithIndex.foreach { case (s, index) =>
for (i <- 0 until totalSizes.length) {
totalSizes(i) += s.getSizeForBlock(i)
}
+ recordsByMapTask(index) = s.numberOfOutput
}
} else {
val threadPool = ThreadUtils.newDaemonFixedThreadPool(parallelism, "map-output-aggregate")
@@ -536,8 +551,11 @@ private[spark] class MapOutputTrackerMaster(
} finally {
threadPool.shutdown()
}
+ statuses.zipWithIndex.foreach { case (s, index) =>
+ recordsByMapTask(index) = s.numberOfOutput
+ }
}
- new MapOutputStatistics(dep.shuffleId, totalSizes)
+ new MapOutputStatistics(dep.shuffleId, totalSizes, recordsByMapTask)
}
}
diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala
index c940cb25d478b..515237558fd87 100644
--- a/core/src/main/scala/org/apache/spark/Partitioner.scala
+++ b/core/src/main/scala/org/apache/spark/Partitioner.scala
@@ -33,6 +33,9 @@ import org.apache.spark.util.random.SamplingUtils
/**
* An object that defines how the elements in a key-value pair RDD are partitioned by key.
* Maps each key to a partition ID, from 0 to `numPartitions - 1`.
+ *
+ * Note that, partitioner must be deterministic, i.e. it must return the same partition id given
+ * the same partition key.
*/
abstract class Partitioner extends Serializable {
def numPartitions: Int
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 74bfb5d6d2ea3..d943087ab6b80 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -213,6 +213,7 @@ class SparkContext(config: SparkConf) extends Logging {
private var _files: Seq[String] = _
private var _shutdownHookRef: AnyRef = _
private var _statusStore: AppStatusStore = _
+ private var _heartbeater: Heartbeater = _
/* ------------------------------------------------------------------------------------- *
| Accessors and public fields. These provide access to the internal state of the |
@@ -254,7 +255,7 @@ class SparkContext(config: SparkConf) extends Logging {
conf: SparkConf,
isLocal: Boolean,
listenerBus: LiveListenerBus): SparkEnv = {
- SparkEnv.createDriverEnv(conf, isLocal, listenerBus, SparkContext.numDriverCores(master))
+ SparkEnv.createDriverEnv(conf, isLocal, listenerBus, SparkContext.numDriverCores(master, conf))
}
private[spark] def env: SparkEnv = _env
@@ -496,6 +497,11 @@ class SparkContext(config: SparkConf) extends Logging {
_dagScheduler = new DAGScheduler(this)
_heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet)
+ // create and start the heartbeater for collecting memory metrics
+ _heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, "driver-heartbeater",
+ conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
+ _heartbeater.start()
+
// start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's
// constructor
_taskScheduler.start()
@@ -571,7 +577,12 @@ class SparkContext(config: SparkConf) extends Logging {
_shutdownHookRef = ShutdownHookManager.addShutdownHook(
ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY) { () =>
logInfo("Invoking stop() from shutdown hook")
- stop()
+ try {
+ stop()
+ } catch {
+ case e: Throwable =>
+ logWarning("Ignoring Exception while stopping SparkContext from shutdown hook", e)
+ }
}
} catch {
case NonFatal(e) =>
@@ -1496,6 +1507,8 @@ class SparkContext(config: SparkConf) extends Logging {
* @param path can be either a local file, a file in HDFS (or other Hadoop-supported
* filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs,
* use `SparkFiles.get(fileName)` to find its download location.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addFile(path: String): Unit = {
addFile(path, false)
@@ -1516,11 +1529,17 @@ class SparkContext(config: SparkConf) extends Logging {
* use `SparkFiles.get(fileName)` to find its download location.
* @param recursive if true, a directory can be given in `path`. Currently directories are
* only supported for Hadoop-supported filesystems.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addFile(path: String, recursive: Boolean): Unit = {
val uri = new Path(path).toUri
val schemeCorrectedPath = uri.getScheme match {
- case null | "local" => new File(path).getCanonicalFile.toURI.toString
+ case null => new File(path).getCanonicalFile.toURI.toString
+ case "local" =>
+ logWarning("File with 'local' scheme is not supported to add to file server, since " +
+ "it is already available on every node.")
+ return
case _ => path
}
@@ -1555,6 +1574,9 @@ class SparkContext(config: SparkConf) extends Logging {
Utils.fetchFile(uri.toString, new File(SparkFiles.getRootDirectory()), conf,
env.securityManager, hadoopConfiguration, timestamp, useCache = false)
postEnvironmentUpdate()
+ } else {
+ logWarning(s"The path $path has been added already. Overwriting of added paths " +
+ "is not supported in the current version.")
}
}
@@ -1586,6 +1608,15 @@ class SparkContext(config: SparkConf) extends Logging {
}
}
+ /**
+ * Get the max number of tasks that can be concurrent launched currently.
+ * Note that please don't cache the value returned by this method, because the number can change
+ * due to add/remove executors.
+ *
+ * @return The max number of tasks that can be concurrent launched currently.
+ */
+ private[spark] def maxNumConcurrentTasks(): Int = schedulerBackend.maxNumConcurrentTasks()
+
/**
* Update the cluster manager on our scheduling needs. Three bits of information are included
* to help it make decisions.
@@ -1803,6 +1834,8 @@ class SparkContext(config: SparkConf) extends Logging {
*
* @param path can be either a local file, a file in HDFS (or other Hadoop-supported filesystems),
* an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addJar(path: String) {
def addJarFile(file: File): String = {
@@ -1849,6 +1882,9 @@ class SparkContext(config: SparkConf) extends Logging {
if (addedJars.putIfAbsent(key, timestamp).isEmpty) {
logInfo(s"Added JAR $path at $key with timestamp $timestamp")
postEnvironmentUpdate()
+ } else {
+ logWarning(s"The jar $path has been added already. Overwriting of added jars " +
+ "is not supported in the current version.")
}
}
}
@@ -1914,6 +1950,12 @@ class SparkContext(config: SparkConf) extends Logging {
Utils.tryLogNonFatalError {
_executorAllocationManager.foreach(_.stop())
}
+ if (_dagScheduler != null) {
+ Utils.tryLogNonFatalError {
+ _dagScheduler.stop()
+ }
+ _dagScheduler = null
+ }
if (_listenerBusStarted) {
Utils.tryLogNonFatalError {
listenerBus.stop()
@@ -1923,11 +1965,11 @@ class SparkContext(config: SparkConf) extends Logging {
Utils.tryLogNonFatalError {
_eventLogger.foreach(_.stop())
}
- if (_dagScheduler != null) {
+ if (_heartbeater != null) {
Utils.tryLogNonFatalError {
- _dagScheduler.stop()
+ _heartbeater.stop()
}
- _dagScheduler = null
+ _heartbeater = null
}
if (env != null && _heartbeatReceiver != null) {
Utils.tryLogNonFatalError {
@@ -2399,6 +2441,14 @@ class SparkContext(config: SparkConf) extends Logging {
}
}
+ /** Reports heartbeat metrics for the driver. */
+ private def reportHeartBeat(): Unit = {
+ val driverUpdates = _heartbeater.getCurrentMetrics()
+ val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0)
+ listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates,
+ Some(driverUpdates)))
+ }
+
// In order to prevent multiple SparkContexts from being active at the same time, mark this
// context as having finished construction.
// NOTE: this must be placed at the end of the SparkContext constructor.
@@ -2652,9 +2702,16 @@ object SparkContext extends Logging {
}
/**
- * The number of driver cores to use for execution in local mode, 0 otherwise.
+ * The number of cores available to the driver to use for tasks such as I/O with Netty
*/
private[spark] def numDriverCores(master: String): Int = {
+ numDriverCores(master, null)
+ }
+
+ /**
+ * The number of cores available to the driver to use for tasks such as I/O with Netty
+ */
+ private[spark] def numDriverCores(master: String, conf: SparkConf): Int = {
def convertToInt(threads: String): Int = {
if (threads == "*") Runtime.getRuntime.availableProcessors() else threads.toInt
}
@@ -2662,7 +2719,13 @@ object SparkContext extends Logging {
case "local" => 1
case SparkMasterRegex.LOCAL_N_REGEX(threads) => convertToInt(threads)
case SparkMasterRegex.LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads)
- case _ => 0 // driver is not used for execution
+ case "yarn" =>
+ if (conf != null && conf.getOption("spark.submit.deployMode").contains("cluster")) {
+ conf.getInt("spark.driver.cores", 0)
+ } else {
+ 0
+ }
+ case _ => 0 // Either driver is not being used, or its core count will be interpolated later
}
}
diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index 69739745aa6cf..2b939dabb1105 100644
--- a/core/src/main/scala/org/apache/spark/TaskContext.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContext.scala
@@ -123,7 +123,10 @@ abstract class TaskContext extends Serializable {
*
* Exceptions thrown by the listener will result in failure of the task.
*/
- def addTaskCompletionListener(f: (TaskContext) => Unit): TaskContext = {
+ def addTaskCompletionListener[U](f: (TaskContext) => U): TaskContext = {
+ // Note that due to this scala bug: https://github.com/scala/bug/issues/11016, we need to make
+ // this function polymorphic for every scala version >= 2.12, otherwise an overloaded method
+ // resolution error occurs at compile time.
addTaskCompletionListener(new TaskCompletionListener {
override def onTaskCompletion(context: TaskContext): Unit = f(context)
})
@@ -218,4 +221,18 @@ abstract class TaskContext extends Serializable {
*/
private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit
+ /** Marks the task for interruption, i.e. cancellation. */
+ private[spark] def markInterrupted(reason: String): Unit
+
+ /** Marks the task as failed and triggers the failure listeners. */
+ private[spark] def markTaskFailed(error: Throwable): Unit
+
+ /** Marks the task as completed and triggers the completion listeners. */
+ private[spark] def markTaskCompleted(error: Option[Throwable]): Unit
+
+ /** Optionally returns the stored fetch failure in the task. */
+ private[spark] def fetchFailed: Option[FetchFailedException]
+
+ /** Gets local properties set upstream in the driver. */
+ private[spark] def getLocalProperties: Properties
}
diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
index 0791fe856ef15..89730424e5acf 100644
--- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
+++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala
@@ -30,6 +30,7 @@ import org.apache.spark.metrics.source.Source
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.util._
+
/**
* A [[TaskContext]] implementation.
*
@@ -98,9 +99,8 @@ private[spark] class TaskContextImpl(
this
}
- /** Marks the task as failed and triggers the failure listeners. */
@GuardedBy("this")
- private[spark] def markTaskFailed(error: Throwable): Unit = synchronized {
+ private[spark] override def markTaskFailed(error: Throwable): Unit = synchronized {
if (failed) return
failed = true
failure = error
@@ -109,9 +109,8 @@ private[spark] class TaskContextImpl(
}
}
- /** Marks the task as completed and triggers the completion listeners. */
@GuardedBy("this")
- private[spark] def markTaskCompleted(error: Option[Throwable]): Unit = synchronized {
+ private[spark] override def markTaskCompleted(error: Option[Throwable]): Unit = synchronized {
if (completed) return
completed = true
invokeListeners(onCompleteCallbacks, "TaskCompletionListener", error) {
@@ -140,8 +139,7 @@ private[spark] class TaskContextImpl(
}
}
- /** Marks the task for interruption, i.e. cancellation. */
- private[spark] def markInterrupted(reason: String): Unit = {
+ private[spark] override def markInterrupted(reason: String): Unit = {
reasonIfKilled = Some(reason)
}
@@ -176,8 +174,7 @@ private[spark] class TaskContextImpl(
this._fetchFailedException = Option(fetchFailed)
}
- private[spark] def fetchFailed: Option[FetchFailedException] = _fetchFailedException
+ private[spark] override def fetchFailed: Option[FetchFailedException] = _fetchFailedException
- // TODO: shall we publish it and define it in `TaskContext`?
- private[spark] def getLocalProperties(): Properties = localProperties
+ private[spark] override def getLocalProperties(): Properties = localProperties
}
diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
index b5c4c705dcbc7..c2ebd388a2365 100644
--- a/core/src/main/scala/org/apache/spark/TestUtils.scala
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets
import java.security.SecureRandom
import java.security.cert.X509Certificate
import java.util.{Arrays, Properties}
-import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit}
+import java.util.concurrent.{TimeoutException, TimeUnit}
import java.util.jar.{JarEntry, JarOutputStream}
import javax.net.ssl._
import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider}
@@ -172,22 +172,24 @@ private[spark] object TestUtils {
/**
* Run some code involving jobs submitted to the given context and assert that the jobs spilled.
*/
- def assertSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
- val spillListener = new SpillListener
- sc.addSparkListener(spillListener)
- body
- assert(spillListener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
+ def assertSpilled(sc: SparkContext, identifier: String)(body: => Unit): Unit = {
+ val listener = new SpillListener
+ withListener(sc, listener) { _ =>
+ body
+ }
+ assert(listener.numSpilledStages > 0, s"expected $identifier to spill, but did not")
}
/**
* Run some code involving jobs submitted to the given context and assert that the jobs
* did not spill.
*/
- def assertNotSpilled[T](sc: SparkContext, identifier: String)(body: => T): Unit = {
- val spillListener = new SpillListener
- sc.addSparkListener(spillListener)
- body
- assert(spillListener.numSpilledStages == 0, s"expected $identifier to not spill, but did")
+ def assertNotSpilled(sc: SparkContext, identifier: String)(body: => Unit): Unit = {
+ val listener = new SpillListener
+ withListener(sc, listener) { _ =>
+ body
+ }
+ assert(listener.numSpilledStages == 0, s"expected $identifier to not spill, but did")
}
/**
@@ -233,6 +235,21 @@ private[spark] object TestUtils {
}
}
+ /**
+ * Runs some code with the given listener installed in the SparkContext. After the code runs,
+ * this method will wait until all events posted to the listener bus are processed, and then
+ * remove the listener from the bus.
+ */
+ def withListener[L <: SparkListener](sc: SparkContext, listener: L) (body: L => Unit): Unit = {
+ sc.addSparkListener(listener)
+ try {
+ body(listener)
+ } finally {
+ sc.listenerBus.waitUntilEmpty(TimeUnit.SECONDS.toMillis(10))
+ sc.listenerBus.removeListener(listener)
+ }
+ }
+
/**
* Wait until at least `numExecutors` executors are up, or throw `TimeoutException` if the waiting
* time elapsed before `numExecutors` executors up. Exposed for testing.
@@ -289,21 +306,17 @@ private[spark] object TestUtils {
private class SpillListener extends SparkListener {
private val stageIdToTaskMetrics = new mutable.HashMap[Int, ArrayBuffer[TaskMetrics]]
private val spilledStageIds = new mutable.HashSet[Int]
- private val stagesDone = new CountDownLatch(1)
- def numSpilledStages: Int = {
- // Long timeout, just in case somehow the job end isn't notified.
- // Fails if a timeout occurs
- assert(stagesDone.await(10, TimeUnit.SECONDS))
+ def numSpilledStages: Int = synchronized {
spilledStageIds.size
}
- override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = {
+ override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized {
stageIdToTaskMetrics.getOrElseUpdate(
taskEnd.stageId, new ArrayBuffer[TaskMetrics]) += taskEnd.taskMetrics
}
- override def onStageCompleted(stageComplete: SparkListenerStageCompleted): Unit = {
+ override def onStageCompleted(stageComplete: SparkListenerStageCompleted): Unit = synchronized {
val stageId = stageComplete.stageInfo.stageId
val metrics = stageIdToTaskMetrics.remove(stageId).toSeq.flatten
val spilled = metrics.map(_.memoryBytesSpilled).sum > 0
@@ -311,8 +324,4 @@ private class SpillListener extends SparkListener {
spilledStageIds += stageId
}
}
-
- override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = {
- stagesDone.countDown()
- }
}
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index f1936bf587282..09c83849e26b2 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -668,6 +668,8 @@ class JavaSparkContext(val sc: SparkContext)
* The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
* filesystems), or an HTTP, HTTPS or FTP URI. To access the file in Spark jobs,
* use `SparkFiles.get(fileName)` to find its download location.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addFile(path: String) {
sc.addFile(path)
@@ -681,6 +683,8 @@ class JavaSparkContext(val sc: SparkContext)
*
* A directory can be given if the recursive option is set to true. Currently directories are only
* supported for Hadoop-supported filesystems.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addFile(path: String, recursive: Boolean): Unit = {
sc.addFile(path, recursive)
@@ -690,6 +694,8 @@ class JavaSparkContext(val sc: SparkContext)
* Adds a JAR dependency for all tasks to be executed on this SparkContext in the future.
* The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported
* filesystems), or an HTTP, HTTPS or FTP URI.
+ *
+ * @note A path can be added only once. Subsequent additions of the same path are ignored.
*/
def addJar(path: String) {
sc.addJar(path)
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index a1ee2f7d1b119..e639a842754bd 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
@@ -45,12 +45,10 @@ import org.apache.spark.util._
private[spark] class PythonRDD(
parent: RDD[_],
func: PythonFunction,
- preservePartitoning: Boolean)
+ preservePartitoning: Boolean,
+ isFromBarrier: Boolean = false)
extends RDD[Array[Byte]](parent) {
- val bufferSize = conf.getInt("spark.buffer.size", 65536)
- val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true)
-
override def getPartitions: Array[Partition] = firstParent.partitions
override val partitioner: Option[Partitioner] = {
@@ -60,9 +58,12 @@ private[spark] class PythonRDD(
val asJavaRDD: JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this)
override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
- val runner = PythonRunner(func, bufferSize, reuseWorker)
+ val runner = PythonRunner(func)
runner.compute(firstParent.iterator(split, context), split.index, context)
}
+
+ @transient protected lazy override val isBarrier_ : Boolean =
+ isFromBarrier || dependencies.exists(_.rdd.isBarrier())
}
/**
@@ -398,6 +399,26 @@ private[spark] object PythonRDD extends Logging {
* data collected from this job, and the secret for authentication.
*/
def serveIterator(items: Iterator[_], threadName: String): Array[Any] = {
+ serveToStream(threadName) { out =>
+ writeIteratorToStream(items, new DataOutputStream(out))
+ }
+ }
+
+ /**
+ * Create a socket server and background thread to execute the writeFunc
+ * with the given OutputStream.
+ *
+ * The socket server can only accept one connection, or close if no connection
+ * in 15 seconds.
+ *
+ * Once a connection comes in, it will execute the block of code and pass in
+ * the socket output stream.
+ *
+ * The thread will terminate after the block of code is executed or any
+ * exceptions happen.
+ */
+ private[spark] def serveToStream(
+ threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = {
val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost"))
// Close the socket if no connection in 15 seconds
serverSocket.setSoTimeout(15000)
@@ -409,9 +430,9 @@ private[spark] object PythonRDD extends Logging {
val sock = serverSocket.accept()
authHelper.authClient(sock)
- val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
+ val out = new BufferedOutputStream(sock.getOutputStream)
Utils.tryWithSafeFinally {
- writeIteratorToStream(items, out)
+ writeFunc(out)
} {
out.close()
sock.close()
@@ -586,8 +607,9 @@ class BytesToString extends org.apache.spark.api.java.function.Function[Array[By
*/
private[spark] class PythonAccumulatorV2(
@transient private val serverHost: String,
- private val serverPort: Int)
- extends CollectionAccumulator[Array[Byte]] {
+ private val serverPort: Int,
+ private val secretToken: String)
+ extends CollectionAccumulator[Array[Byte]] with Logging{
Utils.checkHost(serverHost)
@@ -602,17 +624,22 @@ private[spark] class PythonAccumulatorV2(
private def openSocket(): Socket = synchronized {
if (socket == null || socket.isClosed) {
socket = new Socket(serverHost, serverPort)
+ logInfo(s"Connected to AccumulatorServer at host: $serverHost port: $serverPort")
+ // send the secret just for the initial authentication when opening a new connection
+ socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8))
}
socket
}
// Need to override so the types match with PythonFunction
- override def copyAndReset(): PythonAccumulatorV2 = new PythonAccumulatorV2(serverHost, serverPort)
+ override def copyAndReset(): PythonAccumulatorV2 = {
+ new PythonAccumulatorV2(serverHost, serverPort, secretToken)
+ }
override def merge(other: AccumulatorV2[Array[Byte], JList[Array[Byte]]]): Unit = synchronized {
val otherPythonAccumulator = other.asInstanceOf[PythonAccumulatorV2]
// This conditional isn't strictly speaking needed - merging only currently happens on the
- // driver program - but that isn't gauranteed so incase this changes.
+ // driver program - but that isn't guaranteed so incase this changes.
if (serverHost == null) {
// We are on the worker
super.merge(otherPythonAccumulator)
diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
index ebabedf950e39..4c53bc269a104 100644
--- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
+++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
@@ -20,12 +20,15 @@ package org.apache.spark.api.python
import java.io._
import java.net._
import java.nio.charset.StandardCharsets
+import java.nio.charset.StandardCharsets.UTF_8
import java.util.concurrent.atomic.AtomicBoolean
import scala.collection.JavaConverters._
import org.apache.spark._
import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY
+import org.apache.spark.security.SocketAuthHelper
import org.apache.spark.util._
@@ -60,14 +63,20 @@ private[spark] object PythonEvalType {
*/
private[spark] abstract class BasePythonRunner[IN, OUT](
funcs: Seq[ChainedPythonFunctions],
- bufferSize: Int,
- reuseWorker: Boolean,
evalType: Int,
argOffsets: Array[Array[Int]])
extends Logging {
require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs")
+ private val conf = SparkEnv.get.conf
+ private val bufferSize = conf.getInt("spark.buffer.size", 65536)
+ private val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true)
+ // each python worker gets an equal part of the allocation. the worker pool will grow to the
+ // number of concurrent tasks, which is determined by the number of cores in this executor.
+ private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY)
+ .map(_ / conf.getInt("spark.executor.cores", 1))
+
// All the Python functions should have the same exec, version and envvars.
protected val envVars = funcs.head.funcs.head.envVars
protected val pythonExec = funcs.head.funcs.head.pythonExec
@@ -76,6 +85,12 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
// TODO: support accumulator in multiple UDF
protected val accumulator = funcs.head.funcs.head.accumulator
+ // Expose a ServerSocket to support method calls via socket from Python side.
+ private[spark] var serverSocket: Option[ServerSocket] = None
+
+ // Authentication helper used when serving method calls via socket from Python side.
+ private lazy val authHelper = new SocketAuthHelper(conf)
+
def compute(
inputIterator: Iterator[IN],
partitionIndex: Int,
@@ -87,6 +102,9 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
if (reuseWorker) {
envVars.put("SPARK_REUSE_WORKER", "1")
}
+ if (memoryMb.isDefined) {
+ envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString)
+ }
val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap)
// Whether is the worker released into idle pool
val released = new AtomicBoolean(false)
@@ -94,7 +112,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
// Start a thread to feed the process input from our parent's iterator
val writerThread = newWriterThread(env, worker, inputIterator, partitionIndex, context)
- context.addTaskCompletionListener { _ =>
+ context.addTaskCompletionListener[Unit] { _ =>
writerThread.shutdownOnTaskCompletion()
if (!reuseWorker || !released.get) {
try {
@@ -180,12 +198,79 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
dataOut.writeInt(partitionIndex)
// Python version of driver
PythonRDD.writeUTF(pythonVer, dataOut)
+ // Init a ServerSocket to accept method calls from Python side.
+ val isBarrier = context.isInstanceOf[BarrierTaskContext]
+ if (isBarrier) {
+ serverSocket = Some(new ServerSocket(/* port */ 0,
+ /* backlog */ 1,
+ InetAddress.getByName("localhost")))
+ // A call to accept() for ServerSocket shall block infinitely.
+ serverSocket.map(_.setSoTimeout(0))
+ new Thread("accept-connections") {
+ setDaemon(true)
+
+ override def run(): Unit = {
+ while (!serverSocket.get.isClosed()) {
+ var sock: Socket = null
+ try {
+ sock = serverSocket.get.accept()
+ // Wait for function call from python side.
+ sock.setSoTimeout(10000)
+ authHelper.authClient(sock)
+ val input = new DataInputStream(sock.getInputStream())
+ input.readInt() match {
+ case BarrierTaskContextMessageProtocol.BARRIER_FUNCTION =>
+ // The barrier() function may wait infinitely, socket shall not timeout
+ // before the function finishes.
+ sock.setSoTimeout(0)
+ barrierAndServe(sock)
+
+ case _ =>
+ val out = new DataOutputStream(new BufferedOutputStream(
+ sock.getOutputStream))
+ writeUTF(BarrierTaskContextMessageProtocol.ERROR_UNRECOGNIZED_FUNCTION, out)
+ }
+ } catch {
+ case e: SocketException if e.getMessage.contains("Socket closed") =>
+ // It is possible that the ServerSocket is not closed, but the native socket
+ // has already been closed, we shall catch and silently ignore this case.
+ } finally {
+ if (sock != null) {
+ sock.close()
+ }
+ }
+ }
+ }
+ }.start()
+ }
+ val secret = if (isBarrier) {
+ authHelper.secret
+ } else {
+ ""
+ }
+ // Close ServerSocket on task completion.
+ serverSocket.foreach { server =>
+ context.addTaskCompletionListener[Unit](_ => server.close())
+ }
+ val boundPort: Int = serverSocket.map(_.getLocalPort).getOrElse(0)
+ if (boundPort == -1) {
+ val message = "ServerSocket failed to bind to Java side."
+ logError(message)
+ throw new SparkException(message)
+ } else if (isBarrier) {
+ logDebug(s"Started ServerSocket on port $boundPort.")
+ }
// Write out the TaskContextInfo
+ dataOut.writeBoolean(isBarrier)
+ dataOut.writeInt(boundPort)
+ val secretBytes = secret.getBytes(UTF_8)
+ dataOut.writeInt(secretBytes.length)
+ dataOut.write(secretBytes, 0, secretBytes.length)
dataOut.writeInt(context.stageId())
dataOut.writeInt(context.partitionId())
dataOut.writeInt(context.attemptNumber())
dataOut.writeLong(context.taskAttemptId())
- val localProps = context.asInstanceOf[TaskContextImpl].getLocalProperties.asScala
+ val localProps = context.getLocalProperties.asScala
dataOut.writeInt(localProps.size)
localProps.foreach { case (k, v) =>
PythonRDD.writeUTF(k, dataOut)
@@ -243,6 +328,30 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
}
}
}
+
+ /**
+ * Gateway to call BarrierTaskContext.barrier().
+ */
+ def barrierAndServe(sock: Socket): Unit = {
+ require(serverSocket.isDefined, "No available ServerSocket to redirect the barrier() call.")
+
+ val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream))
+ try {
+ context.asInstanceOf[BarrierTaskContext].barrier()
+ writeUTF(BarrierTaskContextMessageProtocol.BARRIER_RESULT_SUCCESS, out)
+ } catch {
+ case e: SparkException =>
+ writeUTF(e.getMessage, out)
+ } finally {
+ out.close()
+ }
+ }
+
+ def writeUTF(str: String, dataOut: DataOutputStream) {
+ val bytes = str.getBytes(UTF_8)
+ dataOut.writeInt(bytes.length)
+ dataOut.write(bytes)
+ }
}
abstract class ReaderIterator(
@@ -385,20 +494,17 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
private[spark] object PythonRunner {
- def apply(func: PythonFunction, bufferSize: Int, reuseWorker: Boolean): PythonRunner = {
- new PythonRunner(Seq(ChainedPythonFunctions(Seq(func))), bufferSize, reuseWorker)
+ def apply(func: PythonFunction): PythonRunner = {
+ new PythonRunner(Seq(ChainedPythonFunctions(Seq(func))))
}
}
/**
* A helper class to run Python mapPartition in Spark.
*/
-private[spark] class PythonRunner(
- funcs: Seq[ChainedPythonFunctions],
- bufferSize: Int,
- reuseWorker: Boolean)
+private[spark] class PythonRunner(funcs: Seq[ChainedPythonFunctions])
extends BasePythonRunner[Array[Byte], Array[Byte]](
- funcs, bufferSize, reuseWorker, PythonEvalType.NON_UDF, Array(Array(0))) {
+ funcs, PythonEvalType.NON_UDF, Array(Array(0))) {
protected override def newWriterThread(
env: SparkEnv,
@@ -465,3 +571,9 @@ private[spark] object SpecialLengths {
val NULL = -5
val START_ARROW_STREAM = -6
}
+
+private[spark] object BarrierTaskContextMessageProtocol {
+ val BARRIER_FUNCTION = 1
+ val BARRIER_RESULT_SUCCESS = "success"
+ val ERROR_UNRECOGNIZED_FUNCTION = "Not recognized function call from python side."
+}
diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
index 3b2e809408e0f..7ce2581555014 100644
--- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
+++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala
@@ -96,11 +96,11 @@ private[spark] class RBackend {
channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS)
channelFuture = null
}
- if (bootstrap != null && bootstrap.group() != null) {
- bootstrap.group().shutdownGracefully()
+ if (bootstrap != null && bootstrap.config().group() != null) {
+ bootstrap.config().group().shutdownGracefully()
}
if (bootstrap != null && bootstrap.childGroup() != null) {
- bootstrap.childGroup().shutdownGracefully()
+ bootstrap.config().childGroup().shutdownGracefully()
}
bootstrap = null
jvmObjectTracker.clear()
diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
index e125095cf4777..cbd49e070f2eb 100644
--- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
+++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala
@@ -262,7 +262,7 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long)
val blockManager = SparkEnv.get.blockManager
Option(TaskContext.get()) match {
case Some(taskContext) =>
- taskContext.addTaskCompletionListener(_ => blockManager.releaseLock(blockId))
+ taskContext.addTaskCompletionListener[Unit](_ => blockManager.releaseLock(blockId))
case None =>
// This should only happen on the driver, where broadcast variables may be accessed
// outside of running tasks (e.g. when computing rdd.partitions()). In order to allow
diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala
index b59a4fe66587c..f6b3c37f0fe72 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala
@@ -22,7 +22,7 @@ import java.util.concurrent.CountDownLatch
import scala.collection.JavaConverters._
import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.network.TransportContext
import org.apache.spark.network.crypto.AuthServerBootstrap
@@ -45,8 +45,8 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana
protected val masterMetricsSystem =
MetricsSystem.createMetricsSystem("shuffleService", sparkConf, securityManager)
- private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false)
- private val port = sparkConf.getInt("spark.shuffle.service.port", 7337)
+ private val enabled = sparkConf.get(config.SHUFFLE_SERVICE_ENABLED)
+ private val port = sparkConf.get(config.SHUFFLE_SERVICE_PORT)
private val transportConf =
SparkTransportConf.fromSparkConf(sparkConf, "shuffle", numUsableCores = 0)
@@ -131,7 +131,7 @@ object ExternalShuffleService extends Logging {
// we override this value since this service is started from the command line
// and we assume the user really wants it to be running
- sparkConf.set("spark.shuffle.service.enabled", "true")
+ sparkConf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true")
server = newShuffleService(sparkConf, securityManager)
server.start()
diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
index 84aa8944fc1c7..be293f88a9d4a 100644
--- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala
@@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.spark.SparkConf
import org.apache.spark.deploy.master.Master
import org.apache.spark.deploy.worker.Worker
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.rpc.RpcEnv
import org.apache.spark.util.Utils
@@ -52,7 +52,7 @@ class LocalSparkCluster(
// Disable REST server on Master in this mode unless otherwise specified
val _conf = conf.clone()
.setIfMissing("spark.master.rest.enabled", "false")
- .set("spark.shuffle.service.enabled", "false")
+ .set(config.SHUFFLE_SERVICE_ENABLED.key, "false")
/* Start the Master */
val (rpcEnv, webUiPort, _) = Master.startRpcEnvAndEndpoint(localHostname, 0, 0, _conf)
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index 8353e64a619cf..4cc0063d010ef 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -31,7 +31,6 @@ import scala.util.control.NonFatal
import com.google.common.primitives.Longs
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter}
-import org.apache.hadoop.fs.permission.FsAction
import org.apache.hadoop.mapred.JobConf
import org.apache.hadoop.security.{Credentials, UserGroupInformation}
import org.apache.hadoop.security.token.{Token, TokenIdentifier}
@@ -108,7 +107,7 @@ class SparkHadoopUtil extends Logging {
}
/**
- * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop
+ * Return an appropriate (subclass) of Configuration. Creating config can initialize some Hadoop
* subsystems.
*/
def newConfiguration(conf: SparkConf): Configuration = {
@@ -367,28 +366,6 @@ class SparkHadoopUtil extends Logging {
buffer.toString
}
- private[spark] def checkAccessPermission(status: FileStatus, mode: FsAction): Boolean = {
- val perm = status.getPermission
- val ugi = UserGroupInformation.getCurrentUser
-
- if (ugi.getShortUserName == status.getOwner) {
- if (perm.getUserAction.implies(mode)) {
- return true
- }
- } else if (ugi.getGroupNames.contains(status.getGroup)) {
- if (perm.getGroupAction.implies(mode)) {
- return true
- }
- } else if (perm.getOtherAction.implies(mode)) {
- return true
- }
-
- logDebug(s"Permission denied: user=${ugi.getShortUserName}, " +
- s"path=${status.getPath}:${status.getOwner}:${status.getGroup}" +
- s"${if (status.isDirectory) "d" else "-"}$perm")
- false
- }
-
def serialize(creds: Credentials): Array[Byte] = {
val byteStream = new ByteArrayOutputStream
val dataStream = new DataOutputStream(byteStream)
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
index e83d82f847c61..cf902db8709e7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -181,6 +181,7 @@ private[spark] class SparkSubmit extends Logging {
if (args.isStandaloneCluster && args.useRest) {
try {
logInfo("Running Spark using the REST application submission protocol.")
+ doRunMain()
} catch {
// Fail over to use the legacy submission gateway
case e: SubmitRestConnectionException =>
@@ -285,8 +286,6 @@ private[spark] class SparkSubmit extends Logging {
case (STANDALONE, CLUSTER) if args.isR =>
error("Cluster deploy mode is currently not supported for R " +
"applications on standalone clusters.")
- case (KUBERNETES, _) if args.isR =>
- error("R applications are currently not supported for Kubernetes.")
case (LOCAL, CLUSTER) =>
error("Cluster deploy mode is not compatible with master \"local\"")
case (_, CLUSTER) if isShell(args.primaryResource) =>
@@ -385,7 +384,7 @@ private[spark] class SparkSubmit extends Logging {
val forceDownloadSchemes = sparkConf.get(FORCE_DOWNLOAD_SCHEMES)
def shouldDownload(scheme: String): Boolean = {
- forceDownloadSchemes.contains(scheme) ||
+ forceDownloadSchemes.contains("*") || forceDownloadSchemes.contains(scheme) ||
Try { FileSystem.getFileSystemClass(scheme, hadoopConf) }.isFailure
}
@@ -578,7 +577,8 @@ private[spark] class SparkSubmit extends Logging {
}
// Add the main application jar and any added jars to classpath in case YARN client
// requires these jars.
- // This assumes both primaryResource and user jars are local jars, otherwise it will not be
+ // This assumes both primaryResource and user jars are local jars, or already downloaded
+ // to local by configuring "spark.yarn.dist.forceDownloadSchemes", otherwise it will not be
// added to the classpath of YARN client.
if (isYarnCluster) {
if (isUserJar(args.primaryResource)) {
@@ -698,10 +698,16 @@ private[spark] class SparkSubmit extends Logging {
if (args.pyFiles != null) {
childArgs ++= Array("--other-py-files", args.pyFiles)
}
- } else {
+ } else if (args.isR) {
+ childArgs ++= Array("--primary-r-file", args.primaryResource)
+ childArgs ++= Array("--main-class", "org.apache.spark.deploy.RRunner")
+ }
+ else {
childArgs ++= Array("--primary-java-resource", args.primaryResource)
childArgs ++= Array("--main-class", args.mainClass)
}
+ } else {
+ childArgs ++= Array("--main-class", args.mainClass)
}
if (args.childArgs != null) {
args.childArgs.foreach { arg =>
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
index fb232101114b9..0998757715457 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -82,7 +82,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
var driverCores: String = null
var submissionToKill: String = null
var submissionToRequestStatusFor: String = null
- var useRest: Boolean = true // used internally
+ var useRest: Boolean = false // used internally
/** Default properties present in the currently defined defaults file. */
lazy val defaultSparkProperties: HashMap[String, String] = {
@@ -115,6 +115,8 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S
// Use `sparkProperties` map along with env vars to fill in any missing parameters
loadEnvironmentArguments()
+ useRest = sparkProperties.getOrElse("spark.master.rest.enabled", "false").toBoolean
+
validateArguments()
/**
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index bf1eeb0c1bf59..44d23908146c7 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -21,11 +21,12 @@ import java.io.{File, FileNotFoundException, IOException}
import java.nio.file.Files
import java.nio.file.attribute.PosixFilePermissions
import java.util.{Date, ServiceLoader}
-import java.util.concurrent.{ExecutorService, TimeUnit}
+import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit}
import java.util.zip.{ZipEntry, ZipOutputStream}
import scala.collection.JavaConverters._
import scala.collection.mutable
+import scala.concurrent.ExecutionException
import scala.io.Source
import scala.util.Try
import scala.xml.Node
@@ -33,8 +34,7 @@ import scala.xml.Node
import com.fasterxml.jackson.annotation.JsonIgnore
import com.google.common.io.ByteStreams
import com.google.common.util.concurrent.MoreExecutors
-import org.apache.hadoop.fs.{FileStatus, Path}
-import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
import org.apache.hadoop.hdfs.DistributedFileSystem
import org.apache.hadoop.hdfs.protocol.HdfsConstants
import org.apache.hadoop.security.AccessControlException
@@ -114,7 +114,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
"; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString)
private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf)
- private val fs = new Path(logDir).getFileSystem(hadoopConf)
+ // Visible for testing
+ private[history] val fs: FileSystem = new Path(logDir).getFileSystem(hadoopConf)
// Used by check event thread and clean log thread.
// Scheduled thread pool size must be one, otherwise it will have concurrent issues about fs
@@ -161,6 +162,25 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
new HistoryServerDiskManager(conf, path, listing, clock)
}
+ private val blacklist = new ConcurrentHashMap[String, Long]
+
+ // Visible for testing
+ private[history] def isBlacklisted(path: Path): Boolean = {
+ blacklist.containsKey(path.getName)
+ }
+
+ private def blacklist(path: Path): Unit = {
+ blacklist.put(path.getName, clock.getTimeMillis())
+ }
+
+ /**
+ * Removes expired entries in the blacklist, according to the provided `expireTimeInSeconds`.
+ */
+ private def clearBlacklist(expireTimeInSeconds: Long): Unit = {
+ val expiredThreshold = clock.getTimeMillis() - expireTimeInSeconds * 1000
+ blacklist.asScala.retain((_, creationTime) => creationTime >= expiredThreshold)
+ }
+
private val activeUIs = new mutable.HashMap[(String, Option[String]), LoadedAppUI]()
/**
@@ -418,7 +438,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
// reading a garbage file is safe, but we would log an error which can be scary to
// the end-user.
!entry.getPath().getName().startsWith(".") &&
- SparkHadoopUtil.get.checkAccessPermission(entry, FsAction.READ)
+ !isBlacklisted(entry.getPath)
}
.filter { entry =>
try {
@@ -461,32 +481,37 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
logDebug(s"New/updated attempts found: ${updated.size} ${updated.map(_.getPath)}")
}
- val tasks = updated.map { entry =>
+ val tasks = updated.flatMap { entry =>
try {
- replayExecutor.submit(new Runnable {
+ val task: Future[Unit] = replayExecutor.submit(new Runnable {
override def run(): Unit = mergeApplicationListing(entry, newLastScanTime, true)
- })
+ }, Unit)
+ Some(task -> entry.getPath)
} catch {
// let the iteration over the updated entries break, since an exception on
// replayExecutor.submit (..) indicates the ExecutorService is unable
// to take any more submissions at this time
case e: Exception =>
logError(s"Exception while submitting event log for replay", e)
- null
+ None
}
- }.filter(_ != null)
+ }
pendingReplayTasksCount.addAndGet(tasks.size)
// Wait for all tasks to finish. This makes sure that checkForLogs
// is not scheduled again while some tasks are already running in
// the replayExecutor.
- tasks.foreach { task =>
+ tasks.foreach { case (task, path) =>
try {
task.get()
} catch {
case e: InterruptedException =>
throw e
+ case e: ExecutionException if e.getCause.isInstanceOf[AccessControlException] =>
+ // We don't have read permissions on the log file
+ logWarning(s"Unable to read log $path", e.getCause)
+ blacklist(path)
case e: Exception =>
logError("Exception while merging application listings", e)
} finally {
@@ -779,6 +804,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
listing.delete(classOf[LogInfo], log.logPath)
}
}
+ // Clean the blacklist from the expired entries.
+ clearBlacklist(CLEAN_INTERVAL_S)
}
/**
@@ -938,13 +965,17 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
}
private def deleteLog(log: Path): Unit = {
- try {
- fs.delete(log, true)
- } catch {
- case _: AccessControlException =>
- logInfo(s"No permission to delete $log, ignoring.")
- case ioe: IOException =>
- logError(s"IOException in cleaning $log", ioe)
+ if (isBlacklisted(log)) {
+ logDebug(s"Skipping deleting $log as we don't have permissions on it.")
+ } else {
+ try {
+ fs.delete(log, true)
+ } catch {
+ case _: AccessControlException =>
+ logInfo(s"No permission to delete $log, ignoring.")
+ case ioe: IOException =>
+ logError(s"IOException in cleaning $log", ioe)
+ }
}
}
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 2c78c15773af2..e1184248af460 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
@@ -121,10 +121,18 @@ private[deploy] class Master(
}
// Alternative application submission gateway that is stable across Spark versions
- private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", true)
+ private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", false)
private var restServer: Option[StandaloneRestServer] = None
private var restServerBoundPort: Option[Int] = None
+ {
+ val authKey = SecurityManager.SPARK_AUTH_SECRET_CONF
+ require(conf.getOption(authKey).isEmpty || !restServerEnabled,
+ s"The RestSubmissionServer does not support authentication via ${authKey}. Either turn " +
+ "off the RestSubmissionServer with spark.master.rest.enabled=false, or do not use " +
+ "authentication.")
+ }
+
override def onStart(): Unit = {
logInfo("Starting Spark master at " + masterUrl)
logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}")
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
index 742a95841a138..31a8e3e60c067 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala
@@ -233,30 +233,44 @@ private[spark] class RestSubmissionClient(master: String) extends Logging {
private[rest] def readResponse(connection: HttpURLConnection): SubmitRestProtocolResponse = {
import scala.concurrent.ExecutionContext.Implicits.global
val responseFuture = Future {
- val dataStream =
- if (connection.getResponseCode == HttpServletResponse.SC_OK) {
- connection.getInputStream
- } else {
- connection.getErrorStream
+ val responseCode = connection.getResponseCode
+
+ if (responseCode != HttpServletResponse.SC_OK) {
+ val errString = Some(Source.fromInputStream(connection.getErrorStream())
+ .getLines().mkString("\n"))
+ if (responseCode == HttpServletResponse.SC_INTERNAL_SERVER_ERROR &&
+ !connection.getContentType().contains("application/json")) {
+ throw new SubmitRestProtocolException(s"Server responded with exception:\n${errString}")
+ }
+ logError(s"Server responded with error:\n${errString}")
+ val error = new ErrorResponse
+ if (responseCode == RestSubmissionServer.SC_UNKNOWN_PROTOCOL_VERSION) {
+ error.highestProtocolVersion = RestSubmissionServer.PROTOCOL_VERSION
+ }
+ error.message = errString.get
+ error
+ } else {
+ val dataStream = connection.getInputStream
+
+ // If the server threw an exception while writing a response, it will not have a body
+ if (dataStream == null) {
+ throw new SubmitRestProtocolException("Server returned empty body")
+ }
+ val responseJson = Source.fromInputStream(dataStream).mkString
+ logDebug(s"Response from the server:\n$responseJson")
+ val response = SubmitRestProtocolMessage.fromJson(responseJson)
+ response.validate()
+ response match {
+ // If the response is an error, log the message
+ case error: ErrorResponse =>
+ logError(s"Server responded with error:\n${error.message}")
+ error
+ // Otherwise, simply return the response
+ case response: SubmitRestProtocolResponse => response
+ case unexpected =>
+ throw new SubmitRestProtocolException(
+ s"Message received from server was not a response:\n${unexpected.toJson}")
}
- // If the server threw an exception while writing a response, it will not have a body
- if (dataStream == null) {
- throw new SubmitRestProtocolException("Server returned empty body")
- }
- val responseJson = Source.fromInputStream(dataStream).mkString
- logDebug(s"Response from the server:\n$responseJson")
- val response = SubmitRestProtocolMessage.fromJson(responseJson)
- response.validate()
- response match {
- // If the response is an error, log the message
- case error: ErrorResponse =>
- logError(s"Server responded with error:\n${error.message}")
- error
- // Otherwise, simply return the response
- case response: SubmitRestProtocolResponse => response
- case unexpected =>
- throw new SubmitRestProtocolException(
- s"Message received from server was not a response:\n${unexpected.toJson}")
}
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala
index 3d99d085408c6..e59bf3f0eaf44 100644
--- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionServer.scala
@@ -51,6 +51,7 @@ private[spark] abstract class RestSubmissionServer(
val host: String,
val requestedPort: Int,
val masterConf: SparkConf) extends Logging {
+
protected val submitRequestServlet: SubmitRequestServlet
protected val killRequestServlet: KillRequestServlet
protected val statusRequestServlet: StatusRequestServlet
diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
index ee1ca0bba5749..d5ea2523c628b 100755
--- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
@@ -36,7 +36,7 @@ import org.apache.spark.deploy.DeployMessages._
import org.apache.spark.deploy.ExternalShuffleService
import org.apache.spark.deploy.master.{DriverState, Master}
import org.apache.spark.deploy.worker.ui.WorkerWebUI
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.metrics.MetricsSystem
import org.apache.spark.rpc._
import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils}
@@ -758,6 +758,7 @@ private[deploy] class Worker(
private[deploy] object Worker extends Logging {
val SYSTEM_NAME = "sparkWorker"
val ENDPOINT_NAME = "Worker"
+ private val SSL_NODE_LOCAL_CONFIG_PATTERN = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r
def main(argStrings: Array[String]) {
Thread.setDefaultUncaughtExceptionHandler(new SparkUncaughtExceptionHandler(
@@ -772,7 +773,7 @@ private[deploy] object Worker extends Logging {
// bound, we may launch no more than one external shuffle service on each host.
// When this happens, we should give explicit reason of failure instead of fail silently. For
// more detail see SPARK-20989.
- val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
+ val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED)
val sparkWorkerInstances = scala.sys.env.getOrElse("SPARK_WORKER_INSTANCES", "1").toInt
require(externalShuffleServiceEnabled == false || sparkWorkerInstances <= 1,
"Starting multiple workers on one host is failed because we may launch no more than one " +
@@ -803,9 +804,8 @@ private[deploy] object Worker extends Logging {
}
def isUseLocalNodeSSLConfig(cmd: Command): Boolean = {
- val pattern = """\-Dspark\.ssl\.useNodeLocalConf\=(.+)""".r
val result = cmd.javaOpts.collectFirst {
- case pattern(_result) => _result.toBoolean
+ case SSL_NODE_LOCAL_CONFIG_PATTERN(_result) => _result.toBoolean
}
result.getOrElse(false)
}
diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index b1856ff0f3247..072277cb78dc1 100644
--- a/core/src/main/scala/org/apache/spark/executor/Executor.scala
+++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala
@@ -38,7 +38,7 @@ import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.rpc.RpcTimeout
-import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
+import org.apache.spark.scheduler._
import org.apache.spark.shuffle.FetchFailedException
import org.apache.spark.storage.{StorageLevel, TaskResultBlockId}
import org.apache.spark.util._
@@ -148,7 +148,8 @@ private[spark] class Executor(
private val runningTasks = new ConcurrentHashMap[Long, TaskRunner]
// Executor for the heartbeat task.
- private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater")
+ private val heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat,
+ "executor-heartbeater", conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))
// must be initialized before running startDriverHeartbeat()
private val heartbeatReceiverRef =
@@ -167,7 +168,7 @@ private[spark] class Executor(
*/
private var heartbeatFailures = 0
- startDriverHeartbeater()
+ heartbeater.start()
private[executor] def numRunningTasks: Int = runningTasks.size()
@@ -216,8 +217,12 @@ private[spark] class Executor(
def stop(): Unit = {
env.metricsSystem.report()
- heartbeater.shutdown()
- heartbeater.awaitTermination(10, TimeUnit.SECONDS)
+ try {
+ heartbeater.stop()
+ } catch {
+ case NonFatal(e) =>
+ logWarning("Unable to stop heartbeater", e)
+ }
threadPool.shutdown()
if (!isLocal) {
env.stop()
@@ -363,14 +368,14 @@ private[spark] class Executor(
threadMXBean.getCurrentThreadCpuTime
} else 0L
var threwException = true
- val value = try {
+ val value = Utils.tryWithSafeFinally {
val res = task.run(
taskAttemptId = taskId,
attemptNumber = taskDescription.attemptNumber,
metricsSystem = env.metricsSystem)
threwException = false
res
- } finally {
+ } {
val releasedLocks = env.blockManager.releaseAllLocksForTask(taskId)
val freedMemory = taskMemoryManager.cleanUpAllAllocatedMemory()
@@ -787,6 +792,9 @@ private[spark] class Executor(
val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]()
val curGCTime = computeTotalGcTime()
+ // get executor level memory metrics
+ val executorUpdates = heartbeater.getCurrentMetrics()
+
for (taskRunner <- runningTasks.values().asScala) {
if (taskRunner.task != null) {
taskRunner.task.metrics.mergeShuffleReadMetrics()
@@ -795,7 +803,8 @@ private[spark] class Executor(
}
}
- val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId)
+ val message = Heartbeat(executorId, accumUpdates.toArray, env.blockManager.blockManagerId,
+ executorUpdates)
try {
val response = heartbeatReceiverRef.askSync[HeartbeatResponse](
message, RpcTimeout(conf, "spark.executor.heartbeatInterval", "10s"))
@@ -815,21 +824,6 @@ private[spark] class Executor(
}
}
}
-
- /**
- * Schedules a task to report heartbeat and partial metrics for active tasks to driver.
- */
- private def startDriverHeartbeater(): Unit = {
- val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s")
-
- // Wait a random interval so the heartbeats don't end up in sync
- val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int]
-
- val heartbeatTask = new Runnable() {
- override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat())
- }
- heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS)
- }
}
private[spark] object Executor {
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
new file mode 100644
index 0000000000000..2933f3ba6d3b5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.executor
+
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.metrics.ExecutorMetricType
+
+/**
+ * :: DeveloperApi ::
+ * Metrics tracked for executors and the driver.
+ *
+ * Executor-level metrics are sent from each executor to the driver as part of the Heartbeat.
+ */
+@DeveloperApi
+class ExecutorMetrics private[spark] extends Serializable {
+
+ // Metrics are indexed by MetricGetter.values
+ private val metrics = new Array[Long](ExecutorMetricType.values.length)
+
+ // the first element is initialized to -1, indicating that the values for the array
+ // haven't been set yet.
+ metrics(0) = -1
+
+ /** Returns the value for the specified metricType. */
+ def getMetricValue(metricType: ExecutorMetricType): Long = {
+ metrics(ExecutorMetricType.metricIdxMap(metricType))
+ }
+
+ /** Returns true if the values for the metrics have been set, false otherwise. */
+ def isSet(): Boolean = metrics(0) > -1
+
+ private[spark] def this(metrics: Array[Long]) {
+ this()
+ Array.copy(metrics, 0, this.metrics, 0, Math.min(metrics.size, this.metrics.size))
+ }
+
+ /**
+ * Constructor: create the ExecutorMetrics with the values specified.
+ *
+ * @param executorMetrics map of executor metric name to value
+ */
+ private[spark] def this(executorMetrics: Map[String, Long]) {
+ this()
+ (0 until ExecutorMetricType.values.length).foreach { idx =>
+ metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L)
+ }
+ }
+
+ /**
+ * Compare the specified executor metrics values with the current executor metric values,
+ * and update the value for any metrics where the new value for the metric is larger.
+ *
+ * @param executorMetrics the executor metrics to compare
+ * @return if there is a new peak value for any metric
+ */
+ private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = {
+ var updated = false
+
+ (0 until ExecutorMetricType.values.length).foreach { idx =>
+ if (executorMetrics.metrics(idx) > metrics(idx)) {
+ updated = true
+ metrics(idx) = executorMetrics.metrics(idx)
+ }
+ }
+ updated
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
index 669ce63325d0e..a8264022a0aff 100644
--- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
+++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala
@@ -17,9 +17,12 @@
package org.apache.spark.executor
+import java.lang.management.ManagementFactory
import java.util.concurrent.ThreadPoolExecutor
+import javax.management.{MBeanServer, ObjectName}
import scala.collection.JavaConverters._
+import scala.util.control.NonFatal
import com.codahale.metrics.{Gauge, MetricRegistry}
import org.apache.hadoop.fs.FileSystem
@@ -73,6 +76,24 @@ class ExecutorSource(threadPool: ThreadPoolExecutor, executorId: String) extends
registerFileSystemStat(scheme, "write_ops", _.getWriteOps(), 0)
}
+ // Dropwizard metrics gauge measuring the executor's process CPU time.
+ // This Gauge will try to get and return the JVM Process CPU time or return -1 otherwise.
+ // The CPU time value is returned in nanoseconds.
+ // It will use proprietary extensions such as com.sun.management.OperatingSystemMXBean or
+ // com.ibm.lang.management.OperatingSystemMXBean, if available.
+ metricRegistry.register(MetricRegistry.name("jvmCpuTime"), new Gauge[Long] {
+ val mBean: MBeanServer = ManagementFactory.getPlatformMBeanServer
+ val name = new ObjectName("java.lang", "type", "OperatingSystem")
+ override def getValue: Long = {
+ try {
+ // return JVM process CPU time if the ProcessCpuTime method is available
+ mBean.getAttribute(name, "ProcessCpuTime").asInstanceOf[Long]
+ } catch {
+ case NonFatal(_) => -1L
+ }
+ }
+ })
+
// Expose executor task metrics using the Dropwizard metrics system.
// The list is taken from TaskMetrics.scala
val METRIC_CPU_TIME = metricRegistry.counter(MetricRegistry.name("cpuTime"))
diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
index 17cdba4f1305b..ab020aaf6fa4f 100644
--- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
+++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala
@@ -47,7 +47,7 @@ private[spark] abstract class StreamFileInputFormat[T]
def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int) {
val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES)
val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES)
- val defaultParallelism = sc.defaultParallelism
+ val defaultParallelism = Math.max(sc.defaultParallelism, minPartitions)
val files = listStatus(context).asScala
val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum
val bytesPerCore = totalBytes / defaultParallelism
diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
index f47cd38d712c3..04c5c4b90e8a1 100644
--- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
+++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala
@@ -53,6 +53,19 @@ private[spark] class WholeTextFileInputFormat
val totalLen = files.map(file => if (file.isDirectory) 0L else file.getLen).sum
val maxSplitSize = Math.ceil(totalLen * 1.0 /
(if (minPartitions == 0) 1 else minPartitions)).toLong
+
+ // For small files we need to ensure the min split size per node & rack <= maxSplitSize
+ val config = context.getConfiguration
+ val minSplitSizePerNode = config.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 0L)
+ val minSplitSizePerRack = config.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 0L)
+
+ if (maxSplitSize < minSplitSizePerNode) {
+ super.setMinSplitSizeNode(maxSplitSize)
+ }
+
+ if (maxSplitSize < minSplitSizePerRack) {
+ super.setMinSplitSizeRack(maxSplitSize)
+ }
super.setMaxSplitSize(maxSplitSize)
}
}
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 38a043c85ae33..8d827189ebb57 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
@@ -69,9 +69,17 @@ package object config {
.bytesConf(ByteUnit.KiB)
.createWithDefaultString("100k")
+ private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS =
+ ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled")
+ .booleanConf
+ .createWithDefault(false)
+
private[spark] val EVENT_LOG_OVERWRITE =
ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false)
+ private[spark] val EVENT_LOG_CALLSITE_LONG_FORM =
+ ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false)
+
private[spark] val EXECUTOR_CLASS_PATH =
ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional
@@ -111,6 +119,10 @@ package object config {
.checkValue(_ >= 0, "The off-heap memory size must not be negative")
.createWithDefault(0)
+ private[spark] val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory")
+ .bytesConf(ByteUnit.MiB)
+ .createOptional
+
private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal()
.booleanConf.createWithDefault(false)
@@ -137,6 +149,9 @@ package object config {
private[spark] val SHUFFLE_SERVICE_ENABLED =
ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false)
+ private[spark] val SHUFFLE_SERVICE_PORT =
+ ConfigBuilder("spark.shuffle.service.port").intConf.createWithDefault(7337)
+
private[spark] val KEYTAB = ConfigBuilder("spark.yarn.keytab")
.doc("Location of user's keytab.")
.stringConf.createOptional
@@ -429,7 +444,11 @@ package object config {
"external shuffle service, this feature can only be worked when external shuffle" +
"service is newer than Spark 2.2.")
.bytesConf(ByteUnit.BYTE)
- .createWithDefault(Long.MaxValue)
+ // fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might
+ // as well use fetch-to-disk in that case. The message includes some metadata in addition
+ // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave
+ // extra room.
+ .createWithDefault(Int.MaxValue - 512)
private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES =
ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses")
@@ -483,10 +502,11 @@ package object config {
private[spark] val FORCE_DOWNLOAD_SCHEMES =
ConfigBuilder("spark.yarn.dist.forceDownloadSchemes")
- .doc("Comma-separated list of schemes for which files will be downloaded to the " +
+ .doc("Comma-separated list of schemes for which resources will be downloaded to the " +
"local disk prior to being added to YARN's distributed cache. For use in cases " +
"where the YARN service does not support schemes that are supported by Spark, like http, " +
- "https and ftp.")
+ "https and ftp, or jars required to be in the local YARN client's classpath. Wildcard " +
+ "'*' is denoted to download resources for all the schemes.")
.stringConf
.toSequence
.createWithDefault(Nil)
@@ -559,4 +579,48 @@ package object config {
.intConf
.checkValue(v => v > 0, "The value should be a positive integer.")
.createWithDefault(2000)
+
+ private[spark] val MEMORY_MAP_LIMIT_FOR_TESTS =
+ ConfigBuilder("spark.storage.memoryMapLimitForTests")
+ .internal()
+ .doc("For testing only, controls the size of chunks when memory mapping a file")
+ .bytesConf(ByteUnit.BYTE)
+ .createWithDefault(Int.MaxValue)
+
+ private[spark] val BARRIER_SYNC_TIMEOUT =
+ ConfigBuilder("spark.barrier.sync.timeout")
+ .doc("The timeout in seconds for each barrier() call from a barrier task. If the " +
+ "coordinator didn't receive all the sync messages from barrier tasks within the " +
+ "configed time, throw a SparkException to fail all the tasks. The default value is set " +
+ "to 31536000(3600 * 24 * 365) so the barrier() call shall wait for one year.")
+ .timeConf(TimeUnit.SECONDS)
+ .checkValue(v => v > 0, "The value should be a positive time value.")
+ .createWithDefaultString("365d")
+
+ private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL =
+ ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.interval")
+ .doc("Time in seconds to wait between a max concurrent tasks check failure and the next " +
+ "check. A max concurrent tasks check ensures the cluster can launch more concurrent " +
+ "tasks than required by a barrier stage on job submitted. The check can fail in case " +
+ "a cluster has just started and not enough executors have registered, so we wait for a " +
+ "little while and try to perform the check again. If the check fails more than a " +
+ "configured max failure times for a job then fail current job submission. Note this " +
+ "config only applies to jobs that contain one or more barrier stages, we won't perform " +
+ "the check on non-barrier jobs.")
+ .timeConf(TimeUnit.SECONDS)
+ .createWithDefaultString("15s")
+
+ private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES =
+ ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures")
+ .doc("Number of max concurrent tasks check failures allowed before fail a job submission. " +
+ "A max concurrent tasks check ensures the cluster can launch more concurrent tasks than " +
+ "required by a barrier stage on job submitted. The check can fail in case a cluster " +
+ "has just started and not enough executors have registered, so we wait for a little " +
+ "while and try to perform the check again. If the check fails more than a configured " +
+ "max failure times for a job then fail current job submission. Note this config only " +
+ "applies to jobs that contain one or more barrier stages, we won't perform the check on " +
+ "non-barrier jobs.")
+ .intConf
+ .checkValue(v => v > 0, "The max failures should be a positive value.")
+ .createWithDefault(40)
}
diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
index 0641adc2ab699..4fde2d0beaa71 100644
--- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
+++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
@@ -180,6 +180,34 @@ private[spark] abstract class MemoryManager(
onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed
}
+ /**
+ * On heap execution memory currently in use, in bytes.
+ */
+ final def onHeapExecutionMemoryUsed: Long = synchronized {
+ onHeapExecutionMemoryPool.memoryUsed
+ }
+
+ /**
+ * Off heap execution memory currently in use, in bytes.
+ */
+ final def offHeapExecutionMemoryUsed: Long = synchronized {
+ offHeapExecutionMemoryPool.memoryUsed
+ }
+
+ /**
+ * On heap storage memory currently in use, in bytes.
+ */
+ final def onHeapStorageMemoryUsed: Long = synchronized {
+ onHeapStorageMemoryPool.memoryUsed
+ }
+
+ /**
+ * Off heap storage memory currently in use, in bytes.
+ */
+ final def offHeapStorageMemoryUsed: Long = synchronized {
+ offHeapStorageMemoryPool.memoryUsed
+ }
+
/**
* Returns the execution memory consumption, in bytes, for the given task.
*/
diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
new file mode 100644
index 0000000000000..cd10dad25e87b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala
@@ -0,0 +1,104 @@
+/*
+ * 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.metrics
+
+import java.lang.management.{BufferPoolMXBean, ManagementFactory}
+import javax.management.ObjectName
+
+import org.apache.spark.memory.MemoryManager
+
+/**
+ * Executor metric types for executor-level metrics stored in ExecutorMetrics.
+ */
+sealed trait ExecutorMetricType {
+ private[spark] def getMetricValue(memoryManager: MemoryManager): Long
+ private[spark] val name = getClass().getName().stripSuffix("$").split("""\.""").last
+}
+
+private[spark] abstract class MemoryManagerExecutorMetricType(
+ f: MemoryManager => Long) extends ExecutorMetricType {
+ override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
+ f(memoryManager)
+ }
+}
+
+private[spark] abstract class MBeanExecutorMetricType(mBeanName: String)
+ extends ExecutorMetricType {
+ private val bean = ManagementFactory.newPlatformMXBeanProxy(
+ ManagementFactory.getPlatformMBeanServer,
+ new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean])
+
+ override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
+ bean.getMemoryUsed
+ }
+}
+
+case object JVMHeapMemory extends ExecutorMetricType {
+ override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
+ ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed()
+ }
+}
+
+case object JVMOffHeapMemory extends ExecutorMetricType {
+ override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
+ ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed()
+ }
+}
+
+case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType(
+ _.onHeapExecutionMemoryUsed)
+
+case object OffHeapExecutionMemory extends MemoryManagerExecutorMetricType(
+ _.offHeapExecutionMemoryUsed)
+
+case object OnHeapStorageMemory extends MemoryManagerExecutorMetricType(
+ _.onHeapStorageMemoryUsed)
+
+case object OffHeapStorageMemory extends MemoryManagerExecutorMetricType(
+ _.offHeapStorageMemoryUsed)
+
+case object OnHeapUnifiedMemory extends MemoryManagerExecutorMetricType(
+ (m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed))
+
+case object OffHeapUnifiedMemory extends MemoryManagerExecutorMetricType(
+ (m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed))
+
+case object DirectPoolMemory extends MBeanExecutorMetricType(
+ "java.nio:type=BufferPool,name=direct")
+
+case object MappedPoolMemory extends MBeanExecutorMetricType(
+ "java.nio:type=BufferPool,name=mapped")
+
+private[spark] object ExecutorMetricType {
+ // List of all executor metric types
+ val values = IndexedSeq(
+ JVMHeapMemory,
+ JVMOffHeapMemory,
+ OnHeapExecutionMemory,
+ OffHeapExecutionMemory,
+ OnHeapStorageMemory,
+ OffHeapStorageMemory,
+ OnHeapUnifiedMemory,
+ OffHeapUnifiedMemory,
+ DirectPoolMemory,
+ MappedPoolMemory
+ )
+
+ // Map of executor metric type to its index in values.
+ val metricIdxMap =
+ Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*)
+}
diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala
index b3f8bfe8b1d48..e94a01244474c 100644
--- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala
+++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala
@@ -20,6 +20,7 @@ package org.apache.spark.network
import scala.reflect.ClassTag
import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.client.StreamCallbackWithID
import org.apache.spark.storage.{BlockId, StorageLevel}
private[spark]
@@ -43,6 +44,17 @@ trait BlockDataManager {
level: StorageLevel,
classTag: ClassTag[_]): Boolean
+ /**
+ * Put the given block that will be received as a stream.
+ *
+ * When this method is called, the block data itself is not available -- it will be passed to the
+ * returned StreamCallbackWithID.
+ */
+ def putBlockDataAsStream(
+ blockId: BlockId,
+ level: StorageLevel,
+ classTag: ClassTag[_]): StreamCallbackWithID
+
/**
* Release locks acquired by [[putBlockData()]] and [[getBlockData()]].
*/
diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
index eb4cf94164fd4..7076701421e2e 100644
--- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
+++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala
@@ -26,9 +26,9 @@ import scala.reflect.ClassTag
import org.apache.spark.internal.Logging
import org.apache.spark.network.BlockDataManager
import org.apache.spark.network.buffer.NioManagedBuffer
-import org.apache.spark.network.client.{RpcResponseCallback, TransportClient}
+import org.apache.spark.network.client.{RpcResponseCallback, StreamCallbackWithID, TransportClient}
import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager}
-import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, OpenBlocks, StreamHandle, UploadBlock}
+import org.apache.spark.network.shuffle.protocol._
import org.apache.spark.serializer.Serializer
import org.apache.spark.storage.{BlockId, StorageLevel}
@@ -73,10 +73,32 @@ class NettyBlockRpcServer(
}
val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData))
val blockId = BlockId(uploadBlock.blockId)
+ logDebug(s"Receiving replicated block $blockId with level ${level} " +
+ s"from ${client.getSocketAddress}")
blockManager.putBlockData(blockId, data, level, classTag)
responseContext.onSuccess(ByteBuffer.allocate(0))
}
}
+ override def receiveStream(
+ client: TransportClient,
+ messageHeader: ByteBuffer,
+ responseContext: RpcResponseCallback): StreamCallbackWithID = {
+ val message =
+ BlockTransferMessage.Decoder.fromByteBuffer(messageHeader).asInstanceOf[UploadBlockStream]
+ val (level: StorageLevel, classTag: ClassTag[_]) = {
+ serializer
+ .newInstance()
+ .deserialize(ByteBuffer.wrap(message.metadata))
+ .asInstanceOf[(StorageLevel, ClassTag[_])]
+ }
+ val blockId = BlockId(message.blockId)
+ logDebug(s"Receiving replicated block $blockId with level ${level} as stream " +
+ s"from ${client.getSocketAddress}")
+ // This will return immediately, but will setup a callback on streamData which will still
+ // do all the processing in the netty thread.
+ blockManager.putBlockDataAsStream(blockId, level, classTag)
+ }
+
override def getStreamManager(): StreamManager = streamManager
}
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 b7d8c35032763..1905632a936d3 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
@@ -27,13 +27,14 @@ import scala.reflect.ClassTag
import com.codahale.metrics.{Metric, MetricSet}
import org.apache.spark.{SecurityManager, SparkConf}
+import org.apache.spark.internal.config
import org.apache.spark.network._
-import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
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, TempFileManager}
-import org.apache.spark.network.shuffle.protocol.UploadBlock
+import org.apache.spark.network.shuffle.protocol.{UploadBlock, UploadBlockStream}
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.serializer.JavaSerializer
import org.apache.spark.storage.{BlockId, StorageLevel}
@@ -148,20 +149,28 @@ private[spark] class NettyBlockTransferService(
// Everything else is encoded using our binary protocol.
val metadata = JavaUtils.bufferToArray(serializer.newInstance().serialize((level, classTag)))
- // Convert or copy nio buffer into array in order to serialize it.
- val array = JavaUtils.bufferToArray(blockData.nioByteBuffer())
+ val asStream = blockData.size() > conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM)
+ val callback = new RpcResponseCallback {
+ override def onSuccess(response: ByteBuffer): Unit = {
+ logTrace(s"Successfully uploaded block $blockId${if (asStream) " as stream" else ""}")
+ result.success((): Unit)
+ }
- 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")
- result.success((): Unit)
- }
- override def onFailure(e: Throwable): Unit = {
- logError(s"Error while uploading block $blockId", e)
- result.failure(e)
- }
- })
+ override def onFailure(e: Throwable): Unit = {
+ logError(s"Error while uploading $blockId${if (asStream) " as stream" else ""}", e)
+ result.failure(e)
+ }
+ }
+ if (asStream) {
+ val streamHeader = new UploadBlockStream(blockId.name, metadata).toByteBuffer
+ client.uploadStream(new NioManagedBuffer(streamHeader), blockData, callback)
+ } else {
+ // 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.name, metadata, array).toByteBuffer,
+ callback)
+ }
result.future
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
index 4e036c2ed49b5..23cf19d55b4ae 100644
--- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala
@@ -30,7 +30,7 @@ private[spark]
class BlockRDD[T: ClassTag](sc: SparkContext, @transient val blockIds: Array[BlockId])
extends RDD[T](sc, Nil) {
- @transient lazy val _locations = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get)
+ @transient lazy val _locations = BlockManager.blockIdsToLocations(blockIds, SparkEnv.get)
@volatile private var _isValid = true
override def getPartitions: Array[Partition] = {
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 44895abc7bd4d..3974580cfaa11 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -278,7 +278,7 @@ class HadoopRDD[K, V](
null
}
// Register an on-task-completion callback to close the input stream.
- context.addTaskCompletionListener { context =>
+ context.addTaskCompletionListener[Unit] { context =>
// Update the bytes read before closing is to make sure lingering bytesRead statistics in
// this thread get correctly added.
updateBytesRead()
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index aab46b8954bf7..56ef3e107a980 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -77,7 +77,7 @@ class JdbcRDD[T: ClassTag](
override def compute(thePart: Partition, context: TaskContext): Iterator[T] = new NextIterator[T]
{
- context.addTaskCompletionListener{ context => closeIfNeeded() }
+ context.addTaskCompletionListener[Unit]{ context => closeIfNeeded() }
val part = thePart.asInstanceOf[JdbcPartition]
val conn = getConnection()
val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)
diff --git a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
index e4587c96eae1c..aa61997122cf4 100644
--- a/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
@@ -23,11 +23,25 @@ import org.apache.spark.{Partition, TaskContext}
/**
* An RDD that applies the provided function to every partition of the parent RDD.
+ *
+ * @param prev the parent RDD.
+ * @param f The function used to map a tuple of (TaskContext, partition index, input iterator) to
+ * an output iterator.
+ * @param preservesPartitioning Whether the input function preserves the partitioner, which should
+ * be `false` unless `prev` is a pair RDD and the input function
+ * doesn't modify the keys.
+ * @param isFromBarrier Indicates whether this RDD is transformed from an RDDBarrier, a stage
+ * containing at least one RDDBarrier shall be turned into a barrier stage.
+ * @param isOrderSensitive whether or not the function is order-sensitive. If it's order
+ * sensitive, it may return totally different result when the input order
+ * is changed. Mostly stateful functions are order-sensitive.
*/
private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
var prev: RDD[T],
f: (TaskContext, Int, Iterator[T]) => Iterator[U], // (TaskContext, partition index, iterator)
- preservesPartitioning: Boolean = false)
+ preservesPartitioning: Boolean = false,
+ isFromBarrier: Boolean = false,
+ isOrderSensitive: Boolean = false)
extends RDD[U](prev) {
override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None
@@ -41,4 +55,15 @@ private[spark] class MapPartitionsRDD[U: ClassTag, T: ClassTag](
super.clearDependencies()
prev = null
}
+
+ @transient protected lazy override val isBarrier_ : Boolean =
+ isFromBarrier || dependencies.exists(_.rdd.isBarrier())
+
+ override protected def getOutputDeterministicLevel = {
+ if (isOrderSensitive && prev.outputDeterministicLevel == DeterministicLevel.UNORDERED) {
+ DeterministicLevel.INDETERMINATE
+ } else {
+ super.getOutputDeterministicLevel
+ }
+ }
}
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 ff66a04859d10..2d66d25ba39fa 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -214,7 +214,7 @@ class NewHadoopRDD[K, V](
}
// Register an on-task-completion callback to close the input stream.
- context.addTaskCompletionListener { context =>
+ context.addTaskCompletionListener[Unit] { context =>
// Update the bytesRead before closing is to make sure lingering bytesRead statistics in
// this thread get correctly added.
updateBytesRead()
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 0574abdca32ac..61ad6dfdb2215 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.TextOutputFormat
import org.apache.spark._
import org.apache.spark.Partitioner._
-import org.apache.spark.annotation.{DeveloperApi, Since}
+import org.apache.spark.annotation.{DeveloperApi, Experimental, Since}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.internal.Logging
import org.apache.spark.partial.BoundedDouble
@@ -462,8 +462,9 @@ abstract class RDD[T: ClassTag](
// include a shuffle step so that our upstream tasks are still distributed
new CoalescedRDD(
- new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition),
- new HashPartitioner(numPartitions)),
+ new ShuffledRDD[Int, T, T](
+ mapPartitionsWithIndexInternal(distributePartition, isOrderSensitive = true),
+ new HashPartitioner(numPartitions)),
numPartitions,
partitionCoalescer).values
} else {
@@ -807,16 +808,21 @@ abstract class RDD[T: ClassTag](
* serializable and don't require closure cleaning.
*
* @param preservesPartitioning indicates whether the input function preserves the partitioner,
- * which should be `false` unless this is a pair RDD and the input function doesn't modify
- * the keys.
+ * which should be `false` unless this is a pair RDD and the input
+ * function doesn't modify the keys.
+ * @param isOrderSensitive whether or not the function is order-sensitive. If it's order
+ * sensitive, it may return totally different result when the input order
+ * is changed. Mostly stateful functions are order-sensitive.
*/
private[spark] def mapPartitionsWithIndexInternal[U: ClassTag](
f: (Int, Iterator[T]) => Iterator[U],
- preservesPartitioning: Boolean = false): RDD[U] = withScope {
+ preservesPartitioning: Boolean = false,
+ isOrderSensitive: Boolean = false): RDD[U] = withScope {
new MapPartitionsRDD(
this,
(context: TaskContext, index: Int, iter: Iterator[T]) => f(index, iter),
- preservesPartitioning)
+ preservesPartitioning = preservesPartitioning,
+ isOrderSensitive = isOrderSensitive)
}
/**
@@ -1636,6 +1642,16 @@ abstract class RDD[T: ClassTag](
}
}
+ /**
+ * Return whether this RDD is reliably checkpointed and materialized.
+ */
+ private[rdd] def isReliablyCheckpointed: Boolean = {
+ checkpointData match {
+ case Some(reliable: ReliableRDDCheckpointData[_]) if reliable.isCheckpointed => true
+ case _ => false
+ }
+ }
+
/**
* Gets the name of the directory to which this RDD was checkpointed.
* This is not defined if the RDD is checkpointed locally.
@@ -1647,6 +1663,22 @@ abstract class RDD[T: ClassTag](
}
}
+ /**
+ * :: Experimental ::
+ * Marks the current stage as a barrier stage, where Spark must launch all tasks together.
+ * In case of a task failure, instead of only restarting the failed task, Spark will abort the
+ * entire stage and re-launch all tasks for this stage.
+ * The barrier execution mode feature is experimental and it only handles limited scenarios.
+ * Please read the linked SPIP and design docs to understand the limitations and future plans.
+ * @return an [[RDDBarrier]] instance that provides actions within a barrier stage
+ * @see [[org.apache.spark.BarrierTaskContext]]
+ * @see SPIP: Barrier Execution Mode
+ * @see Design Doc
+ */
+ @Experimental
+ @Since("2.4.0")
+ def barrier(): RDDBarrier[T] = withScope(new RDDBarrier[T](this))
+
// =======================================================================
// Other internal methods and fields
// =======================================================================
@@ -1839,6 +1871,81 @@ abstract class RDD[T: ClassTag](
def toJavaRDD() : JavaRDD[T] = {
new JavaRDD(this)(elementClassTag)
}
+
+ /**
+ * Whether the RDD is in a barrier stage. Spark must launch all the tasks at the same time for a
+ * barrier stage.
+ *
+ * An RDD is in a barrier stage, if at least one of its parent RDD(s), or itself, are mapped from
+ * an [[RDDBarrier]]. This function always returns false for a [[ShuffledRDD]], since a
+ * [[ShuffledRDD]] indicates start of a new stage.
+ *
+ * A [[MapPartitionsRDD]] can be transformed from an [[RDDBarrier]], under that case the
+ * [[MapPartitionsRDD]] shall be marked as barrier.
+ */
+ private[spark] def isBarrier(): Boolean = isBarrier_
+
+ // From performance concern, cache the value to avoid repeatedly compute `isBarrier()` on a long
+ // RDD chain.
+ @transient protected lazy val isBarrier_ : Boolean =
+ dependencies.filter(!_.isInstanceOf[ShuffleDependency[_, _, _]]).exists(_.rdd.isBarrier())
+
+ /**
+ * Returns the deterministic level of this RDD's output. Please refer to [[DeterministicLevel]]
+ * for the definition.
+ *
+ * By default, an reliably checkpointed RDD, or RDD without parents(root RDD) is DETERMINATE. For
+ * RDDs with parents, we will generate a deterministic level candidate per parent according to
+ * the dependency. The deterministic level of the current RDD is the deterministic level
+ * candidate that is deterministic least. Please override [[getOutputDeterministicLevel]] to
+ * provide custom logic of calculating output deterministic level.
+ */
+ // TODO: make it public so users can set deterministic level to their custom RDDs.
+ // TODO: this can be per-partition. e.g. UnionRDD can have different deterministic level for
+ // different partitions.
+ private[spark] final lazy val outputDeterministicLevel: DeterministicLevel.Value = {
+ if (isReliablyCheckpointed) {
+ DeterministicLevel.DETERMINATE
+ } else {
+ getOutputDeterministicLevel
+ }
+ }
+
+ @DeveloperApi
+ protected def getOutputDeterministicLevel: DeterministicLevel.Value = {
+ val deterministicLevelCandidates = dependencies.map {
+ // The shuffle is not really happening, treat it like narrow dependency and assume the output
+ // deterministic level of current RDD is same as parent.
+ case dep: ShuffleDependency[_, _, _] if dep.rdd.partitioner.exists(_ == dep.partitioner) =>
+ dep.rdd.outputDeterministicLevel
+
+ case dep: ShuffleDependency[_, _, _] =>
+ if (dep.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) {
+ // If map output was indeterminate, shuffle output will be indeterminate as well
+ DeterministicLevel.INDETERMINATE
+ } else if (dep.keyOrdering.isDefined && dep.aggregator.isDefined) {
+ // if aggregator specified (and so unique keys) and key ordering specified - then
+ // consistent ordering.
+ DeterministicLevel.DETERMINATE
+ } else {
+ // In Spark, the reducer fetches multiple remote shuffle blocks at the same time, and
+ // the arrival order of these shuffle blocks are totally random. Even if the parent map
+ // RDD is DETERMINATE, the reduce RDD is always UNORDERED.
+ DeterministicLevel.UNORDERED
+ }
+
+ // For narrow dependency, assume the output deterministic level of current RDD is same as
+ // parent.
+ case dep => dep.rdd.outputDeterministicLevel
+ }
+
+ if (deterministicLevelCandidates.isEmpty) {
+ // By default we assume the root RDD is determinate.
+ DeterministicLevel.DETERMINATE
+ } else {
+ deterministicLevelCandidates.maxBy(_.id)
+ }
+ }
}
@@ -1892,3 +1999,18 @@ object RDD {
new DoubleRDDFunctions(rdd.map(x => num.toDouble(x)))
}
}
+
+/**
+ * The deterministic level of RDD's output (i.e. what `RDD#compute` returns). This explains how
+ * the output will diff when Spark reruns the tasks for the RDD. There are 3 deterministic levels:
+ * 1. DETERMINATE: The RDD output is always the same data set in the same order after a rerun.
+ * 2. UNORDERED: The RDD output is always the same data set but the order can be different
+ * after a rerun.
+ * 3. INDETERMINATE. The RDD output can be different after a rerun.
+ *
+ * Note that, the output of an RDD usually relies on the parent RDDs. When the parent RDD's output
+ * is INDETERMINATE, it's very likely the RDD's output is also INDETERMINATE.
+ */
+private[spark] object DeterministicLevel extends Enumeration {
+ val DETERMINATE, UNORDERED, INDETERMINATE = Value
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala b/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala
new file mode 100644
index 0000000000000..42802f7113a19
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.rdd
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.TaskContext
+import org.apache.spark.annotation.{Experimental, Since}
+
+/**
+ * :: Experimental ::
+ * Wraps an RDD in a barrier stage, which forces Spark to launch tasks of this stage together.
+ * [[org.apache.spark.rdd.RDDBarrier]] instances are created by
+ * [[org.apache.spark.rdd.RDD#barrier]].
+ */
+@Experimental
+@Since("2.4.0")
+class RDDBarrier[T: ClassTag] private[spark] (rdd: RDD[T]) {
+
+ /**
+ * :: Experimental ::
+ * Returns a new RDD by applying a function to each partition of the wrapped RDD,
+ * where tasks are launched together in a barrier stage.
+ * The interface is the same as [[org.apache.spark.rdd.RDD#mapPartitions]].
+ * Please see the API doc there.
+ * @see [[org.apache.spark.BarrierTaskContext]]
+ */
+ @Experimental
+ @Since("2.4.0")
+ def mapPartitions[S: ClassTag](
+ f: Iterator[T] => Iterator[S],
+ preservesPartitioning: Boolean = false): RDD[S] = rdd.withScope {
+ val cleanedF = rdd.sparkContext.clean(f)
+ new MapPartitionsRDD(
+ rdd,
+ (context: TaskContext, index: Int, iter: Iterator[T]) => cleanedF(iter),
+ preservesPartitioning,
+ isFromBarrier = true
+ )
+ }
+
+ // TODO: [SPARK-25247] add extra conf to RDDBarrier, e.g., timeout.
+}
diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
index 979152b55f957..8273d8a9eb476 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala
@@ -300,7 +300,7 @@ private[spark] object ReliableCheckpointRDD extends Logging {
val deserializeStream = serializer.deserializeStream(fileInputStream)
// Register an on-task-completion callback to close the input stream.
- context.addTaskCompletionListener(context => deserializeStream.close())
+ context.addTaskCompletionListener[Unit](context => deserializeStream.close())
deserializeStream.asIterator.asInstanceOf[Iterator[T]]
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
index 26eaa9aa3d03f..e8f9b27b7eb55 100644
--- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala
@@ -110,4 +110,6 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag](
super.clearDependencies()
prev = null
}
+
+ private[spark] override def isBarrier(): Boolean = false
}
diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
index a2936d6ad539c..47576959322d1 100644
--- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
+++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala
@@ -50,7 +50,7 @@ private[netty] class NettyRpcEnv(
private[netty] val transportConf = SparkTransportConf.fromSparkConf(
conf.clone.set("spark.rpc.io.numConnectionsPerPeer", "1"),
"rpc",
- conf.getInt("spark.rpc.io.threads", 0))
+ conf.getInt("spark.rpc.io.threads", numUsableCores))
private val dispatcher: Dispatcher = new Dispatcher(this, numUsableCores)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
index 949e88f606275..6e4d062749d5f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ActiveJob.scala
@@ -60,4 +60,10 @@ private[spark] class ActiveJob(
val finished = Array.fill[Boolean](numPartitions)(false)
var numFinished = 0
+
+ /** Resets the status of all partitions in this stage so they are marked as not finished. */
+ def resetAllPartitions(): Unit = {
+ (0 until numPartitions).foreach(finished.update(_, false))
+ numFinished = 0
+ }
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala
new file mode 100644
index 0000000000000..803a0a1226d6c
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala
@@ -0,0 +1,62 @@
+/*
+ * 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
+
+import org.apache.spark.SparkException
+
+/**
+ * Exception thrown when submit a job with barrier stage(s) failing a required check.
+ */
+private[spark] class BarrierJobAllocationFailed(message: String) extends SparkException(message)
+
+private[spark] class BarrierJobUnsupportedRDDChainException
+ extends BarrierJobAllocationFailed(
+ BarrierJobAllocationFailed.ERROR_MESSAGE_RUN_BARRIER_WITH_UNSUPPORTED_RDD_CHAIN_PATTERN)
+
+private[spark] class BarrierJobRunWithDynamicAllocationException
+ extends BarrierJobAllocationFailed(
+ BarrierJobAllocationFailed.ERROR_MESSAGE_RUN_BARRIER_WITH_DYN_ALLOCATION)
+
+private[spark] class BarrierJobSlotsNumberCheckFailed
+ extends BarrierJobAllocationFailed(
+ BarrierJobAllocationFailed.ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER)
+
+private[spark] object BarrierJobAllocationFailed {
+
+ // Error message when running a barrier stage that have unsupported RDD chain pattern.
+ val ERROR_MESSAGE_RUN_BARRIER_WITH_UNSUPPORTED_RDD_CHAIN_PATTERN =
+ "[SPARK-24820][SPARK-24821]: Barrier execution mode does not allow the following pattern of " +
+ "RDD chain within a barrier stage:\n1. Ancestor RDDs that have different number of " +
+ "partitions from the resulting RDD (eg. union()/coalesce()/first()/take()/" +
+ "PartitionPruningRDD). A workaround for first()/take() can be barrierRdd.collect().head " +
+ "(scala) or barrierRdd.collect()[0] (python).\n" +
+ "2. An RDD that depends on multiple barrier RDDs (eg. barrierRdd1.zip(barrierRdd2))."
+
+ // Error message when running a barrier stage with dynamic resource allocation enabled.
+ val ERROR_MESSAGE_RUN_BARRIER_WITH_DYN_ALLOCATION =
+ "[SPARK-24942]: Barrier execution mode does not support dynamic resource allocation for " +
+ "now. You can disable dynamic resource allocation by setting Spark conf " +
+ "\"spark.dynamicAllocation.enabled\" to \"false\"."
+
+ // Error message when running a barrier stage that requires more slots than current total number.
+ val ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER =
+ "[SPARK-24819]: Barrier execution mode does not allow run a barrier stage that requires " +
+ "more slots than the total number of slots in the cluster currently. Please init a new " +
+ "cluster with more CPU cores or repartition the input RDD(s) to reduce the number of " +
+ "slots required to run this barrier stage."
+}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index f74425d73b392..47108353583a8 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
@@ -19,8 +19,9 @@ package org.apache.spark.scheduler
import java.io.NotSerializableException
import java.util.Properties
-import java.util.concurrent.TimeUnit
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
import java.util.concurrent.atomic.AtomicInteger
+import java.util.function.BiFunction
import scala.annotation.tailrec
import scala.collection.Map
@@ -34,12 +35,12 @@ import org.apache.commons.lang3.SerializationUtils
import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult}
-import org.apache.spark.rdd.{RDD, RDDCheckpointData}
+import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.storage._
import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat
@@ -111,8 +112,7 @@ import org.apache.spark.util._
* - When adding a new data structure, update `DAGSchedulerSuite.assertDataStructuresEmpty` to
* include the new structure. This will help to catch memory leaks.
*/
-private[spark]
-class DAGScheduler(
+private[spark] class DAGScheduler(
private[scheduler] val sc: SparkContext,
private[scheduler] val taskScheduler: TaskScheduler,
listenerBus: LiveListenerBus,
@@ -203,6 +203,24 @@ class DAGScheduler(
sc.getConf.getInt("spark.stage.maxConsecutiveAttempts",
DAGScheduler.DEFAULT_MAX_CONSECUTIVE_STAGE_ATTEMPTS)
+ /**
+ * Number of max concurrent tasks check failures for each barrier job.
+ */
+ private[scheduler] val barrierJobIdToNumTasksCheckFailures = new ConcurrentHashMap[Int, Int]
+
+ /**
+ * Time in seconds to wait between a max concurrent tasks check failure and the next check.
+ */
+ private val timeIntervalNumTasksCheck = sc.getConf
+ .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL)
+
+ /**
+ * Max number of max concurrent tasks check failures allowed for a job before fail the job
+ * submission.
+ */
+ private val maxFailureNumTasksCheck = sc.getConf
+ .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES)
+
private val messageScheduler =
ThreadUtils.newDaemonSingleThreadScheduledExecutor("dag-scheduler-message")
@@ -246,8 +264,11 @@ class DAGScheduler(
execId: String,
// (taskId, stageId, stageAttemptId, accumUpdates)
accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])],
- blockManagerId: BlockManagerId): Boolean = {
- listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates))
+ blockManagerId: BlockManagerId,
+ // executor metrics indexed by MetricGetter.values
+ executorUpdates: ExecutorMetrics): Boolean = {
+ listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates,
+ Some(executorUpdates)))
blockManagerMaster.driverEndpoint.askSync[Boolean](
BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat"))
}
@@ -340,6 +361,21 @@ class DAGScheduler(
}
}
+ /**
+ * Check to make sure we don't launch a barrier stage with unsupported RDD chain pattern. The
+ * following patterns are not supported:
+ * 1. Ancestor RDDs that have different number of partitions from the resulting RDD (eg.
+ * union()/coalesce()/first()/take()/PartitionPruningRDD);
+ * 2. An RDD that depends on multiple barrier RDDs (eg. barrierRdd1.zip(barrierRdd2)).
+ */
+ private def checkBarrierStageWithRDDChainPattern(rdd: RDD[_], numTasksInStage: Int): Unit = {
+ val predicate: RDD[_] => Boolean = (r =>
+ r.getNumPartitions == numTasksInStage && r.dependencies.filter(_.rdd.isBarrier()).size <= 1)
+ if (rdd.isBarrier() && !traverseParentRDDsWithinStage(rdd, predicate)) {
+ throw new BarrierJobUnsupportedRDDChainException
+ }
+ }
+
/**
* Creates a ShuffleMapStage that generates the given shuffle dependency's partitions. If a
* previously run stage generated the same shuffle data, this function will copy the output
@@ -348,6 +384,9 @@ class DAGScheduler(
*/
def createShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): ShuffleMapStage = {
val rdd = shuffleDep.rdd
+ checkBarrierStageWithDynamicAllocation(rdd)
+ checkBarrierStageWithNumSlots(rdd)
+ checkBarrierStageWithRDDChainPattern(rdd, rdd.getNumPartitions)
val numTasks = rdd.partitions.length
val parents = getOrCreateParentStages(rdd, jobId)
val id = nextStageId.getAndIncrement()
@@ -367,6 +406,36 @@ class DAGScheduler(
stage
}
+ /**
+ * We don't support run a barrier stage with dynamic resource allocation enabled, it shall lead
+ * to some confusing behaviors (eg. with dynamic resource allocation enabled, it may happen that
+ * we acquire some executors (but not enough to launch all the tasks in a barrier stage) and
+ * later release them due to executor idle time expire, and then acquire again).
+ *
+ * We perform the check on job submit and fail fast if running a barrier stage with dynamic
+ * resource allocation enabled.
+ *
+ * TODO SPARK-24942 Improve cluster resource management with jobs containing barrier stage
+ */
+ private def checkBarrierStageWithDynamicAllocation(rdd: RDD[_]): Unit = {
+ if (rdd.isBarrier() && Utils.isDynamicAllocationEnabled(sc.getConf)) {
+ throw new BarrierJobRunWithDynamicAllocationException
+ }
+ }
+
+ /**
+ * Check whether the barrier stage requires more slots (to be able to launch all tasks in the
+ * barrier stage together) than the total number of active slots currently. Fail current check
+ * if trying to submit a barrier stage that requires more slots than current total number. If
+ * the check fails consecutively beyond a configured number for a job, then fail current job
+ * submission.
+ */
+ private def checkBarrierStageWithNumSlots(rdd: RDD[_]): Unit = {
+ if (rdd.isBarrier() && rdd.getNumPartitions > sc.maxNumConcurrentTasks) {
+ throw new BarrierJobSlotsNumberCheckFailed
+ }
+ }
+
/**
* Create a ResultStage associated with the provided jobId.
*/
@@ -376,6 +445,9 @@ class DAGScheduler(
partitions: Array[Int],
jobId: Int,
callSite: CallSite): ResultStage = {
+ checkBarrierStageWithDynamicAllocation(rdd)
+ checkBarrierStageWithNumSlots(rdd)
+ checkBarrierStageWithRDDChainPattern(rdd, partitions.toSet.size)
val parents = getOrCreateParentStages(rdd, jobId)
val id = nextStageId.getAndIncrement()
val stage = new ResultStage(id, rdd, func, partitions, parents, jobId, callSite)
@@ -451,6 +523,32 @@ class DAGScheduler(
parents
}
+ /**
+ * Traverses the given RDD and its ancestors within the same stage and checks whether all of the
+ * RDDs satisfy a given predicate.
+ */
+ private def traverseParentRDDsWithinStage(rdd: RDD[_], predicate: RDD[_] => Boolean): Boolean = {
+ val visited = new HashSet[RDD[_]]
+ val waitingForVisit = new ArrayStack[RDD[_]]
+ waitingForVisit.push(rdd)
+ while (waitingForVisit.nonEmpty) {
+ val toVisit = waitingForVisit.pop()
+ if (!visited(toVisit)) {
+ if (!predicate(toVisit)) {
+ return false
+ }
+ visited += toVisit
+ toVisit.dependencies.foreach {
+ case _: ShuffleDependency[_, _, _] =>
+ // Not within the same stage with current rdd, do nothing.
+ case dependency =>
+ waitingForVisit.push(dependency.rdd)
+ }
+ }
+ }
+ true
+ }
+
private def getMissingParentStages(stage: Stage): List[Stage] = {
val missing = new HashSet[Stage]
val visited = new HashSet[RDD[_]]
@@ -866,11 +964,38 @@ class DAGScheduler(
// HadoopRDD whose underlying HDFS files have been deleted.
finalStage = createResultStage(finalRDD, func, partitions, jobId, callSite)
} catch {
+ case e: BarrierJobSlotsNumberCheckFailed =>
+ logWarning(s"The job $jobId requires to run a barrier stage that requires more slots " +
+ "than the total number of slots in the cluster currently.")
+ // If jobId doesn't exist in the map, Scala coverts its value null to 0: Int automatically.
+ val numCheckFailures = barrierJobIdToNumTasksCheckFailures.compute(jobId,
+ new BiFunction[Int, Int, Int] {
+ override def apply(key: Int, value: Int): Int = value + 1
+ })
+ if (numCheckFailures <= maxFailureNumTasksCheck) {
+ messageScheduler.schedule(
+ new Runnable {
+ override def run(): Unit = eventProcessLoop.post(JobSubmitted(jobId, finalRDD, func,
+ partitions, callSite, listener, properties))
+ },
+ timeIntervalNumTasksCheck,
+ TimeUnit.SECONDS
+ )
+ return
+ } else {
+ // Job failed, clear internal data.
+ barrierJobIdToNumTasksCheckFailures.remove(jobId)
+ listener.jobFailed(e)
+ return
+ }
+
case e: Exception =>
logWarning("Creating new stage failed due to exception - job: " + jobId, e)
listener.jobFailed(e)
return
}
+ // Job submitted, clear internal data.
+ barrierJobIdToNumTasksCheckFailures.remove(jobId)
val job = new ActiveJob(jobId, finalStage, callSite, listener, properties)
clearCacheLocs()
@@ -1062,7 +1187,7 @@ class DAGScheduler(
stage.pendingPartitions += id
new ShuffleMapTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, properties, serializedTaskMetrics, Option(jobId),
- Option(sc.applicationId), sc.applicationAttemptId)
+ Option(sc.applicationId), sc.applicationAttemptId, stage.rdd.isBarrier())
}
case stage: ResultStage =>
@@ -1072,7 +1197,8 @@ class DAGScheduler(
val locs = taskIdToLocations(id)
new ResultTask(stage.id, stage.latestInfo.attemptNumber,
taskBinary, part, locs, id, properties, serializedTaskMetrics,
- Option(jobId), Option(sc.applicationId), sc.applicationAttemptId)
+ Option(jobId), Option(sc.applicationId), sc.applicationAttemptId,
+ stage.rdd.isBarrier())
}
}
} catch {
@@ -1250,18 +1376,10 @@ class DAGScheduler(
case smt: ShuffleMapTask =>
val shuffleStage = stage.asInstanceOf[ShuffleMapStage]
+ shuffleStage.pendingPartitions -= task.partitionId
val status = event.result.asInstanceOf[MapStatus]
val execId = status.location.executorId
logDebug("ShuffleMapTask finished on " + execId)
- if (stageIdToStage(task.stageId).latestInfo.attemptNumber == task.stageAttemptId) {
- // This task was for the currently running attempt of the stage. Since the task
- // completed successfully from the perspective of the TaskSetManager, mark it as
- // no longer pending (the TaskSetManager may consider the task complete even
- // when the output needs to be ignored because the task's epoch is too small below.
- // In this case, when pending partitions is empty, there will still be missing
- // output locations, which will cause the DAGScheduler to resubmit the stage below.)
- shuffleStage.pendingPartitions -= task.partitionId
- }
if (failedEpoch.contains(execId) && smt.epoch <= failedEpoch(execId)) {
logInfo(s"Ignoring possibly bogus $smt completion from executor $execId")
} else {
@@ -1270,13 +1388,6 @@ class DAGScheduler(
// available.
mapOutputTracker.registerMapOutput(
shuffleStage.shuffleDep.shuffleId, smt.partitionId, status)
- // Remove the task's partition from pending partitions. This may have already been
- // done above, but will not have been done yet in cases where the task attempt was
- // from an earlier attempt of the stage (i.e., not the attempt that's currently
- // running). This allows the DAGScheduler to mark the stage as complete when one
- // copy of each task has finished successfully, even if the currently active stage
- // still has tasks running.
- shuffleStage.pendingPartitions -= task.partitionId
}
if (runningStages.contains(shuffleStage) && shuffleStage.pendingPartitions.isEmpty) {
@@ -1311,17 +1422,6 @@ class DAGScheduler(
}
}
- case Resubmitted =>
- logInfo("Resubmitted " + task + ", so marking it as still running")
- stage match {
- case sms: ShuffleMapStage =>
- sms.pendingPartitions += task.partitionId
-
- case _ =>
- assert(false, "TaskSetManagers should only send Resubmitted task statuses for " +
- "tasks in ShuffleMapStages.")
- }
-
case FetchFailed(bmAddress, shuffleId, mapId, _, failureMessage) =>
val failedStage = stageIdToStage(task.stageId)
val mapStage = shuffleIdToMapStage(shuffleId)
@@ -1331,9 +1431,9 @@ class DAGScheduler(
s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
} else {
- failedStage.fetchFailedAttemptIds.add(task.stageAttemptId)
+ failedStage.failedAttemptIds.add(task.stageAttemptId)
val shouldAbortStage =
- failedStage.fetchFailedAttemptIds.size >= maxConsecutiveStageAttempts ||
+ failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts ||
disallowStageRetryForTest
// It is likely that we receive multiple FetchFailed for a single stage (because we have
@@ -1349,6 +1449,31 @@ class DAGScheduler(
s"longer running")
}
+ if (mapStage.rdd.isBarrier()) {
+ // Mark all the map as broken in the map stage, to ensure retry all the tasks on
+ // resubmitted stage attempt.
+ mapOutputTracker.unregisterAllMapOutput(shuffleId)
+ } else if (mapId != -1) {
+ // Mark the map whose fetch failed as broken in the map stage
+ mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress)
+ }
+
+ if (failedStage.rdd.isBarrier()) {
+ failedStage match {
+ case failedMapStage: ShuffleMapStage =>
+ // Mark all the map as broken in the map stage, to ensure retry all the tasks on
+ // resubmitted stage attempt.
+ mapOutputTracker.unregisterAllMapOutput(failedMapStage.shuffleDep.shuffleId)
+
+ case failedResultStage: ResultStage =>
+ // Abort the failed result stage since we may have committed output for some
+ // partitions.
+ val reason = "Could not recover from a failed barrier ResultStage. Most recent " +
+ s"failure reason: $failureMessage"
+ abortStage(failedResultStage, reason, None)
+ }
+ }
+
if (shouldAbortStage) {
val abortMessage = if (disallowStageRetryForTest) {
"Fetch failure will not retry stage due to testing config"
@@ -1365,6 +1490,63 @@ class DAGScheduler(
failedStages += failedStage
failedStages += mapStage
if (noResubmitEnqueued) {
+ // If the map stage is INDETERMINATE, which means the map tasks may return
+ // different result when re-try, we need to re-try all the tasks of the failed
+ // stage and its succeeding stages, because the input data will be changed after the
+ // map tasks are re-tried.
+ // Note that, if map stage is UNORDERED, we are fine. The shuffle partitioner is
+ // guaranteed to be determinate, so the input data of the reducers will not change
+ // even if the map tasks are re-tried.
+ if (mapStage.rdd.outputDeterministicLevel == DeterministicLevel.INDETERMINATE) {
+ // It's a little tricky to find all the succeeding stages of `failedStage`, because
+ // each stage only know its parents not children. Here we traverse the stages from
+ // the leaf nodes (the result stages of active jobs), and rollback all the stages
+ // in the stage chains that connect to the `failedStage`. To speed up the stage
+ // traversing, we collect the stages to rollback first. If a stage needs to
+ // rollback, all its succeeding stages need to rollback to.
+ val stagesToRollback = scala.collection.mutable.HashSet(failedStage)
+
+ def collectStagesToRollback(stageChain: List[Stage]): Unit = {
+ if (stagesToRollback.contains(stageChain.head)) {
+ stageChain.drop(1).foreach(s => stagesToRollback += s)
+ } else {
+ stageChain.head.parents.foreach { s =>
+ collectStagesToRollback(s :: stageChain)
+ }
+ }
+ }
+
+ def generateErrorMessage(stage: Stage): String = {
+ "A shuffle map stage with indeterminate output was failed and retried. " +
+ s"However, Spark cannot rollback the $stage to re-process the input data, " +
+ "and has to fail this job. Please eliminate the indeterminacy by " +
+ "checkpointing the RDD before repartition and try again."
+ }
+
+ activeJobs.foreach(job => collectStagesToRollback(job.finalStage :: Nil))
+
+ stagesToRollback.foreach {
+ case mapStage: ShuffleMapStage =>
+ val numMissingPartitions = mapStage.findMissingPartitions().length
+ if (numMissingPartitions < mapStage.numTasks) {
+ // TODO: support to rollback shuffle files.
+ // Currently the shuffle writing is "first write wins", so we can't re-run a
+ // shuffle map stage and overwrite existing shuffle files. We have to finish
+ // SPARK-8029 first.
+ abortStage(mapStage, generateErrorMessage(mapStage), None)
+ }
+
+ case resultStage: ResultStage if resultStage.activeJob.isDefined =>
+ val numMissingPartitions = resultStage.findMissingPartitions().length
+ if (numMissingPartitions < resultStage.numTasks) {
+ // TODO: support to rollback result tasks.
+ abortStage(resultStage, generateErrorMessage(resultStage), None)
+ }
+
+ case _ =>
+ }
+ }
+
// We expect one executor failure to trigger many FetchFailures in rapid succession,
// but all of those task failures can typically be handled by a single resubmission of
// the failed stage. We avoid flooding the scheduler's event queue with resubmit
@@ -1375,7 +1557,7 @@ class DAGScheduler(
// simpler while not producing an overwhelming number of scheduler events.
logInfo(
s"Resubmitting $mapStage (${mapStage.name}) and " +
- s"$failedStage (${failedStage.name}) due to fetch failure"
+ s"$failedStage (${failedStage.name}) due to fetch failure"
)
messageScheduler.schedule(
new Runnable {
@@ -1386,10 +1568,6 @@ class DAGScheduler(
)
}
}
- // Mark the map whose fetch failed as broken in the map stage
- if (mapId != -1) {
- mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress)
- }
// TODO: mark the executor as failed only if there were lots of fetch failures on it
if (bmAddress != null) {
@@ -1411,6 +1589,91 @@ class DAGScheduler(
}
}
+ case failure: TaskFailedReason if task.isBarrier =>
+ // Also handle the task failed reasons here.
+ failure match {
+ case Resubmitted =>
+ handleResubmittedFailure(task, stage)
+
+ case _ => // Do nothing.
+ }
+
+ // Always fail the current stage and retry all the tasks when a barrier task fail.
+ val failedStage = stageIdToStage(task.stageId)
+ if (failedStage.latestInfo.attemptNumber != task.stageAttemptId) {
+ logInfo(s"Ignoring task failure from $task as it's from $failedStage attempt" +
+ s" ${task.stageAttemptId} and there is a more recent attempt for that stage " +
+ s"(attempt ${failedStage.latestInfo.attemptNumber}) running")
+ } else {
+ logInfo(s"Marking $failedStage (${failedStage.name}) as failed due to a barrier task " +
+ "failed.")
+ val message = s"Stage failed because barrier task $task finished unsuccessfully.\n" +
+ failure.toErrorString
+ try {
+ // killAllTaskAttempts will fail if a SchedulerBackend does not implement killTask.
+ val reason = s"Task $task from barrier stage $failedStage (${failedStage.name}) " +
+ "failed."
+ taskScheduler.killAllTaskAttempts(stageId, interruptThread = false, reason)
+ } catch {
+ case e: UnsupportedOperationException =>
+ // Cannot continue with barrier stage if failed to cancel zombie barrier tasks.
+ // TODO SPARK-24877 leave the zombie tasks and ignore their completion events.
+ logWarning(s"Could not kill all tasks for stage $stageId", e)
+ abortStage(failedStage, "Could not kill zombie barrier tasks for stage " +
+ s"$failedStage (${failedStage.name})", Some(e))
+ }
+ markStageAsFinished(failedStage, Some(message))
+
+ failedStage.failedAttemptIds.add(task.stageAttemptId)
+ // TODO Refactor the failure handling logic to combine similar code with that of
+ // FetchFailed.
+ val shouldAbortStage =
+ failedStage.failedAttemptIds.size >= maxConsecutiveStageAttempts ||
+ disallowStageRetryForTest
+
+ if (shouldAbortStage) {
+ val abortMessage = if (disallowStageRetryForTest) {
+ "Barrier stage will not retry stage due to testing config. Most recent failure " +
+ s"reason: $message"
+ } else {
+ s"""$failedStage (${failedStage.name})
+ |has failed the maximum allowable number of
+ |times: $maxConsecutiveStageAttempts.
+ |Most recent failure reason: $message
+ """.stripMargin.replaceAll("\n", " ")
+ }
+ abortStage(failedStage, abortMessage, None)
+ } else {
+ failedStage match {
+ case failedMapStage: ShuffleMapStage =>
+ // Mark all the map as broken in the map stage, to ensure retry all the tasks on
+ // resubmitted stage attempt.
+ mapOutputTracker.unregisterAllMapOutput(failedMapStage.shuffleDep.shuffleId)
+
+ case failedResultStage: ResultStage =>
+ // Abort the failed result stage since we may have committed output for some
+ // partitions.
+ val reason = "Could not recover from a failed barrier ResultStage. Most recent " +
+ s"failure reason: $message"
+ abortStage(failedResultStage, reason, None)
+ }
+ // In case multiple task failures triggered for a single stage attempt, ensure we only
+ // resubmit the failed stage once.
+ val noResubmitEnqueued = !failedStages.contains(failedStage)
+ failedStages += failedStage
+ if (noResubmitEnqueued) {
+ logInfo(s"Resubmitting $failedStage (${failedStage.name}) due to barrier stage " +
+ "failure.")
+ messageScheduler.schedule(new Runnable {
+ override def run(): Unit = eventProcessLoop.post(ResubmitFailedStages)
+ }, DAGScheduler.RESUBMIT_TIMEOUT, TimeUnit.MILLISECONDS)
+ }
+ }
+ }
+
+ case Resubmitted =>
+ handleResubmittedFailure(task, stage)
+
case _: TaskCommitDenied =>
// Do nothing here, left up to the TaskScheduler to decide how to handle denied commits
@@ -1426,6 +1689,18 @@ class DAGScheduler(
}
}
+ private def handleResubmittedFailure(task: Task[_], stage: Stage): Unit = {
+ logInfo(s"Resubmitted $task, so marking it as still running.")
+ stage match {
+ case sms: ShuffleMapStage =>
+ sms.pendingPartitions += task.partitionId
+
+ case _ =>
+ throw new SparkException("TaskSetManagers should only send Resubmitted task " +
+ "statuses for tasks in ShuffleMapStages.")
+ }
+ }
+
private[scheduler] def markMapStageJobsAsFinished(shuffleStage: ShuffleMapStage): Unit = {
// Mark any map-stage jobs waiting on this stage as finished
if (shuffleStage.isAvailable && shuffleStage.mapStageJobs.nonEmpty) {
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 69bc51c1ecf90..1629e1797977f 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -23,8 +23,7 @@ import java.nio.charset.StandardCharsets
import java.util.EnumSet
import java.util.Locale
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.mutable.{ArrayBuffer, Map}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
@@ -36,6 +35,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark.{SPARK_VERSION, SparkConf}
import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.io.CompressionCodec
@@ -51,6 +51,7 @@ import org.apache.spark.util.{JsonProtocol, Utils}
* spark.eventLog.overwrite - Whether to overwrite any existing files.
* spark.eventLog.dir - Path to the directory in which events are logged.
* spark.eventLog.buffer.kb - Buffer size to use when writing to output streams
+ * spark.eventLog.logStageExecutorMetrics.enabled - Whether to log stage executor metrics
*/
private[spark] class EventLoggingListener(
appId: String,
@@ -69,6 +70,7 @@ private[spark] class EventLoggingListener(
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 shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS)
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)
@@ -93,6 +95,9 @@ private[spark] class EventLoggingListener(
// Visible for tests only.
private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
+ // map of (stageId, stageAttempt), to peak executor metrics for the stage
+ private val liveStageExecutorMetrics = Map.empty[(Int, Int), Map[String, ExecutorMetrics]]
+
/**
* Creates the log file in the configured log directory.
*/
@@ -155,7 +160,14 @@ private[spark] class EventLoggingListener(
}
// Events that do not trigger a flush
- override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event)
+ override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = {
+ logEvent(event)
+ if (shouldLogStageExecutorMetrics) {
+ // record the peak metrics for the new stage
+ liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()),
+ Map.empty[String, ExecutorMetrics])
+ }
+ }
override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event)
@@ -169,6 +181,26 @@ private[spark] class EventLoggingListener(
// Events that trigger a flush
override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
+ if (shouldLogStageExecutorMetrics) {
+ // clear out any previous attempts, that did not have a stage completed event
+ val prevAttemptId = event.stageInfo.attemptNumber() - 1
+ for (attemptId <- 0 to prevAttemptId) {
+ liveStageExecutorMetrics.remove((event.stageInfo.stageId, attemptId))
+ }
+
+ // log the peak executor metrics for the stage, for each live executor,
+ // whether or not the executor is running tasks for the stage
+ val executorOpt = liveStageExecutorMetrics.remove(
+ (event.stageInfo.stageId, event.stageInfo.attemptNumber()))
+ executorOpt.foreach { execMap =>
+ execMap.foreach { case (executorId, peakExecutorMetrics) =>
+ logEvent(new SparkListenerStageExecutorMetrics(executorId, event.stageInfo.stageId,
+ event.stageInfo.attemptNumber(), peakExecutorMetrics))
+ }
+ }
+ }
+
+ // log stage completed event
logEvent(event, flushLogger = true)
}
@@ -234,8 +266,18 @@ private[spark] class EventLoggingListener(
}
}
- // No-op because logging every update would be overkill
- override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { }
+ override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = {
+ if (shouldLogStageExecutorMetrics) {
+ // For the active stages, record any new peak values for the memory metrics for the executor
+ event.executorUpdates.foreach { executorUpdates =>
+ liveStageExecutorMetrics.values.foreach { peakExecutorMetrics =>
+ val peakMetrics = peakExecutorMetrics.getOrElseUpdate(
+ event.execId, new ExecutorMetrics())
+ peakMetrics.compareAndUpdatePeakValues(executorUpdates)
+ }
+ }
+ }
+ }
override def onOtherEvent(event: SparkListenerEvent): Unit = {
if (event.logEvent) {
@@ -296,7 +338,7 @@ private[spark] object EventLoggingListener extends Logging {
private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
// A cache for compression codecs to avoid creating the same codec many times
- private val codecMap = new mutable.HashMap[String, CompressionCodec]
+ private val codecMap = Map.empty[String, CompressionCodec]
/**
* Write metadata about an event log to the given stream.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
index 659694dd189ad..7e1d75fe723d6 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala
@@ -31,7 +31,8 @@ import org.apache.spark.util.Utils
/**
* Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the
- * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks.
+ * task ran on, the sizes of outputs for each reducer, and the number of outputs of the map task,
+ * for passing on to the reduce tasks.
*/
private[spark] sealed trait MapStatus {
/** Location where this task was run. */
@@ -44,18 +45,23 @@ private[spark] sealed trait MapStatus {
* necessary for correctness, since block fetchers are allowed to skip zero-size blocks.
*/
def getSizeForBlock(reduceId: Int): Long
+
+ /**
+ * The number of outputs for the map task.
+ */
+ def numberOfOutput: Long
}
private[spark] object MapStatus {
- def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = {
+ def apply(loc: BlockManagerId, uncompressedSizes: Array[Long], numOutput: Long): MapStatus = {
if (uncompressedSizes.length > Option(SparkEnv.get)
.map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS))
.getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get)) {
- HighlyCompressedMapStatus(loc, uncompressedSizes)
+ HighlyCompressedMapStatus(loc, uncompressedSizes, numOutput)
} else {
- new CompressedMapStatus(loc, uncompressedSizes)
+ new CompressedMapStatus(loc, uncompressedSizes, numOutput)
}
}
@@ -98,29 +104,34 @@ private[spark] object MapStatus {
*/
private[spark] class CompressedMapStatus(
private[this] var loc: BlockManagerId,
- private[this] var compressedSizes: Array[Byte])
+ private[this] var compressedSizes: Array[Byte],
+ private[this] var numOutput: Long)
extends MapStatus with Externalizable {
- protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only
+ protected def this() = this(null, null.asInstanceOf[Array[Byte]], -1) // For deserialization only
- def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) {
- this(loc, uncompressedSizes.map(MapStatus.compressSize))
+ def this(loc: BlockManagerId, uncompressedSizes: Array[Long], numOutput: Long) {
+ this(loc, uncompressedSizes.map(MapStatus.compressSize), numOutput)
}
override def location: BlockManagerId = loc
+ override def numberOfOutput: Long = numOutput
+
override def getSizeForBlock(reduceId: Int): Long = {
MapStatus.decompressSize(compressedSizes(reduceId))
}
override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
loc.writeExternal(out)
+ out.writeLong(numOutput)
out.writeInt(compressedSizes.length)
out.write(compressedSizes)
}
override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
loc = BlockManagerId(in)
+ numOutput = in.readLong()
val len = in.readInt()
compressedSizes = new Array[Byte](len)
in.readFully(compressedSizes)
@@ -143,17 +154,20 @@ private[spark] class HighlyCompressedMapStatus private (
private[this] var numNonEmptyBlocks: Int,
private[this] var emptyBlocks: RoaringBitmap,
private[this] var avgSize: Long,
- private var hugeBlockSizes: Map[Int, Byte])
+ private var hugeBlockSizes: Map[Int, Byte],
+ private[this] var numOutput: Long)
extends MapStatus with Externalizable {
// loc could be null when the default constructor is called during deserialization
require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0,
"Average size can only be zero for map stages that produced no output")
- protected def this() = this(null, -1, null, -1, null) // For deserialization only
+ protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only
override def location: BlockManagerId = loc
+ override def numberOfOutput: Long = numOutput
+
override def getSizeForBlock(reduceId: Int): Long = {
assert(hugeBlockSizes != null)
if (emptyBlocks.contains(reduceId)) {
@@ -168,6 +182,7 @@ private[spark] class HighlyCompressedMapStatus private (
override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException {
loc.writeExternal(out)
+ out.writeLong(numOutput)
emptyBlocks.writeExternal(out)
out.writeLong(avgSize)
out.writeInt(hugeBlockSizes.size)
@@ -179,6 +194,7 @@ private[spark] class HighlyCompressedMapStatus private (
override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException {
loc = BlockManagerId(in)
+ numOutput = in.readLong()
emptyBlocks = new RoaringBitmap()
emptyBlocks.readExternal(in)
avgSize = in.readLong()
@@ -194,7 +210,10 @@ private[spark] class HighlyCompressedMapStatus private (
}
private[spark] object HighlyCompressedMapStatus {
- def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = {
+ def apply(
+ loc: BlockManagerId,
+ uncompressedSizes: Array[Long],
+ numOutput: Long): HighlyCompressedMapStatus = {
// We must keep track of which blocks are empty so that we don't report a zero-sized
// block as being non-empty (or vice-versa) when using the average block size.
var i = 0
@@ -235,6 +254,6 @@ private[spark] object HighlyCompressedMapStatus {
emptyBlocks.trim()
emptyBlocks.runOptimize()
new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize,
- hugeBlockSizesArray.toMap)
+ hugeBlockSizesArray.toMap, numOutput)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
index e36c759a42556..aafeae05b566c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala
@@ -48,7 +48,9 @@ import org.apache.spark.rdd.RDD
* @param jobId id of the job this task belongs to
* @param appId id of the app this task belongs to
* @param appAttemptId attempt id of the app this task belongs to
- */
+ * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks
+ * at the same time for a barrier stage.
+ */
private[spark] class ResultTask[T, U](
stageId: Int,
stageAttemptId: Int,
@@ -60,9 +62,10 @@ private[spark] class ResultTask[T, U](
serializedTaskMetrics: Array[Byte],
jobId: Option[Int] = None,
appId: Option[String] = None,
- appAttemptId: Option[String] = None)
+ appAttemptId: Option[String] = None,
+ isBarrier: Boolean = false)
extends Task[U](stageId, stageAttemptId, partition.index, localProperties, serializedTaskMetrics,
- jobId, appId, appAttemptId)
+ jobId, appId, appAttemptId, isBarrier)
with Serializable {
@transient private[this] val preferredLocs: Seq[TaskLocation] = {
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
index 22db3350abfa7..c187ee146301b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala
@@ -69,4 +69,13 @@ private[spark] trait SchedulerBackend {
*/
def getDriverLogUrls: Option[Map[String, String]] = None
+ /**
+ * Get the max number of tasks that can be concurrent launched currently.
+ * Note that please don't cache the value returned by this method, because the number can change
+ * due to add/remove executors.
+ *
+ * @return The max number of tasks that can be concurrent launched currently.
+ */
+ def maxNumConcurrentTasks(): Int
+
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
index 7a25c47e2cab3..f2cd65fd523ab 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala
@@ -49,6 +49,8 @@ import org.apache.spark.shuffle.ShuffleWriter
* @param jobId id of the job this task belongs to
* @param appId id of the app this task belongs to
* @param appAttemptId attempt id of the app this task belongs to
+ * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks
+ * at the same time for a barrier stage.
*/
private[spark] class ShuffleMapTask(
stageId: Int,
@@ -60,9 +62,10 @@ private[spark] class ShuffleMapTask(
serializedTaskMetrics: Array[Byte],
jobId: Option[Int] = None,
appId: Option[String] = None,
- appAttemptId: Option[String] = None)
+ appAttemptId: Option[String] = None,
+ isBarrier: Boolean = false)
extends Task[MapStatus](stageId, stageAttemptId, partition.index, localProperties,
- serializedTaskMetrics, jobId, appId, appAttemptId)
+ serializedTaskMetrics, jobId, appId, appAttemptId, isBarrier)
with Logging {
/** A constructor used only in test suites. This does not require passing in an RDD. */
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 8a112f6a37b96..293e8369677f0 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -26,7 +26,7 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo
import org.apache.spark.{SparkConf, TaskEndReason}
import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
import org.apache.spark.scheduler.cluster.ExecutorInfo
import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo}
import org.apache.spark.ui.SparkUI
@@ -160,11 +160,29 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends
* Periodic updates from executors.
* @param execId executor id
* @param accumUpdates sequence of (taskId, stageId, stageAttemptId, accumUpdates)
+ * @param executorUpdates executor level metrics updates
*/
@DeveloperApi
case class SparkListenerExecutorMetricsUpdate(
execId: String,
- accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])])
+ accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])],
+ executorUpdates: Option[ExecutorMetrics] = None)
+ extends SparkListenerEvent
+
+/**
+ * Peak metric values for the executor for the stage, written to the history log at stage
+ * completion.
+ * @param execId executor id
+ * @param stageId stage id
+ * @param stageAttemptId stage attempt
+ * @param executorMetrics executor level metrics, indexed by MetricGetter.values
+ */
+@DeveloperApi
+case class SparkListenerStageExecutorMetrics(
+ execId: String,
+ stageId: Int,
+ stageAttemptId: Int,
+ executorMetrics: ExecutorMetrics)
extends SparkListenerEvent
@DeveloperApi
@@ -264,6 +282,13 @@ private[spark] trait SparkListenerInterface {
*/
def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit
+ /**
+ * Called with the peak memory metrics for a given (executor, stage) combination. Note that this
+ * is only present when reading from the event log (as in the history server), and is never
+ * called in a live application.
+ */
+ def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit
+
/**
* Called when the driver registers a new executor.
*/
@@ -361,6 +386,9 @@ abstract class SparkListener extends SparkListenerInterface {
override def onExecutorMetricsUpdate(
executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { }
+ override def onStageExecutorMetrics(
+ executorMetrics: SparkListenerStageExecutorMetrics): Unit = { }
+
override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { }
override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { }
diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
index ff19cc65552e0..8f6b7ad309602 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala
@@ -57,6 +57,8 @@ private[spark] trait SparkListenerBus
listener.onApplicationEnd(applicationEnd)
case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
listener.onExecutorMetricsUpdate(metricsUpdate)
+ case stageExecutorMetrics: SparkListenerStageExecutorMetrics =>
+ listener.onStageExecutorMetrics(stageExecutorMetrics)
case executorAdded: SparkListenerExecutorAdded =>
listener.onExecutorAdded(executorAdded)
case executorRemoved: SparkListenerExecutorRemoved =>
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
index 290fd073caf27..26cca334d3bd5 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala
@@ -82,15 +82,15 @@ private[scheduler] abstract class Stage(
private var _latestInfo: StageInfo = StageInfo.fromStage(this, nextAttemptId)
/**
- * Set of stage attempt IDs that have failed with a FetchFailure. We keep track of these
- * failures in order to avoid endless retries if a stage keeps failing with a FetchFailure.
+ * Set of stage attempt IDs that have failed. We keep track of these failures in order to avoid
+ * endless retries if a stage keeps failing.
* We keep track of each attempt ID that has failed to avoid recording duplicate failures if
* multiple tasks from the same stage attempt fail (SPARK-5945).
*/
- val fetchFailedAttemptIds = new HashSet[Int]
+ val failedAttemptIds = new HashSet[Int]
private[scheduler] def clearFailures() : Unit = {
- fetchFailedAttemptIds.clear()
+ failedAttemptIds.clear()
}
/** Creates a new attempt for this stage by creating a new StageInfo with a new attempt ID. */
diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
index f536fc2a5f0a1..eb059f12be6d3 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala
@@ -49,6 +49,8 @@ import org.apache.spark.util._
* @param jobId id of the job this task belongs to
* @param appId id of the app this task belongs to
* @param appAttemptId attempt id of the app this task belongs to
+ * @param isBarrier whether this task belongs to a barrier stage. Spark must launch all the tasks
+ * at the same time for a barrier stage.
*/
private[spark] abstract class Task[T](
val stageId: Int,
@@ -60,7 +62,8 @@ private[spark] abstract class Task[T](
SparkEnv.get.closureSerializer.newInstance().serialize(TaskMetrics.registered).array(),
val jobId: Option[Int] = None,
val appId: Option[String] = None,
- val appAttemptId: Option[String] = None) extends Serializable {
+ val appAttemptId: Option[String] = None,
+ val isBarrier: Boolean = false) extends Serializable {
@transient lazy val metrics: TaskMetrics =
SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(serializedTaskMetrics))
@@ -77,7 +80,9 @@ private[spark] abstract class Task[T](
attemptNumber: Int,
metricsSystem: MetricsSystem): T = {
SparkEnv.get.blockManager.registerTask(taskAttemptId)
- context = new TaskContextImpl(
+ // TODO SPARK-24874 Allow create BarrierTaskContext based on partitions, instead of whether
+ // the stage is barrier.
+ val taskContext = new TaskContextImpl(
stageId,
stageAttemptId, // stageAttemptId and stageAttemptNumber are semantically equal
partitionId,
@@ -87,6 +92,13 @@ private[spark] abstract class Task[T](
localProperties,
metricsSystem,
metrics)
+
+ context = if (isBarrier) {
+ new BarrierTaskContext(taskContext)
+ } else {
+ taskContext
+ }
+
TaskContext.setTaskContext(context)
taskThread = Thread.currentThread()
@@ -161,7 +173,7 @@ private[spark] abstract class Task[T](
var epoch: Long = -1
// Task context, to be initialized in run().
- @transient var context: TaskContextImpl = _
+ @transient var context: TaskContext = _
// The actual Thread on which the task is running, if any. Initialized in run().
@volatile @transient private var taskThread: Thread = _
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
index c98b87148e404..bb4a4442b9433 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala
@@ -50,6 +50,7 @@ private[spark] class TaskDescription(
val executorId: String,
val name: String,
val index: Int, // Index within this task's TaskSet
+ val partitionId: Int,
val addedFiles: Map[String, Long],
val addedJars: Map[String, Long],
val properties: Properties,
@@ -76,6 +77,7 @@ private[spark] object TaskDescription {
dataOut.writeUTF(taskDescription.executorId)
dataOut.writeUTF(taskDescription.name)
dataOut.writeInt(taskDescription.index)
+ dataOut.writeInt(taskDescription.partitionId)
// Write files.
serializeStringLongMap(taskDescription.addedFiles, dataOut)
@@ -117,6 +119,7 @@ private[spark] object TaskDescription {
val executorId = dataIn.readUTF()
val name = dataIn.readUTF()
val index = dataIn.readInt()
+ val partitionId = dataIn.readInt()
// Read files.
val taskFiles = deserializeStringLongMap(dataIn)
@@ -138,7 +141,7 @@ private[spark] object TaskDescription {
// Create a sub-buffer for the serialized task into its own buffer (to be deserialized later).
val serializedTask = byteBuffer.slice()
- new TaskDescription(taskId, attemptNumber, executorId, name, index, taskFiles, taskJars,
- properties, serializedTask)
+ new TaskDescription(taskId, attemptNumber, executorId, name, index, partitionId, taskFiles,
+ taskJars, properties, serializedTask)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
index 90644fea23ab1..94221eb0d5515 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
@@ -17,6 +17,7 @@
package org.apache.spark.scheduler
+import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.AccumulatorV2
@@ -51,16 +52,22 @@ private[spark] trait TaskScheduler {
// Submit a sequence of tasks to run.
def submitTasks(taskSet: TaskSet): Unit
- // Cancel a stage.
+ // Kill all the tasks in a stage and fail the stage and all the jobs that depend on the stage.
+ // Throw UnsupportedOperationException if the backend doesn't support kill tasks.
def cancelTasks(stageId: Int, interruptThread: Boolean): Unit
/**
* Kills a task attempt.
+ * Throw UnsupportedOperationException if the backend doesn't support kill a task.
*
* @return Whether the task was successfully killed.
*/
def killTaskAttempt(taskId: Long, interruptThread: Boolean, reason: String): Boolean
+ // Kill all the running task attempts in a stage.
+ // Throw UnsupportedOperationException if the backend doesn't support kill tasks.
+ def killAllTaskAttempts(stageId: Int, interruptThread: Boolean, reason: String): Unit
+
// Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called.
def setDAGScheduler(dagScheduler: DAGScheduler): Unit
@@ -68,14 +75,15 @@ private[spark] trait TaskScheduler {
def defaultParallelism(): Int
/**
- * Update metrics for in-progress tasks and let the master know that the BlockManager is still
- * alive. Return true if the driver knows about the given block manager. Otherwise, return false,
- * indicating that the block manager should re-register.
+ * Update metrics for in-progress tasks and executor metrics, and let the master know that the
+ * BlockManager is still alive. Return true if the driver knows about the given block manager.
+ * Otherwise, return false, indicating that the block manager should re-register.
*/
def executorHeartbeatReceived(
execId: String,
accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])],
- blockManagerId: BlockManagerId): Boolean
+ blockManagerId: BlockManagerId,
+ executorUpdates: ExecutorMetrics): Boolean
/**
* Get an application ID associated with the job.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 598b62f85a1fa..4f870e85ad38d 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -19,7 +19,7 @@ package org.apache.spark.scheduler
import java.nio.ByteBuffer
import java.util.{Locale, Timer, TimerTask}
-import java.util.concurrent.TimeUnit
+import java.util.concurrent.{ConcurrentHashMap, TimeUnit}
import java.util.concurrent.atomic.AtomicLong
import scala.collection.Set
@@ -28,8 +28,10 @@ import scala.util.Random
import org.apache.spark._
import org.apache.spark.TaskState.TaskState
+import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config
+import org.apache.spark.rpc.RpcEndpoint
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.scheduler.TaskLocality.TaskLocality
import org.apache.spark.storage.BlockManagerId
@@ -90,7 +92,7 @@ private[spark] class TaskSchedulerImpl(
private val taskSetsByStageIdAndAttempt = new HashMap[Int, HashMap[Int, TaskSetManager]]
// Protected by `this`
- private[scheduler] val taskIdToTaskSetManager = new HashMap[Long, TaskSetManager]
+ private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, TaskSetManager]
val taskIdToExecutorId = new HashMap[Long, String]
@volatile private var hasReceivedTask = false
@@ -138,6 +140,19 @@ private[spark] class TaskSchedulerImpl(
// This is a var so that we can reset it for testing purposes.
private[spark] var taskResultGetter = new TaskResultGetter(sc.env, this)
+ private lazy val barrierSyncTimeout = conf.get(config.BARRIER_SYNC_TIMEOUT)
+
+ private[scheduler] var barrierCoordinator: RpcEndpoint = null
+
+ private def maybeInitBarrierCoordinator(): Unit = {
+ if (barrierCoordinator == null) {
+ barrierCoordinator = new BarrierCoordinator(barrierSyncTimeout, sc.listenerBus,
+ sc.env.rpcEnv)
+ sc.env.rpcEnv.setupEndpoint("barrierSync", barrierCoordinator)
+ logInfo("Registered BarrierCoordinator endpoint")
+ }
+ }
+
override def setDAGScheduler(dagScheduler: DAGScheduler) {
this.dagScheduler = dagScheduler
}
@@ -222,18 +237,11 @@ private[spark] class TaskSchedulerImpl(
override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized {
logInfo("Cancelling stage " + stageId)
+ // Kill all running tasks for the stage.
+ killAllTaskAttempts(stageId, interruptThread, reason = "Stage cancelled")
+ // Cancel all attempts for the stage.
taskSetsByStageIdAndAttempt.get(stageId).foreach { attempts =>
attempts.foreach { case (_, tsm) =>
- // There are two possible cases here:
- // 1. The task set manager has been created and some tasks have been scheduled.
- // In this case, send a kill signal to the executors to kill the task and then abort
- // the stage.
- // 2. The task set manager has been created but no tasks have been scheduled. In this case,
- // simply abort the stage.
- tsm.runningTasksSet.foreach { tid =>
- taskIdToExecutorId.get(tid).foreach(execId =>
- backend.killTask(tid, execId, interruptThread, reason = "Stage cancelled"))
- }
tsm.abort("Stage %s cancelled".format(stageId))
logInfo("Stage %d was cancelled".format(stageId))
}
@@ -252,6 +260,27 @@ private[spark] class TaskSchedulerImpl(
}
}
+ override def killAllTaskAttempts(
+ stageId: Int,
+ interruptThread: Boolean,
+ reason: String): Unit = synchronized {
+ logInfo(s"Killing all running tasks in stage $stageId: $reason")
+ taskSetsByStageIdAndAttempt.get(stageId).foreach { attempts =>
+ attempts.foreach { case (_, tsm) =>
+ // There are two possible cases here:
+ // 1. The task set manager has been created and some tasks have been scheduled.
+ // In this case, send a kill signal to the executors to kill the task.
+ // 2. The task set manager has been created but no tasks have been scheduled. In this case,
+ // simply continue.
+ tsm.runningTasksSet.foreach { tid =>
+ taskIdToExecutorId.get(tid).foreach { execId =>
+ backend.killTask(tid, execId, interruptThread, reason)
+ }
+ }
+ }
+ }
+ }
+
/**
* Called to indicate that all task attempts (including speculated tasks) associated with the
* given TaskSetManager have completed, so state associated with the TaskSetManager should be
@@ -274,7 +303,8 @@ private[spark] class TaskSchedulerImpl(
maxLocality: TaskLocality,
shuffledOffers: Seq[WorkerOffer],
availableCpus: Array[Int],
- tasks: IndexedSeq[ArrayBuffer[TaskDescription]]) : Boolean = {
+ tasks: IndexedSeq[ArrayBuffer[TaskDescription]],
+ addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = {
var launchedTask = false
// nodes and executors that are blacklisted for the entire application have already been
// filtered out by this point
@@ -286,11 +316,16 @@ private[spark] class TaskSchedulerImpl(
for (task <- taskSet.resourceOffer(execId, host, maxLocality)) {
tasks(i) += task
val tid = task.taskId
- taskIdToTaskSetManager(tid) = taskSet
+ taskIdToTaskSetManager.put(tid, taskSet)
taskIdToExecutorId(tid) = execId
executorIdToRunningTaskIds(execId).add(tid)
availableCpus(i) -= CPUS_PER_TASK
assert(availableCpus(i) >= 0)
+ // Only update hosts for a barrier task.
+ if (taskSet.isBarrier) {
+ // The executor address is expected to be non empty.
+ addressesWithDescs += (shuffledOffers(i).address.get -> task)
+ }
launchedTask = true
}
} catch {
@@ -346,6 +381,7 @@ private[spark] class TaskSchedulerImpl(
// Build a list of tasks to assign to each worker.
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK))
val availableCpus = shuffledOffers.map(o => o.cores).toArray
+ val availableSlots = shuffledOffers.map(o => o.cores / CPUS_PER_TASK).sum
val sortedTaskSets = rootPool.getSortedTaskSetQueue
for (taskSet <- sortedTaskSets) {
logDebug("parentName: %s, name: %s, runningTasks: %s".format(
@@ -359,20 +395,58 @@ private[spark] class TaskSchedulerImpl(
// of locality levels so that it gets a chance to launch local tasks on all of them.
// NOTE: the preferredLocality order: PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY
for (taskSet <- sortedTaskSets) {
- var launchedAnyTask = false
- var launchedTaskAtCurrentMaxLocality = false
- for (currentMaxLocality <- taskSet.myLocalityLevels) {
- do {
- launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet(
- taskSet, currentMaxLocality, shuffledOffers, availableCpus, tasks)
- launchedAnyTask |= launchedTaskAtCurrentMaxLocality
- } while (launchedTaskAtCurrentMaxLocality)
- }
- if (!launchedAnyTask) {
- taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
+ // Skip the barrier taskSet if the available slots are less than the number of pending tasks.
+ if (taskSet.isBarrier && availableSlots < taskSet.numTasks) {
+ // Skip the launch process.
+ // TODO SPARK-24819 If the job requires more slots than available (both busy and free
+ // slots), fail the job on submit.
+ logInfo(s"Skip current round of resource offers for barrier stage ${taskSet.stageId} " +
+ s"because the barrier taskSet requires ${taskSet.numTasks} slots, while the total " +
+ s"number of available slots is $availableSlots.")
+ } else {
+ var launchedAnyTask = false
+ // Record all the executor IDs assigned barrier tasks on.
+ val addressesWithDescs = ArrayBuffer[(String, TaskDescription)]()
+ for (currentMaxLocality <- taskSet.myLocalityLevels) {
+ var launchedTaskAtCurrentMaxLocality = false
+ do {
+ launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet(taskSet,
+ currentMaxLocality, shuffledOffers, availableCpus, tasks, addressesWithDescs)
+ launchedAnyTask |= launchedTaskAtCurrentMaxLocality
+ } while (launchedTaskAtCurrentMaxLocality)
+ }
+ if (!launchedAnyTask) {
+ taskSet.abortIfCompletelyBlacklisted(hostToExecutors)
+ }
+ if (launchedAnyTask && taskSet.isBarrier) {
+ // Check whether the barrier tasks are partially launched.
+ // TODO SPARK-24818 handle the assert failure case (that can happen when some locality
+ // requirements are not fulfilled, and we should revert the launched tasks).
+ require(addressesWithDescs.size == taskSet.numTasks,
+ s"Skip current round of resource offers for barrier stage ${taskSet.stageId} " +
+ s"because only ${addressesWithDescs.size} out of a total number of " +
+ s"${taskSet.numTasks} tasks got resource offers. The resource offers may have " +
+ "been blacklisted or cannot fulfill task locality requirements.")
+
+ // materialize the barrier coordinator.
+ maybeInitBarrierCoordinator()
+
+ // Update the taskInfos into all the barrier task properties.
+ val addressesStr = addressesWithDescs
+ // Addresses ordered by partitionId
+ .sortBy(_._2.partitionId)
+ .map(_._1)
+ .mkString(",")
+ addressesWithDescs.foreach(_._2.properties.setProperty("addresses", addressesStr))
+
+ logInfo(s"Successfully scheduled all the ${addressesWithDescs.size} tasks for barrier " +
+ s"stage ${taskSet.stageId}.")
+ }
}
}
+ // TODO SPARK-24823 Cancel a job that contains barrier stage(s) if the barrier tasks don't get
+ // launched within a configured time.
if (tasks.size > 0) {
hasLaunchedTask = true
}
@@ -392,7 +466,7 @@ private[spark] class TaskSchedulerImpl(
var reason: Option[ExecutorLossReason] = None
synchronized {
try {
- taskIdToTaskSetManager.get(tid) match {
+ Option(taskIdToTaskSetManager.get(tid)) match {
case Some(taskSet) =>
if (state == TaskState.LOST) {
// TaskState.LOST is only used by the deprecated Mesos fine-grained scheduling mode,
@@ -435,24 +509,26 @@ private[spark] class TaskSchedulerImpl(
}
/**
- * Update metrics for in-progress tasks and let the master know that the BlockManager is still
- * alive. Return true if the driver knows about the given block manager. Otherwise, return false,
- * indicating that the block manager should re-register.
+ * Update metrics for in-progress tasks and executor metrics, and let the master know that the
+ * BlockManager is still alive. Return true if the driver knows about the given block manager.
+ * Otherwise, return false, indicating that the block manager should re-register.
*/
override def executorHeartbeatReceived(
execId: String,
accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])],
- blockManagerId: BlockManagerId): Boolean = {
+ blockManagerId: BlockManagerId,
+ executorMetrics: ExecutorMetrics): Boolean = {
// (taskId, stageId, stageAttemptId, accumUpdates)
- val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = synchronized {
+ val accumUpdatesWithTaskIds: Array[(Long, Int, Int, Seq[AccumulableInfo])] = {
accumUpdates.flatMap { case (id, updates) =>
val accInfos = updates.map(acc => acc.toInfo(Some(acc.value), None))
- taskIdToTaskSetManager.get(id).map { taskSetMgr =>
+ Option(taskIdToTaskSetManager.get(id)).map { taskSetMgr =>
(id, taskSetMgr.stageId, taskSetMgr.taskSet.stageAttemptId, accInfos)
}
}
}
- dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId)
+ dagScheduler.executorHeartbeatReceived(execId, accumUpdatesWithTaskIds, blockManagerId,
+ executorMetrics)
}
def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized {
@@ -510,6 +586,9 @@ private[spark] class TaskSchedulerImpl(
if (taskResultGetter != null) {
taskResultGetter.stop()
}
+ if (barrierCoordinator != null) {
+ barrierCoordinator.stop()
+ }
starvationTimer.cancel()
}
@@ -697,9 +776,12 @@ private[spark] class TaskSchedulerImpl(
* do not also submit those same tasks. That also means that a task completion from an earlier
* attempt can lead to the entire stage getting marked as successful.
*/
- private[scheduler] def markPartitionCompletedInAllTaskSets(stageId: Int, partitionId: Int) = {
+ private[scheduler] def markPartitionCompletedInAllTaskSets(
+ stageId: Int,
+ partitionId: Int,
+ taskInfo: TaskInfo) = {
taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm =>
- tsm.markPartitionCompleted(partitionId)
+ tsm.markPartitionCompleted(partitionId, taskInfo)
}
}
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 a18c66596852a..d5e85a11cb279 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -29,7 +29,7 @@ import org.apache.spark._
import org.apache.spark.TaskState.TaskState
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.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils}
import org.apache.spark.util.collection.MedianHeap
/**
@@ -84,10 +84,10 @@ 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
+ // Add the tid of task into this HashSet when the task is 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)
+ private val killedByOtherAttempt = new HashSet[Long]
val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil)
private[scheduler] var tasksSuccessful = 0
@@ -123,6 +123,10 @@ private[spark] class TaskSetManager(
// TODO: We should kill any running task attempts when the task set manager becomes a zombie.
private[scheduler] var isZombie = false
+ // Whether the taskSet run tasks from a barrier stage. Spark must launch all the tasks at the
+ // same time for a barrier stage.
+ private[scheduler] def isBarrier = taskSet.tasks.nonEmpty && taskSet.tasks(0).isBarrier
+
// Set of pending tasks for each executor. These collections are actually
// treated as stacks, in which new tasks are added to the end of the
// ArrayBuffer and removed from the end. This makes it faster to detect
@@ -512,6 +516,7 @@ private[spark] class TaskSetManager(
execId,
taskName,
index,
+ task.partitionId,
addedFiles,
addedJars,
task.localProperties,
@@ -723,6 +728,23 @@ private[spark] class TaskSetManager(
def handleSuccessfulTask(tid: Long, result: DirectTaskResult[_]): Unit = {
val info = taskInfos(tid)
val index = info.index
+ // Check if any other attempt succeeded before this and this attempt has not been handled
+ if (successful(index) && killedByOtherAttempt.contains(tid)) {
+ // Undo the effect on calculatedTasks and totalResultSize made earlier when
+ // checking if can fetch more results
+ calculatedTasks -= 1
+ val resultSizeAcc = result.accumUpdates.find(a =>
+ a.name == Some(InternalAccumulator.RESULT_SIZE))
+ if (resultSizeAcc.isDefined) {
+ totalResultSize -= resultSizeAcc.get.asInstanceOf[LongAccumulator].value
+ }
+
+ // Handle this task as a killed task
+ handleFailedTask(tid, TaskState.KILLED,
+ TaskKilled("Finish but did not commit due to another attempt succeeded"))
+ return
+ }
+
info.markFinished(TaskState.FINISHED, clock.getTimeMillis())
if (speculationEnabled) {
successfulTaskDurations.insert(info.duration)
@@ -735,7 +757,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
+ killedByOtherAttempt += attemptInfo.taskId
sched.backend.killTask(
attemptInfo.taskId,
attemptInfo.executorId,
@@ -758,7 +780,7 @@ private[spark] class TaskSetManager(
}
// There may be multiple tasksets for this stage -- we let all of them know that the partition
// was completed. This may result in some of the tasksets getting completed.
- sched.markPartitionCompletedInAllTaskSets(stageId, tasks(index).partitionId)
+ sched.markPartitionCompletedInAllTaskSets(stageId, tasks(index).partitionId, info)
// This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the
// "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not
// "deserialize" the value when holding a lock to avoid blocking other threads. So we call
@@ -769,9 +791,12 @@ private[spark] class TaskSetManager(
maybeFinishTaskSet()
}
- private[scheduler] def markPartitionCompleted(partitionId: Int): Unit = {
+ private[scheduler] def markPartitionCompleted(partitionId: Int, taskInfo: TaskInfo): Unit = {
partitionToIndex.get(partitionId).foreach { index =>
if (!successful(index)) {
+ if (speculationEnabled && !isZombie) {
+ successfulTaskDurations.insert(taskInfo.duration)
+ }
tasksSuccessful += 1
successful(index) = true
if (tasksSuccessful == numTasks) {
@@ -868,6 +893,10 @@ private[spark] class TaskSetManager(
None
}
+ if (tasks(index).isBarrier) {
+ isZombie = true
+ }
+
sched.dagScheduler.taskEnded(tasks(index), reason, null, accumUpdates, info)
if (!isZombie && reason.countTowardsTaskFailures) {
@@ -944,7 +973,7 @@ private[spark] class TaskSetManager(
&& !isZombie) {
for ((tid, info) <- taskInfos if info.executorId == execId) {
val index = taskInfos(tid).index
- if (successful(index) && !killedByOtherAttempt(index)) {
+ if (successful(index) && !killedByOtherAttempt.contains(tid)) {
successful(index) = false
copiesRunning(index) -= 1
tasksSuccessful -= 1
@@ -976,8 +1005,8 @@ private[spark] class TaskSetManager(
*/
override def checkSpeculatableTasks(minTimeToSpeculation: Int): Boolean = {
// Can't speculate if we only have one task, and no need to speculate if the task set is a
- // zombie.
- if (isZombie || numTasks == 1) {
+ // zombie or is from a barrier stage.
+ if (isZombie || isBarrier || numTasks == 1) {
return false
}
var foundTasks = false
diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
index 810b36cddf835..6ec74913e42f2 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala
@@ -21,4 +21,10 @@ package org.apache.spark.scheduler
* Represents free resources available on an executor.
*/
private[spark]
-case class WorkerOffer(executorId: String, host: String, cores: Int)
+case class WorkerOffer(
+ executorId: String,
+ host: String,
+ cores: Int,
+ // `address` is an optional hostPort string, it provide more useful information than `host`
+ // when multiple executors are launched on the same host.
+ address: Option[String] = None)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index 9b90e309d2e04..de7c0d813ae65 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -242,7 +242,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
val activeExecutors = executorDataMap.filterKeys(executorIsAlive)
val workOffers = activeExecutors.map {
case (id, executorData) =>
- new WorkerOffer(id, executorData.executorHost, executorData.freeCores)
+ new WorkerOffer(id, executorData.executorHost, executorData.freeCores,
+ Some(executorData.executorAddress.hostPort))
}.toIndexedSeq
scheduler.resourceOffers(workOffers)
}
@@ -267,7 +268,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
if (executorIsAlive(executorId)) {
val executorData = executorDataMap(executorId)
val workOffers = IndexedSeq(
- new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores))
+ new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores,
+ Some(executorData.executorAddress.hostPort)))
scheduler.resourceOffers(workOffers)
} else {
Seq.empty
@@ -288,7 +290,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
for (task <- tasks.flatten) {
val serializedTask = TaskDescription.encode(task)
if (serializedTask.limit() >= maxRpcMessageSize) {
- scheduler.taskIdToTaskSetManager.get(task.taskId).foreach { taskSetMgr =>
+ Option(scheduler.taskIdToTaskSetManager.get(task.taskId)).foreach { taskSetMgr =>
try {
var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " +
"spark.rpc.message.maxSize (%d bytes). Consider increasing " +
@@ -494,6 +496,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp
executorDataMap.keySet.toSeq
}
+ override def maxNumConcurrentTasks(): Int = {
+ executorDataMap.values.map { executor =>
+ executor.totalCores / scheduler.CPUS_PER_TASK
+ }.sum
+ }
+
/**
* Request an additional number of executors from the cluster manager.
* @return whether the request is acknowledged.
diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
index 4c614c5c0f602..0de57fbd5600c 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala
@@ -81,7 +81,8 @@ private[spark] class LocalEndpoint(
}
def reviveOffers() {
- val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores))
+ val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores,
+ Some(rpcEnv.address.hostPort)))
for (task <- scheduler.resourceOffers(offers).flatten) {
freeCores -= scheduler.CPUS_PER_TASK
executor.launchTask(executorBackend, task)
@@ -155,6 +156,8 @@ private[spark] class LocalSchedulerBackend(
override def applicationId(): String = appId
+ override def maxNumConcurrentTasks(): Int = totalCores / scheduler.CPUS_PER_TASK
+
private def stop(finalState: SparkAppHandle.State): Unit = {
localEndpoint.ask(StopExecutor)
try {
diff --git a/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
index d15e7937b0523..ea38ccb289c30 100644
--- a/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
+++ b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala
@@ -42,43 +42,59 @@ private[spark] class SocketAuthHelper(conf: SparkConf) {
* Read the auth secret from the socket and compare to the expected value. Write the reply back
* to the socket.
*
- * If authentication fails, this method will close the socket.
+ * If authentication fails or error is thrown, this method will close the socket.
*
* @param s The client socket.
* @throws IllegalArgumentException If authentication fails.
*/
def authClient(s: Socket): Unit = {
- // Set the socket timeout while checking the auth secret. Reset it before returning.
- val currentTimeout = s.getSoTimeout()
+ var shouldClose = true
try {
- s.setSoTimeout(10000)
- val clientSecret = readUtf8(s)
- if (secret == clientSecret) {
- writeUtf8("ok", s)
- } else {
- writeUtf8("err", s)
- JavaUtils.closeQuietly(s)
+ // Set the socket timeout while checking the auth secret. Reset it before returning.
+ val currentTimeout = s.getSoTimeout()
+ try {
+ s.setSoTimeout(10000)
+ val clientSecret = readUtf8(s)
+ if (secret == clientSecret) {
+ writeUtf8("ok", s)
+ shouldClose = false
+ } else {
+ writeUtf8("err", s)
+ throw new IllegalArgumentException("Authentication failed.")
+ }
+ } finally {
+ s.setSoTimeout(currentTimeout)
}
} finally {
- s.setSoTimeout(currentTimeout)
+ if (shouldClose) {
+ JavaUtils.closeQuietly(s)
+ }
}
}
/**
* Authenticate with a server by writing the auth secret and checking the server's reply.
*
- * If authentication fails, this method will close the socket.
+ * If authentication fails or error is thrown, this method will close the socket.
*
* @param s The socket connected to the server.
* @throws IllegalArgumentException If authentication fails.
*/
def authToServer(s: Socket): Unit = {
- writeUtf8(secret, s)
+ var shouldClose = true
+ try {
+ writeUtf8(secret, s)
- val reply = readUtf8(s)
- if (reply != "ok") {
- JavaUtils.closeQuietly(s)
- throw new IllegalArgumentException("Authentication failed.")
+ val reply = readUtf8(s)
+ if (reply != "ok") {
+ throw new IllegalArgumentException("Authentication failed.")
+ } else {
+ shouldClose = false
+ }
+ } finally {
+ if (shouldClose) {
+ JavaUtils.closeQuietly(s)
+ }
}
}
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 4103dfb10175e..74b0e0b3a741a 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala
@@ -104,7 +104,7 @@ private[spark] class BlockStoreShuffleReader[K, C](
context.taskMetrics().incDiskBytesSpilled(sorter.diskBytesSpilled)
context.taskMetrics().incPeakExecutionMemory(sorter.peakMemoryUsedBytes)
// Use completion callback to stop sorter if task was finished/cancelled.
- context.addTaskCompletionListener(_ => {
+ context.addTaskCompletionListener[Unit](_ => {
sorter.stop()
})
CompletionIterator[Product2[K, C], Iterator[Product2[K, C]]](sorter.iterator, sorter.stop())
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
index d9fad64f34c7c..0caf84c6050a8 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala
@@ -27,7 +27,7 @@ import org.apache.spark.shuffle._
* In sort-based shuffle, incoming records are sorted according to their target partition ids, then
* written to a single map output file. Reducers fetch contiguous regions of this file in order to
* read their portion of the map output. In cases where the map output data is too large to fit in
- * memory, sorted subsets of the output can are spilled to disk and those on-disk files are merged
+ * memory, sorted subsets of the output can be spilled to disk and those on-disk files are merged
* to produce the final output file.
*
* Sort-based shuffle has two different write paths for producing its map output files:
diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
index 274399b9cc1f3..91fc26762e533 100644
--- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
+++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala
@@ -70,7 +70,8 @@ private[spark] class SortShuffleWriter[K, V, C](
val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID)
val partitionLengths = sorter.writePartitionedFile(blockId, tmp)
shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp)
- mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths)
+ mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths,
+ writeMetrics.recordsWritten)
} finally {
if (tmp.exists() && !tmp.delete()) {
logError(s"Error while deleting temp file ${tmp.getAbsolutePath}")
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index 5ea161cd0d151..f21eee1965761 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -25,7 +25,7 @@ import scala.collection.JavaConverters._
import scala.collection.mutable.HashMap
import org.apache.spark._
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
import org.apache.spark.internal.Logging
import org.apache.spark.scheduler._
import org.apache.spark.status.api.v1
@@ -66,6 +66,7 @@ private[spark] class AppStatusListener(
private val liveStages = new ConcurrentHashMap[(Int, Int), LiveStage]()
private val liveJobs = new HashMap[Int, LiveJob]()
private val liveExecutors = new HashMap[String, LiveExecutor]()
+ private val deadExecutors = new HashMap[String, LiveExecutor]()
private val liveTasks = new HashMap[Long, LiveTask]()
private val liveRDDs = new HashMap[Int, LiveRDD]()
private val pools = new HashMap[String, SchedulerPool]()
@@ -204,6 +205,19 @@ private[spark] class AppStatusListener(
update(rdd, now)
}
}
+ if (isExecutorActiveForLiveStages(exec)) {
+ // the executor was running for a currently active stage, so save it for now in
+ // deadExecutors, and remove when there are no active stages overlapping with the
+ // executor.
+ deadExecutors.put(event.executorId, exec)
+ }
+ }
+ }
+
+ /** Was the specified executor active for any currently live stages? */
+ private def isExecutorActiveForLiveStages(exec: LiveExecutor): Boolean = {
+ liveStages.values.asScala.exists { stage =>
+ stage.info.submissionTime.getOrElse(0L) < exec.removeTime.getTime
}
}
@@ -350,11 +364,20 @@ private[spark] class AppStatusListener(
val e = it.next()
if (job.stageIds.contains(e.getKey()._1)) {
val stage = e.getValue()
- stage.status = v1.StageStatus.SKIPPED
- job.skippedStages += stage.info.stageId
- job.skippedTasks += stage.info.numTasks
- it.remove()
- update(stage, now)
+ if (v1.StageStatus.PENDING.equals(stage.status)) {
+ stage.status = v1.StageStatus.SKIPPED
+ job.skippedStages += stage.info.stageId
+ job.skippedTasks += stage.info.numTasks
+ job.activeStages -= 1
+
+ pools.get(stage.schedulingPool).foreach { pool =>
+ pool.stageIds = pool.stageIds - stage.info.stageId
+ update(pool, now)
+ }
+
+ it.remove()
+ update(stage, now, last = true)
+ }
}
}
@@ -506,7 +529,16 @@ private[spark] class AppStatusListener(
if (killedDelta > 0) {
stage.killedSummary = killedTasksSummary(event.reason, stage.killedSummary)
}
- maybeUpdate(stage, now)
+ // [SPARK-24415] Wait for all tasks to finish before removing stage from live list
+ val removeStage =
+ stage.activeTasks == 0 &&
+ (v1.StageStatus.COMPLETE.equals(stage.status) ||
+ v1.StageStatus.FAILED.equals(stage.status))
+ if (removeStage) {
+ update(stage, now, last = true)
+ } else {
+ maybeUpdate(stage, now)
+ }
// Store both stage ID and task index in a single long variable for tracking at job level.
val taskIndex = (event.stageId.toLong << Integer.SIZE) | event.taskInfo.index
@@ -521,7 +553,7 @@ private[spark] class AppStatusListener(
if (killedDelta > 0) {
job.killedSummary = killedTasksSummary(event.reason, job.killedSummary)
}
- maybeUpdate(job, now)
+ conditionalLiveUpdate(job, now, removeStage)
}
val esummary = stage.executorSummary(event.taskInfo.executorId)
@@ -532,7 +564,7 @@ private[spark] class AppStatusListener(
if (metricsDelta != null) {
esummary.metrics = LiveEntityHelpers.addMetrics(esummary.metrics, metricsDelta)
}
- maybeUpdate(esummary, now)
+ conditionalLiveUpdate(esummary, now, removeStage)
if (!stage.cleaning && stage.savedTasks.get() > maxTasksPerStage) {
stage.cleaning = true
@@ -540,6 +572,9 @@ private[spark] class AppStatusListener(
cleanupTasks(stage)
}
}
+ if (removeStage) {
+ liveStages.remove((event.stageId, event.stageAttemptId))
+ }
}
liveExecutors.get(event.taskInfo.executorId).foreach { exec =>
@@ -564,17 +599,13 @@ private[spark] class AppStatusListener(
// Force an update on live applications when the number of active tasks reaches 0. This is
// checked in some tests (e.g. SQLTestUtilsBase) so it needs to be reliably up to date.
- if (exec.activeTasks == 0) {
- liveUpdate(exec, now)
- } else {
- maybeUpdate(exec, now)
- }
+ conditionalLiveUpdate(exec, now, exec.activeTasks == 0)
}
}
override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
val maybeStage =
- Option(liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber)))
+ Option(liveStages.get((event.stageInfo.stageId, event.stageInfo.attemptNumber)))
maybeStage.foreach { stage =>
val now = System.nanoTime()
stage.info = event.stageInfo
@@ -608,7 +639,6 @@ private[spark] class AppStatusListener(
}
stage.executorSummaries.values.foreach(update(_, now))
- update(stage, now, last = true)
val executorIdsForStage = stage.blackListedExecutors
executorIdsForStage.foreach { executorId =>
@@ -616,8 +646,18 @@ private[spark] class AppStatusListener(
removeBlackListedStageFrom(exec, event.stageInfo.stageId, now)
}
}
+
+ // Remove stage only if there are no active tasks remaining
+ val removeStage = stage.activeTasks == 0
+ update(stage, now, last = removeStage)
+ if (removeStage) {
+ liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber))
+ }
}
+ // remove any dead executors that were not running for any currently active stages
+ deadExecutors.retain((execId, exec) => isExecutorActiveForLiveStages(exec))
+
appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1)
kvstore.write(appSummary)
}
@@ -646,7 +686,37 @@ private[spark] class AppStatusListener(
}
override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = {
- liveRDDs.remove(event.rddId)
+ liveRDDs.remove(event.rddId).foreach { liveRDD =>
+ val storageLevel = liveRDD.info.storageLevel
+
+ // Use RDD partition info to update executor block info.
+ liveRDD.getPartitions().foreach { case (_, part) =>
+ part.executors.foreach { executorId =>
+ liveExecutors.get(executorId).foreach { exec =>
+ exec.rddBlocks = exec.rddBlocks - 1
+ }
+ }
+ }
+
+ val now = System.nanoTime()
+
+ // Use RDD distribution to update executor memory and disk usage info.
+ liveRDD.getDistributions().foreach { case (executorId, rddDist) =>
+ liveExecutors.get(executorId).foreach { exec =>
+ if (exec.hasMemoryInfo) {
+ if (storageLevel.useOffHeap) {
+ exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, -rddDist.offHeapUsed)
+ } else {
+ exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, -rddDist.onHeapUsed)
+ }
+ }
+ exec.memoryUsed = addDeltaToValue(exec.memoryUsed, -rddDist.memoryUsed)
+ exec.diskUsed = addDeltaToValue(exec.diskUsed, -rddDist.diskUsed)
+ maybeUpdate(exec, now)
+ }
+ }
+ }
+
kvstore.delete(classOf[RDDStorageInfoWrapper], event.rddId)
}
@@ -669,6 +739,31 @@ private[spark] class AppStatusListener(
}
}
}
+
+ // check if there is a new peak value for any of the executor level memory metrics
+ // for the live UI. SparkListenerExecutorMetricsUpdate events are only processed
+ // for the live UI.
+ event.executorUpdates.foreach { updates =>
+ liveExecutors.get(event.execId).foreach { exec =>
+ if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) {
+ maybeUpdate(exec, now)
+ }
+ }
+ }
+ }
+
+ override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = {
+ val now = System.nanoTime()
+
+ // check if there is a new peak value for any of the executor level memory metrics,
+ // while reading from the log. SparkListenerStageExecutorMetrics are only processed
+ // when reading logs.
+ liveExecutors.get(executorMetrics.execId)
+ .orElse(deadExecutors.get(executorMetrics.execId)).map { exec =>
+ if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) {
+ update(exec, now)
+ }
+ }
}
override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {
@@ -705,6 +800,11 @@ private[spark] class AppStatusListener(
.sortBy(_.stageId)
}
+ /**
+ * Apply a delta to a value, but ensure that it doesn't go negative.
+ */
+ private def addDeltaToValue(old: Long, delta: Long): Long = math.max(0, old + delta)
+
private def updateRDDBlock(event: SparkListenerBlockUpdated, block: RDDBlockId): Unit = {
val now = System.nanoTime()
val executorId = event.blockUpdatedInfo.blockManagerId.executorId
@@ -714,9 +814,6 @@ private[spark] class AppStatusListener(
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 {
@@ -733,13 +830,13 @@ private[spark] class AppStatusListener(
maybeExec.foreach { exec =>
if (exec.hasMemoryInfo) {
if (storageLevel.useOffHeap) {
- exec.usedOffHeap = newValue(exec.usedOffHeap, memoryDelta)
+ exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta)
} else {
- exec.usedOnHeap = newValue(exec.usedOnHeap, memoryDelta)
+ exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta)
}
}
- exec.memoryUsed = newValue(exec.memoryUsed, memoryDelta)
- exec.diskUsed = newValue(exec.diskUsed, diskDelta)
+ exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta)
+ exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta)
}
// Update the block entry in the RDD info, keeping track of the deltas above so that we
@@ -767,8 +864,8 @@ private[spark] class AppStatusListener(
// Only update the partition if it's still stored in some executor, otherwise get rid of it.
if (executors.nonEmpty) {
partition.update(executors, rdd.storageLevel,
- newValue(partition.memoryUsed, memoryDelta),
- newValue(partition.diskUsed, diskDelta))
+ addDeltaToValue(partition.memoryUsed, memoryDelta),
+ addDeltaToValue(partition.diskUsed, diskDelta))
} else {
rdd.removePartition(block.name)
}
@@ -776,14 +873,14 @@ private[spark] class AppStatusListener(
maybeExec.foreach { exec =>
if (exec.rddBlocks + rddBlocksDelta > 0) {
val dist = rdd.distribution(exec)
- dist.memoryUsed = newValue(dist.memoryUsed, memoryDelta)
- dist.diskUsed = newValue(dist.diskUsed, diskDelta)
+ dist.memoryUsed = addDeltaToValue(dist.memoryUsed, memoryDelta)
+ dist.diskUsed = addDeltaToValue(dist.diskUsed, diskDelta)
if (exec.hasMemoryInfo) {
if (storageLevel.useOffHeap) {
- dist.offHeapUsed = newValue(dist.offHeapUsed, memoryDelta)
+ dist.offHeapUsed = addDeltaToValue(dist.offHeapUsed, memoryDelta)
} else {
- dist.onHeapUsed = newValue(dist.onHeapUsed, memoryDelta)
+ dist.onHeapUsed = addDeltaToValue(dist.onHeapUsed, memoryDelta)
}
}
dist.lastUpdate = null
@@ -802,8 +899,8 @@ private[spark] class AppStatusListener(
}
}
- rdd.memoryUsed = newValue(rdd.memoryUsed, memoryDelta)
- rdd.diskUsed = newValue(rdd.diskUsed, diskDelta)
+ rdd.memoryUsed = addDeltaToValue(rdd.memoryUsed, memoryDelta)
+ rdd.diskUsed = addDeltaToValue(rdd.diskUsed, diskDelta)
update(rdd, now)
}
@@ -882,6 +979,14 @@ private[spark] class AppStatusListener(
}
}
+ private def conditionalLiveUpdate(entity: LiveEntity, now: Long, condition: Boolean): Unit = {
+ if (condition) {
+ liveUpdate(entity, now)
+ } else {
+ maybeUpdate(entity, now)
+ }
+ }
+
private def cleanupExecutors(count: Long): Unit = {
// Because the limit is on the number of *dead* executors, we need to calculate whether
// there are actually enough dead executors to be deleted.
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
index 688f25a9fdea1..e237281c552b1 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala
@@ -471,7 +471,7 @@ private[spark] class AppStatusStore(
def operationGraphForJob(jobId: Int): Seq[RDDOperationGraph] = {
val job = store.read(classOf[JobDataWrapper], jobId)
- val stages = job.info.stageIds
+ val stages = job.info.stageIds.sorted
stages.map { id =>
val g = store.read(classOf[RDDOperationGraphWrapper], id).toRDDOperationGraph()
diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
index 79e3f13b826ce..8708e64db3c17 100644
--- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
+++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala
@@ -26,14 +26,13 @@ import scala.collection.mutable.HashMap
import com.google.common.collect.Interners
import org.apache.spark.JobExecutionStatus
-import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.executor.{ExecutorMetrics, 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.collection.OpenHashSet
-import org.apache.spark.util.kvstore.KVStore
/**
* A mutable representation of a live entity in Spark (jobs, stages, tasks, et al). Every live
@@ -268,6 +267,9 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
def hasMemoryInfo: Boolean = totalOnHeap >= 0L
+ // peak values for executor level metrics
+ val peakExecutorMetrics = new ExecutorMetrics()
+
def hostname: String = if (host != null) host else hostPort.split(":")(0)
override protected def doUpdate(): Any = {
@@ -302,10 +304,10 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
Option(removeReason),
executorLogs,
memoryMetrics,
- blacklistedInStages)
+ blacklistedInStages,
+ Some(peakExecutorMetrics).filter(_.isSet))
new ExecutorSummaryWrapper(info)
}
-
}
private class LiveExecutorStageSummary(
@@ -538,6 +540,10 @@ private class LiveRDD(val info: RDDInfo) extends LiveEntity {
distributions.get(exec.executorId)
}
+ def getPartitions(): scala.collection.Map[String, LiveRDDPartition] = partitions
+
+ def getDistributions(): scala.collection.Map[String, LiveRDDDistribution] = distributions
+
override protected def doUpdate(): Any = {
val dists = if (distributions.nonEmpty) {
Some(distributions.values.map(_.toApi()).toSeq)
@@ -581,8 +587,7 @@ private object LiveEntityHelpers {
.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))
+ !acc.internal && acc.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)
}
.map { acc =>
new v1.AccumulableInfo(
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 971d7e90fa7b8..77466b62ff6ed 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
@@ -22,9 +22,14 @@ import java.util.Date
import scala.xml.{NodeSeq, Text}
import com.fasterxml.jackson.annotation.JsonIgnoreProperties
-import com.fasterxml.jackson.databind.annotation.JsonDeserialize
+import com.fasterxml.jackson.core.{JsonGenerator, JsonParser}
+import com.fasterxml.jackson.core.`type`.TypeReference
+import com.fasterxml.jackson.databind.{DeserializationContext, JsonDeserializer, JsonSerializer, SerializerProvider}
+import com.fasterxml.jackson.databind.annotation.{JsonDeserialize, JsonSerialize}
import org.apache.spark.JobExecutionStatus
+import org.apache.spark.executor.ExecutorMetrics
+import org.apache.spark.metrics.ExecutorMetricType
case class ApplicationInfo private[spark](
id: String,
@@ -98,7 +103,10 @@ class ExecutorSummary private[spark](
val removeReason: Option[String],
val executorLogs: Map[String, String],
val memoryMetrics: Option[MemoryMetrics],
- val blacklistedInStages: Set[Int])
+ val blacklistedInStages: Set[Int],
+ @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer])
+ @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer])
+ val peakMemoryMetrics: Option[ExecutorMetrics])
class MemoryMetrics private[spark](
val usedOnHeapStorageMemory: Long,
@@ -106,6 +114,33 @@ class MemoryMetrics private[spark](
val totalOnHeapStorageMemory: Long,
val totalOffHeapStorageMemory: Long)
+/** deserializer for peakMemoryMetrics: convert map to ExecutorMetrics */
+private[spark] class ExecutorMetricsJsonDeserializer
+ extends JsonDeserializer[Option[ExecutorMetrics]] {
+ override def deserialize(
+ jsonParser: JsonParser,
+ deserializationContext: DeserializationContext): Option[ExecutorMetrics] = {
+ val metricsMap = jsonParser.readValueAs[Option[Map[String, Long]]](
+ new TypeReference[Option[Map[String, java.lang.Long]]] {})
+ metricsMap.map(metrics => new ExecutorMetrics(metrics))
+ }
+}
+/** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */
+private[spark] class ExecutorMetricsJsonSerializer
+ extends JsonSerializer[Option[ExecutorMetrics]] {
+ override def serialize(
+ metrics: Option[ExecutorMetrics],
+ jsonGenerator: JsonGenerator,
+ serializerProvider: SerializerProvider): Unit = {
+ metrics.foreach { m: ExecutorMetrics =>
+ val metricsMap = ExecutorMetricType.values.map { metricType =>
+ metricType.name -> m.getMetricValue(metricType)
+ }.toMap
+ jsonGenerator.writeObject(metricsMap)
+ }
+ }
+}
+
class JobData private[spark](
val jobId: Int,
val name: String,
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 df1a4bef616b2..f5c69ad241e3a 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -41,10 +41,12 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode}
import org.apache.spark.metrics.source.Source
import org.apache.spark.network._
import org.apache.spark.network.buffer.ManagedBuffer
+import org.apache.spark.network.client.StreamCallbackWithID
import org.apache.spark.network.netty.SparkTransportConf
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.scheduler.ExecutorCacheTaskLocation
import org.apache.spark.serializer.{SerializerInstance, SerializerManager}
import org.apache.spark.shuffle.ShuffleManager
import org.apache.spark.storage.memory._
@@ -128,7 +130,11 @@ private[spark] class BlockManager(
extends BlockDataManager with BlockEvictionHandler with Logging {
private[spark] val externalShuffleServiceEnabled =
- conf.getBoolean("spark.shuffle.service.enabled", false)
+ conf.get(config.SHUFFLE_SERVICE_ENABLED)
+ private val chunkSize =
+ conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt
+ private val remoteReadNioBufferConversion =
+ conf.getBoolean("spark.network.remoteReadNioBufferConversion", false)
val diskBlockManager = {
// Only perform cleanup if an external service is not serving our shuffle files.
@@ -159,12 +165,13 @@ private[spark] class BlockManager(
// Port used by the external shuffle service. In Yarn mode, this may be already be
// set through the Hadoop configuration as the server is launched in the Yarn NM.
private val externalShuffleServicePort = {
- val tmpPort = Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt
+ val tmpPort = Utils.getSparkOrYarnConfig(conf, config.SHUFFLE_SERVICE_PORT.key,
+ config.SHUFFLE_SERVICE_PORT.defaultValueString).toInt
if (tmpPort == 0) {
// for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds
// an open port. But we still need to tell our spark apps the right port to use. So
// only if the yarn config has the port set to 0, we prefer the value in the spark config
- conf.get("spark.shuffle.service.port").toInt
+ conf.get(config.SHUFFLE_SERVICE_PORT.key).toInt
} else {
tmpPort
}
@@ -401,6 +408,63 @@ private[spark] class BlockManager(
putBytes(blockId, new ChunkedByteBuffer(data.nioByteBuffer()), level)(classTag)
}
+ override def putBlockDataAsStream(
+ blockId: BlockId,
+ level: StorageLevel,
+ classTag: ClassTag[_]): StreamCallbackWithID = {
+ // TODO if we're going to only put the data in the disk store, we should just write it directly
+ // to the final location, but that would require a deeper refactor of this code. So instead
+ // we just write to a temp file, and call putBytes on the data in that file.
+ val tmpFile = diskBlockManager.createTempLocalBlock()._2
+ val channel = new CountingWritableChannel(
+ Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile))))
+ logTrace(s"Streaming block $blockId to tmp file $tmpFile")
+ new StreamCallbackWithID {
+
+ override def getID: String = blockId.name
+
+ override def onData(streamId: String, buf: ByteBuffer): Unit = {
+ while (buf.hasRemaining) {
+ channel.write(buf)
+ }
+ }
+
+ override def onComplete(streamId: String): Unit = {
+ logTrace(s"Done receiving block $blockId, now putting into local blockManager")
+ // Read the contents of the downloaded file as a buffer to put into the blockManager.
+ // Note this is all happening inside the netty thread as soon as it reads the end of the
+ // stream.
+ channel.close()
+ // TODO SPARK-25035 Even if we're only going to write the data to disk after this, we end up
+ // using a lot of memory here. With encryption, we'll read the whole file into a regular
+ // byte buffer and OOM. Without encryption, we'll memory map the file and won't get a jvm
+ // OOM, but might get killed by the OS / cluster manager. We could at least read the tmp
+ // file as a stream in both cases.
+ val buffer = securityManager.getIOEncryptionKey() match {
+ case Some(key) =>
+ // we need to pass in the size of the unencrypted block
+ val blockSize = channel.getCount
+ val allocator = level.memoryMode match {
+ case MemoryMode.ON_HEAP => ByteBuffer.allocate _
+ case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _
+ }
+ new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator)
+
+ case None =>
+ ChunkedByteBuffer.map(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt)
+ }
+ putBytes(blockId, buffer, level)(classTag)
+ tmpFile.delete()
+ }
+
+ override def onFailure(streamId: String, cause: Throwable): Unit = {
+ // the framework handles the connection itself, we just need to do local cleanup
+ channel.close()
+ tmpFile.delete()
+ }
+ }
+ }
+
/**
* Get the BlockStatus for the block identified by the given ID, if it exists.
* NOTE: This is mainly for testing.
@@ -659,6 +723,11 @@ private[spark] class BlockManager(
* Get block from remote block managers as serialized bytes.
*/
def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = {
+ // TODO if we change this method to return the ManagedBuffer, then getRemoteValues
+ // could just use the inputStream on the temp file, rather than memory-mapping the file.
+ // Until then, replication can cause the process to use too much memory and get killed
+ // by the OS / cluster manager (not a java OOM, since it's a memory-mapped file) even though
+ // we've read the data to disk.
logDebug(s"Getting remote block $blockId")
require(blockId != null, "BlockId is null")
var runningFailureCount = 0
@@ -689,7 +758,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, tempFileManager).nioByteBuffer()
+ loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager)
} catch {
case NonFatal(e) =>
runningFailureCount += 1
@@ -723,7 +792,14 @@ private[spark] class BlockManager(
}
if (data != null) {
- return Some(new ChunkedByteBuffer(data))
+ // SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to
+ // ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if
+ // new path is stable.
+ if (remoteReadNioBufferConversion) {
+ return Some(new ChunkedByteBuffer(data.nioByteBuffer()))
+ } else {
+ return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize))
+ }
}
logDebug(s"The value of block $blockId is null")
}
@@ -1341,12 +1417,16 @@ private[spark] class BlockManager(
try {
val onePeerStartTime = System.nanoTime
logTrace(s"Trying to replicate $blockId of ${data.size} bytes to $peer")
+ // This thread keeps a lock on the block, so we do not want the netty thread to unlock
+ // block when it finishes sending the message.
+ val buffer = new BlockManagerManagedBuffer(blockInfoManager, blockId, data, false,
+ unlockOnDeallocate = false)
blockTransferService.uploadBlockSync(
peer.host,
peer.port,
peer.executorId,
blockId,
- new BlockManagerManagedBuffer(blockInfoManager, blockId, data, false),
+ buffer,
tLevel,
classTag)
logTrace(s"Replicated $blockId of ${data.size} bytes to $peer" +
@@ -1554,7 +1634,7 @@ private[spark] class BlockManager(
private[spark] object BlockManager {
private val ID_GENERATOR = new IdGenerator
- def blockIdsToHosts(
+ def blockIdsToLocations(
blockIds: Array[BlockId],
env: SparkEnv,
blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = {
@@ -1569,7 +1649,9 @@ private[spark] object BlockManager {
val blockManagers = new HashMap[BlockId, Seq[String]]
for (i <- 0 until blockIds.length) {
- blockManagers(blockIds(i)) = blockLocations(i).map(_.host)
+ blockManagers(blockIds(i)) = blockLocations(i).map { loc =>
+ ExecutorCacheTaskLocation(loc.host, loc.executorId).toString
+ }
}
blockManagers.toMap
}
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala
index 3d3806126676c..5c12b5cee4d2f 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerManagedBuffer.scala
@@ -38,7 +38,8 @@ private[storage] class BlockManagerManagedBuffer(
blockInfoManager: BlockInfoManager,
blockId: BlockId,
data: BlockData,
- dispose: Boolean) extends ManagedBuffer {
+ dispose: Boolean,
+ unlockOnDeallocate: Boolean = true) extends ManagedBuffer {
private val refCount = new AtomicInteger(1)
@@ -58,7 +59,9 @@ private[storage] class BlockManagerManagedBuffer(
}
override def release(): ManagedBuffer = {
- blockInfoManager.unlock(blockId)
+ if (unlockOnDeallocate) {
+ blockInfoManager.unlock(blockId)
+ }
if (refCount.decrementAndGet() == 0 && dispose) {
data.dispose()
}
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 8e8f7d197c9ef..f984cf76e3463 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
@@ -54,7 +54,8 @@ class BlockManagerMasterEndpoint(
// Mapping from block id to the set of block managers that have the block.
private val blockLocations = new JHashMap[BlockId, mutable.HashSet[BlockManagerId]]
- private val askThreadPool = ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool")
+ private val askThreadPool =
+ ThreadUtils.newDaemonCachedThreadPool("block-manager-ask-thread-pool", 100)
private implicit val askExecutionContext = ExecutionContext.fromExecutorService(askThreadPool)
private val topologyMapper = {
diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
index 742cf4fe393f9..67544b20408a6 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveEndpoint.scala
@@ -37,7 +37,7 @@ class BlockManagerSlaveEndpoint(
extends ThreadSafeRpcEndpoint with Logging {
private val asyncThreadPool =
- ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool")
+ ThreadUtils.newDaemonCachedThreadPool("block-manager-slave-async-thread-pool", 100)
private implicit val asyncExecutionContext = ExecutionContext.fromExecutorService(asyncThreadPool)
// Operations that involve removing blocks may be slow and should be done asynchronously
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 39249d411b582..a820bc70b33b2 100644
--- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
+++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala
@@ -29,7 +29,7 @@ import com.google.common.io.Closeables
import io.netty.channel.DefaultFileRegion
import org.apache.spark.{SecurityManager, SparkConf}
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils}
import org.apache.spark.security.CryptoStreamUtils
import org.apache.spark.util.Utils
@@ -44,8 +44,7 @@ private[spark] class DiskStore(
securityManager: SecurityManager) extends Logging {
private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m")
- private val maxMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests",
- Int.MaxValue.toString)
+ private val maxMemoryMapBytes = conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS)
private val blockSizes = new ConcurrentHashMap[BlockId, Long]()
def getSize(blockId: BlockId): Long = blockSizes.get(blockId)
@@ -279,7 +278,7 @@ private class ReadableChannelFileRegion(source: ReadableByteChannel, blockSize:
override def transferred(): Long = _transferred
override def transferTo(target: WritableByteChannel, pos: Long): Long = {
- assert(pos == transfered(), "Invalid position.")
+ assert(pos == transferred(), "Invalid position.")
var written = 0L
var lastWrite = -1L
diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
index e5abbf745cc41..19f86569c1e3c 100644
--- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
+++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala
@@ -17,7 +17,9 @@
package org.apache.spark.storage
+import org.apache.spark.SparkEnv
import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.internal.config._
import org.apache.spark.rdd.{RDD, RDDOperationScope}
import org.apache.spark.util.Utils
@@ -53,10 +55,17 @@ class RDDInfo(
}
private[spark] object RDDInfo {
+ private val callsiteLongForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_LONG_FORM)
+
def fromRdd(rdd: RDD[_]): RDDInfo = {
val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd))
val parentIds = rdd.dependencies.map(_.rdd.id)
+ val callSite = if (callsiteLongForm) {
+ rdd.creationSite.longForm
+ } else {
+ rdd.creationSite.shortForm
+ }
new RDDInfo(rdd.id, rddName, rdd.partitions.length,
- rdd.getStorageLevel, parentIds, rdd.creationSite.shortForm, rdd.scope)
+ rdd.getStorageLevel, parentIds, callSite, rdd.scope)
}
}
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 b31862323a895..00d01dd28afb5 100644
--- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
+++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
@@ -346,7 +346,7 @@ final class ShuffleBlockFetcherIterator(
private[this] def initialize(): Unit = {
// Add a task completion callback (called in both success case and failure case) to cleanup.
- context.addTaskCompletionListener(_ => cleanup())
+ context.addTaskCompletionListener[Unit](_ => cleanup())
// Split local and remote blocks.
val remoteRequests = splitLocalRemoteBlocks()
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 4cc5bcb7f9baf..06fd56e54d9c8 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
@@ -827,7 +827,7 @@ private[storage] class PartiallySerializedBlock[T](
// completion listener here in order to ensure that `unrolled.dispose()` is called at least once.
// The dispose() method is idempotent, so it's safe to call it unconditionally.
Option(TaskContext.get()).foreach { taskContext =>
- taskContext.addTaskCompletionListener { _ =>
+ taskContext.addTaskCompletionListener[Unit] { _ =>
// When a task completes, its unroll memory will automatically be freed. Thus we do not call
// releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing.
unrolledBuffer.dispose()
diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
index ad0c0639521f6..43d62561e8eba 100644
--- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
+++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala
@@ -18,12 +18,13 @@
package org.apache.spark.util
import java.io.{ByteArrayInputStream, ByteArrayOutputStream}
+import java.lang.invoke.SerializedLambda
import scala.collection.mutable.{Map, Set, Stack}
import scala.language.existentials
-import org.apache.xbean.asm5.{ClassReader, ClassVisitor, MethodVisitor, Type}
-import org.apache.xbean.asm5.Opcodes._
+import org.apache.xbean.asm6.{ClassReader, ClassVisitor, MethodVisitor, Type}
+import org.apache.xbean.asm6.Opcodes._
import org.apache.spark.{SparkEnv, SparkException}
import org.apache.spark.internal.Logging
@@ -33,6 +34,8 @@ import org.apache.spark.internal.Logging
*/
private[spark] object ClosureCleaner extends Logging {
+ private val isScala2_11 = scala.util.Properties.versionString.contains("2.11")
+
// Get an ASM class reader for a given class from the JAR that loaded it
private[util] def getClassReader(cls: Class[_]): ClassReader = {
// Copy data over, before delegating to ClassReader - else we can run out of open file handles.
@@ -159,6 +162,42 @@ private[spark] object ClosureCleaner extends Logging {
clean(closure, checkSerializable, cleanTransitively, Map.empty)
}
+ /**
+ * Try to get a serialized Lambda from the closure.
+ *
+ * @param closure the closure to check.
+ */
+ private def getSerializedLambda(closure: AnyRef): Option[SerializedLambda] = {
+ if (isScala2_11) {
+ return None
+ }
+ val isClosureCandidate =
+ closure.getClass.isSynthetic &&
+ closure
+ .getClass
+ .getInterfaces.exists(_.getName == "scala.Serializable")
+
+ if (isClosureCandidate) {
+ try {
+ Option(inspect(closure))
+ } catch {
+ case e: Exception =>
+ // no need to check if debug is enabled here the Spark
+ // logging api covers this.
+ logDebug("Closure is not a serialized lambda.", e)
+ None
+ }
+ } else {
+ None
+ }
+ }
+
+ private def inspect(closure: AnyRef): SerializedLambda = {
+ val writeReplace = closure.getClass.getDeclaredMethod("writeReplace")
+ writeReplace.setAccessible(true)
+ writeReplace.invoke(closure).asInstanceOf[java.lang.invoke.SerializedLambda]
+ }
+
/**
* Helper method to clean the given closure in place.
*
@@ -206,7 +245,12 @@ private[spark] object ClosureCleaner extends Logging {
cleanTransitively: Boolean,
accessedFields: Map[Class[_], Set[String]]): Unit = {
- if (!isClosure(func.getClass)) {
+ // most likely to be the case with 2.12, 2.13
+ // so we check first
+ // non LMF-closures should be less frequent from now on
+ val lambdaFunc = getSerializedLambda(func)
+
+ if (!isClosure(func.getClass) && lambdaFunc.isEmpty) {
logDebug(s"Expected a closure; got ${func.getClass.getName}")
return
}
@@ -218,118 +262,132 @@ private[spark] object ClosureCleaner extends Logging {
return
}
- logDebug(s"+++ Cleaning closure $func (${func.getClass.getName}) +++")
-
- // A list of classes that represents closures enclosed in the given one
- val innerClasses = getInnerClosureClasses(func)
-
- // A list of enclosing objects and their respective classes, from innermost to outermost
- // An outer object at a given index is of type outer class at the same index
- val (outerClasses, outerObjects) = getOuterClassesAndObjects(func)
-
- // For logging purposes only
- val declaredFields = func.getClass.getDeclaredFields
- val declaredMethods = func.getClass.getDeclaredMethods
-
- if (log.isDebugEnabled) {
- logDebug(" + declared fields: " + declaredFields.size)
- declaredFields.foreach { f => logDebug(" " + f) }
- logDebug(" + declared methods: " + declaredMethods.size)
- declaredMethods.foreach { m => logDebug(" " + m) }
- logDebug(" + inner classes: " + innerClasses.size)
- innerClasses.foreach { c => logDebug(" " + c.getName) }
- logDebug(" + outer classes: " + outerClasses.size)
- outerClasses.foreach { c => logDebug(" " + c.getName) }
- logDebug(" + outer objects: " + outerObjects.size)
- outerObjects.foreach { o => logDebug(" " + o) }
- }
+ if (lambdaFunc.isEmpty) {
+ logDebug(s"+++ Cleaning closure $func (${func.getClass.getName}) +++")
+
+ // A list of classes that represents closures enclosed in the given one
+ val innerClasses = getInnerClosureClasses(func)
+
+ // A list of enclosing objects and their respective classes, from innermost to outermost
+ // An outer object at a given index is of type outer class at the same index
+ val (outerClasses, outerObjects) = getOuterClassesAndObjects(func)
+
+ // For logging purposes only
+ val declaredFields = func.getClass.getDeclaredFields
+ val declaredMethods = func.getClass.getDeclaredMethods
+
+ if (log.isDebugEnabled) {
+ logDebug(s" + declared fields: ${declaredFields.size}")
+ declaredFields.foreach { f => logDebug(s" $f") }
+ logDebug(s" + declared methods: ${declaredMethods.size}")
+ declaredMethods.foreach { m => logDebug(s" $m") }
+ logDebug(s" + inner classes: ${innerClasses.size}")
+ innerClasses.foreach { c => logDebug(s" ${c.getName}") }
+ logDebug(s" + outer classes: ${outerClasses.size}" )
+ outerClasses.foreach { c => logDebug(s" ${c.getName}") }
+ logDebug(s" + outer objects: ${outerObjects.size}")
+ outerObjects.foreach { o => logDebug(s" $o") }
+ }
- // Fail fast if we detect return statements in closures
- getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0)
-
- // If accessed fields is not populated yet, we assume that
- // the closure we are trying to clean is the starting one
- if (accessedFields.isEmpty) {
- logDebug(s" + populating accessed fields because this is the starting closure")
- // Initialize accessed fields with the outer classes first
- // This step is needed to associate the fields to the correct classes later
- initAccessedFields(accessedFields, outerClasses)
-
- // Populate accessed fields by visiting all fields and methods accessed by this and
- // all of its inner closures. If transitive cleaning is enabled, this may recursively
- // visits methods that belong to other classes in search of transitively referenced fields.
- for (cls <- func.getClass :: innerClasses) {
- getClassReader(cls).accept(new FieldAccessFinder(accessedFields, cleanTransitively), 0)
+ // Fail fast if we detect return statements in closures
+ getClassReader(func.getClass).accept(new ReturnStatementFinder(), 0)
+
+ // If accessed fields is not populated yet, we assume that
+ // the closure we are trying to clean is the starting one
+ if (accessedFields.isEmpty) {
+ logDebug(" + populating accessed fields because this is the starting closure")
+ // Initialize accessed fields with the outer classes first
+ // This step is needed to associate the fields to the correct classes later
+ initAccessedFields(accessedFields, outerClasses)
+
+ // Populate accessed fields by visiting all fields and methods accessed by this and
+ // all of its inner closures. If transitive cleaning is enabled, this may recursively
+ // visits methods that belong to other classes in search of transitively referenced fields.
+ for (cls <- func.getClass :: innerClasses) {
+ getClassReader(cls).accept(new FieldAccessFinder(accessedFields, cleanTransitively), 0)
+ }
}
- }
- logDebug(s" + fields accessed by starting closure: " + accessedFields.size)
- accessedFields.foreach { f => logDebug(" " + f) }
-
- // List of outer (class, object) pairs, ordered from outermost to innermost
- // Note that all outer objects but the outermost one (first one in this list) must be closures
- var outerPairs: List[(Class[_], AnyRef)] = (outerClasses zip outerObjects).reverse
- var parent: AnyRef = null
- if (outerPairs.size > 0) {
- val (outermostClass, outermostObject) = outerPairs.head
- if (isClosure(outermostClass)) {
- logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}")
- } else if (outermostClass.getName.startsWith("$line")) {
- // SPARK-14558: if the outermost object is a REPL line object, we should clone and clean it
- // as it may carray a lot of unnecessary information, e.g. hadoop conf, spark conf, etc.
- logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}")
+ logDebug(s" + fields accessed by starting closure: " + accessedFields.size)
+ accessedFields.foreach { f => logDebug(" " + f) }
+
+ // List of outer (class, object) pairs, ordered from outermost to innermost
+ // Note that all outer objects but the outermost one (first one in this list) must be closures
+ var outerPairs: List[(Class[_], AnyRef)] = outerClasses.zip(outerObjects).reverse
+ var parent: AnyRef = null
+ if (outerPairs.nonEmpty) {
+ val (outermostClass, outermostObject) = outerPairs.head
+ if (isClosure(outermostClass)) {
+ logDebug(s" + outermost object is a closure, so we clone it: ${outerPairs.head}")
+ } else if (outermostClass.getName.startsWith("$line")) {
+ // SPARK-14558: if the outermost object is a REPL line object, we should clone
+ // and clean it as it may carray a lot of unnecessary information,
+ // e.g. hadoop conf, spark conf, etc.
+ logDebug(s" + outermost object is a REPL line object, so we clone it: ${outerPairs.head}")
+ } else {
+ // The closure is ultimately nested inside a class; keep the object of that
+ // class without cloning it since we don't want to clone the user's objects.
+ // Note that we still need to keep around the outermost object itself because
+ // we need it to clone its child closure later (see below).
+ logDebug(" + outermost object is not a closure or REPL line object," +
+ "so do not clone it: " + outerPairs.head)
+ parent = outermostObject // e.g. SparkContext
+ outerPairs = outerPairs.tail
+ }
} else {
- // The closure is ultimately nested inside a class; keep the object of that
- // class without cloning it since we don't want to clone the user's objects.
- // Note that we still need to keep around the outermost object itself because
- // we need it to clone its child closure later (see below).
- logDebug(" + outermost object is not a closure or REPL line object, so do not clone it: " +
- outerPairs.head)
- parent = outermostObject // e.g. SparkContext
- outerPairs = outerPairs.tail
+ logDebug(" + there are no enclosing objects!")
}
- } else {
- logDebug(" + there are no enclosing objects!")
- }
- // Clone the closure objects themselves, nulling out any fields that are not
- // used in the closure we're working on or any of its inner closures.
- for ((cls, obj) <- outerPairs) {
- logDebug(s" + cloning the object $obj of class ${cls.getName}")
- // We null out these unused references by cloning each object and then filling in all
- // required fields from the original object. We need the parent here because the Java
- // language specification requires the first constructor parameter of any closure to be
- // its enclosing object.
- val clone = cloneAndSetFields(parent, obj, cls, accessedFields)
-
- // If transitive cleaning is enabled, we recursively clean any enclosing closure using
- // the already populated accessed fields map of the starting closure
- if (cleanTransitively && isClosure(clone.getClass)) {
- logDebug(s" + cleaning cloned closure $clone recursively (${cls.getName})")
- // No need to check serializable here for the outer closures because we're
- // only interested in the serializability of the starting closure
- clean(clone, checkSerializable = false, cleanTransitively, accessedFields)
+ // Clone the closure objects themselves, nulling out any fields that are not
+ // used in the closure we're working on or any of its inner closures.
+ for ((cls, obj) <- outerPairs) {
+ logDebug(s" + cloning the object $obj of class ${cls.getName}")
+ // We null out these unused references by cloning each object and then filling in all
+ // required fields from the original object. We need the parent here because the Java
+ // language specification requires the first constructor parameter of any closure to be
+ // its enclosing object.
+ val clone = cloneAndSetFields(parent, obj, cls, accessedFields)
+
+ // If transitive cleaning is enabled, we recursively clean any enclosing closure using
+ // the already populated accessed fields map of the starting closure
+ if (cleanTransitively && isClosure(clone.getClass)) {
+ logDebug(s" + cleaning cloned closure $clone recursively (${cls.getName})")
+ // No need to check serializable here for the outer closures because we're
+ // only interested in the serializability of the starting closure
+ clean(clone, checkSerializable = false, cleanTransitively, accessedFields)
+ }
+ parent = clone
}
- parent = clone
- }
- // Update the parent pointer ($outer) of this closure
- if (parent != null) {
- val field = func.getClass.getDeclaredField("$outer")
- field.setAccessible(true)
- // If the starting closure doesn't actually need our enclosing object, then just null it out
- if (accessedFields.contains(func.getClass) &&
- !accessedFields(func.getClass).contains("$outer")) {
- logDebug(s" + the starting closure doesn't actually need $parent, so we null it out")
- field.set(func, null)
- } else {
- // Update this closure's parent pointer to point to our enclosing object,
- // which could either be a cloned closure or the original user object
- field.set(func, parent)
+ // Update the parent pointer ($outer) of this closure
+ if (parent != null) {
+ val field = func.getClass.getDeclaredField("$outer")
+ field.setAccessible(true)
+ // If the starting closure doesn't actually need our enclosing object, then just null it out
+ if (accessedFields.contains(func.getClass) &&
+ !accessedFields(func.getClass).contains("$outer")) {
+ logDebug(s" + the starting closure doesn't actually need $parent, so we null it out")
+ field.set(func, null)
+ } else {
+ // Update this closure's parent pointer to point to our enclosing object,
+ // which could either be a cloned closure or the original user object
+ field.set(func, parent)
+ }
}
- }
- logDebug(s" +++ closure $func (${func.getClass.getName}) is now cleaned +++")
+ logDebug(s" +++ closure $func (${func.getClass.getName}) is now cleaned +++")
+ } else {
+ logDebug(s"Cleaning lambda: ${lambdaFunc.get.getImplMethodName}")
+
+ // scalastyle:off classforname
+ val captClass = Class.forName(lambdaFunc.get.getCapturingClass.replace('/', '.'),
+ false, Thread.currentThread.getContextClassLoader)
+ // scalastyle:on classforname
+ // Fail fast if we detect return statements in closures
+ getClassReader(captClass)
+ .accept(new ReturnStatementFinder(Some(lambdaFunc.get.getImplMethodName)), 0)
+ logDebug(s" +++ Lambda closure (${lambdaFunc.get.getImplMethodName}) is now cleaned +++")
+ }
if (checkSerializable) {
ensureSerializable(func)
@@ -366,20 +424,30 @@ private[spark] object ClosureCleaner extends Logging {
private[spark] class ReturnStatementInClosureException
extends SparkException("Return statements aren't allowed in Spark closures")
-private class ReturnStatementFinder extends ClassVisitor(ASM5) {
+private class ReturnStatementFinder(targetMethodName: Option[String] = None)
+ extends ClassVisitor(ASM6) {
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
+
// $anonfun$ covers Java 8 lambdas
if (name.contains("apply") || name.contains("$anonfun$")) {
- new MethodVisitor(ASM5) {
+ // A method with suffix "$adapted" will be generated in cases like
+ // { _:Int => return; Seq()} but not { _:Int => return; true}
+ // closure passed is $anonfun$t$1$adapted while actual code resides in $anonfun$s$1
+ // visitor will see only $anonfun$s$1$adapted, so we remove the suffix, see
+ // https://github.com/scala/scala-dev/issues/109
+ val isTargetMethod = targetMethodName.isEmpty ||
+ name == targetMethodName.get || name == targetMethodName.get.stripSuffix("$adapted")
+
+ new MethodVisitor(ASM6) {
override def visitTypeInsn(op: Int, tp: String) {
- if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl")) {
+ if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl") && isTargetMethod) {
throw new ReturnStatementInClosureException
}
}
}
} else {
- new MethodVisitor(ASM5) {}
+ new MethodVisitor(ASM6) {}
}
}
}
@@ -403,7 +471,7 @@ private[util] class FieldAccessFinder(
findTransitively: Boolean,
specificMethod: Option[MethodIdentifier[_]] = None,
visitedMethods: Set[MethodIdentifier[_]] = Set.empty)
- extends ClassVisitor(ASM5) {
+ extends ClassVisitor(ASM6) {
override def visitMethod(
access: Int,
@@ -418,7 +486,7 @@ private[util] class FieldAccessFinder(
return null
}
- new MethodVisitor(ASM5) {
+ new MethodVisitor(ASM6) {
override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) {
if (op == GETFIELD) {
for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) {
@@ -458,7 +526,7 @@ private[util] class FieldAccessFinder(
}
}
-private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM5) {
+private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM6) {
var myName: String = null
// TODO: Recursively find inner closures that we indirectly reference, e.g.
@@ -473,7 +541,7 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM
override def visitMethod(access: Int, name: String, desc: String,
sig: String, exceptions: Array[String]): MethodVisitor = {
- new MethodVisitor(ASM5) {
+ new MethodVisitor(ASM6) {
override def visitMethodInsn(
op: Int, owner: String, name: String, desc: String, itf: Boolean) {
val argTypes = Type.getArgumentTypes(desc)
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 50c6461373dee..0cd8612b8fd1c 100644
--- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
+++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala
@@ -31,6 +31,7 @@ import org.json4s.jackson.JsonMethods._
import org.apache.spark._
import org.apache.spark.executor._
+import org.apache.spark.metrics.ExecutorMetricType
import org.apache.spark.rdd.RDDOperationScope
import org.apache.spark.scheduler._
import org.apache.spark.scheduler.cluster.ExecutorInfo
@@ -98,6 +99,8 @@ private[spark] object JsonProtocol {
logStartToJson(logStart)
case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
executorMetricsUpdateToJson(metricsUpdate)
+ case stageExecutorMetrics: SparkListenerStageExecutorMetrics =>
+ stageExecutorMetricsToJson(stageExecutorMetrics)
case blockUpdate: SparkListenerBlockUpdated =>
blockUpdateToJson(blockUpdate)
case _ => parse(mapper.writeValueAsString(event))
@@ -236,6 +239,7 @@ private[spark] object JsonProtocol {
def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = {
val execId = metricsUpdate.execId
val accumUpdates = metricsUpdate.accumUpdates
+ val executorMetrics = metricsUpdate.executorUpdates.map(executorMetricsToJson(_))
("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.metricsUpdate) ~
("Executor ID" -> execId) ~
("Metrics Updated" -> accumUpdates.map { case (taskId, stageId, stageAttemptId, updates) =>
@@ -243,7 +247,16 @@ private[spark] object JsonProtocol {
("Stage ID" -> stageId) ~
("Stage Attempt ID" -> stageAttemptId) ~
("Accumulator Updates" -> JArray(updates.map(accumulableInfoToJson).toList))
- })
+ }) ~
+ ("Executor Metrics Updated" -> executorMetrics)
+ }
+
+ def stageExecutorMetricsToJson(metrics: SparkListenerStageExecutorMetrics): JValue = {
+ ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.stageExecutorMetrics) ~
+ ("Executor ID" -> metrics.execId) ~
+ ("Stage ID" -> metrics.stageId) ~
+ ("Stage Attempt ID" -> metrics.stageAttemptId) ~
+ ("Executor Metrics" -> executorMetricsToJson(metrics.executorMetrics))
}
def blockUpdateToJson(blockUpdate: SparkListenerBlockUpdated): JValue = {
@@ -379,6 +392,14 @@ private[spark] object JsonProtocol {
("Updated Blocks" -> updatedBlocks)
}
+ /** Convert executor metrics to JSON. */
+ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = {
+ val metrics = ExecutorMetricType.values.map{ metricType =>
+ JField(metricType.name, executorMetrics.getMetricValue(metricType))
+ }
+ JObject(metrics: _*)
+ }
+
def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = {
val reason = Utils.getFormattedClassName(taskEndReason)
val json: JObject = taskEndReason match {
@@ -531,6 +552,7 @@ private[spark] object JsonProtocol {
val executorRemoved = Utils.getFormattedClassName(SparkListenerExecutorRemoved)
val logStart = Utils.getFormattedClassName(SparkListenerLogStart)
val metricsUpdate = Utils.getFormattedClassName(SparkListenerExecutorMetricsUpdate)
+ val stageExecutorMetrics = Utils.getFormattedClassName(SparkListenerStageExecutorMetrics)
val blockUpdate = Utils.getFormattedClassName(SparkListenerBlockUpdated)
}
@@ -555,6 +577,7 @@ private[spark] object JsonProtocol {
case `executorRemoved` => executorRemovedFromJson(json)
case `logStart` => logStartFromJson(json)
case `metricsUpdate` => executorMetricsUpdateFromJson(json)
+ case `stageExecutorMetrics` => stageExecutorMetricsFromJson(json)
case `blockUpdate` => blockUpdateFromJson(json)
case other => mapper.readValue(compact(render(json)), Utils.classForName(other))
.asInstanceOf[SparkListenerEvent]
@@ -585,6 +608,15 @@ private[spark] object JsonProtocol {
SparkListenerTaskGettingResult(taskInfo)
}
+ /** Extract the executor metrics from JSON. */
+ def executorMetricsFromJson(json: JValue): ExecutorMetrics = {
+ val metrics =
+ ExecutorMetricType.values.map { metric =>
+ metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L)
+ }.toMap
+ new ExecutorMetrics(metrics)
+ }
+
def taskEndFromJson(json: JValue): SparkListenerTaskEnd = {
val stageId = (json \ "Stage ID").extract[Int]
val stageAttemptId =
@@ -691,7 +723,18 @@ private[spark] object JsonProtocol {
(json \ "Accumulator Updates").extract[List[JValue]].map(accumulableInfoFromJson)
(taskId, stageId, stageAttemptId, updates)
}
- SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates)
+ val executorUpdates = jsonOption(json \ "Executor Metrics Updated").map {
+ executorUpdate => executorMetricsFromJson(executorUpdate)
+ }
+ SparkListenerExecutorMetricsUpdate(execInfo, accumUpdates, executorUpdates)
+ }
+
+ def stageExecutorMetricsFromJson(json: JValue): SparkListenerStageExecutorMetrics = {
+ val execId = (json \ "Executor ID").extract[String]
+ val stageId = (json \ "Stage ID").extract[Int]
+ val stageAttemptId = (json \ "Stage Attempt ID").extract[Int]
+ val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics")
+ SparkListenerStageExecutorMetrics(execId, stageId, stageAttemptId, executorMetrics)
}
def blockUpdateFromJson(json: JValue): SparkListenerBlockUpdated = {
diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
index d4474a90b26f1..a8f10684d5a2c 100644
--- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
+++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala
@@ -61,7 +61,7 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging {
}
/**
- * This can be overriden by subclasses if there is any extra cleanup to do when removing a
+ * This can be overridden by subclasses if there is any extra cleanup to do when removing a
* listener. In particular AsyncEventQueues can clean up queues in the LiveListenerBus.
*/
def removeListenerOnError(listener: L): Unit = {
diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
index 0f08a2b0ad895..cb0c20541d0d7 100644
--- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala
@@ -19,8 +19,12 @@ package org.apache.spark.util
import java.util.concurrent._
+import scala.collection.TraversableLike
+import scala.collection.generic.CanBuildFrom
+import scala.language.higherKinds
+
import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder}
-import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor}
+import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future}
import scala.concurrent.duration.{Duration, FiniteDuration}
import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread}
import scala.util.control.NonFatal
@@ -254,4 +258,38 @@ private[spark] object ThreadUtils {
executor.shutdownNow()
}
}
+
+ /**
+ * Transforms input collection by applying the given function to each element in parallel fashion.
+ * Comparing to the map() method of Scala parallel collections, this method can be interrupted
+ * at any time. This is useful on canceling of task execution, for example.
+ *
+ * @param in - the input collection which should be transformed in parallel.
+ * @param prefix - the prefix assigned to the underlying thread pool.
+ * @param maxThreads - maximum number of thread can be created during execution.
+ * @param f - the lambda function will be applied to each element of `in`.
+ * @tparam I - the type of elements in the input collection.
+ * @tparam O - the type of elements in resulted collection.
+ * @return new collection in which each element was given from the input collection `in` by
+ * applying the lambda function `f`.
+ */
+ def parmap[I, O, Col[X] <: TraversableLike[X, Col[X]]]
+ (in: Col[I], prefix: String, maxThreads: Int)
+ (f: I => O)
+ (implicit
+ cbf: CanBuildFrom[Col[I], Future[O], Col[Future[O]]], // For in.map
+ cbf2: CanBuildFrom[Col[Future[O]], O, Col[O]] // for Future.sequence
+ ): Col[O] = {
+ val pool = newForkJoinPool(prefix, maxThreads)
+ try {
+ implicit val ec = ExecutionContext.fromExecutor(pool)
+
+ val futures = in.map(x => Future(f(x)))
+ val futureSeq = Future.sequence(futures)
+
+ awaitResult(futureSeq, Duration.Inf)
+ } finally {
+ pool.shutdownNow()
+ }
+ }
}
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 a6fd3637663e8..14f68cd6f3509 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -19,7 +19,6 @@ package org.apache.spark.util
import java.io._
import java.lang.{Byte => JByte}
-import java.lang.InternalError
import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo}
import java.lang.reflect.InvocationTargetException
import java.math.{MathContext, RoundingMode}
@@ -60,7 +59,7 @@ import org.slf4j.Logger
import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.internal.Logging
+import org.apache.spark.internal.{config, Logging}
import org.apache.spark.internal.config._
import org.apache.spark.launcher.SparkLauncher
import org.apache.spark.network.util.JavaUtils
@@ -83,6 +82,7 @@ private[spark] object Utils extends Logging {
val random = new Random()
private val sparkUncaughtExceptionHandler = new SparkUncaughtExceptionHandler
+ @volatile private var cachedLocalDir: String = ""
/**
* Define a default value for driver memory here since this value is referenced across the code
@@ -462,7 +462,15 @@ private[spark] object Utils extends Logging {
if (useCache && fetchCacheEnabled) {
val cachedFileName = s"${url.hashCode}${timestamp}_cache"
val lockFileName = s"${url.hashCode}${timestamp}_lock"
- val localDir = new File(getLocalDir(conf))
+ // Set the cachedLocalDir for the first time and re-use it later
+ if (cachedLocalDir.isEmpty) {
+ this.synchronized {
+ if (cachedLocalDir.isEmpty) {
+ cachedLocalDir = getLocalDir(conf)
+ }
+ }
+ }
+ val localDir = new File(cachedLocalDir)
val lockFile = new File(localDir, lockFileName)
val lockFileChannel = new RandomAccessFile(lockFile, "rw").getChannel()
// Only one executor entry.
@@ -767,13 +775,17 @@ private[spark] object Utils extends Logging {
* - Otherwise, this will return java.io.tmpdir.
*
* Some of these configuration options might be lists of multiple paths, but this method will
- * always return a single directory.
+ * always return a single directory. The return directory is chosen randomly from the array
+ * of directories it gets from getOrCreateLocalRootDirs.
*/
def getLocalDir(conf: SparkConf): String = {
- getOrCreateLocalRootDirs(conf).headOption.getOrElse {
+ val localRootDirs = getOrCreateLocalRootDirs(conf)
+ if (localRootDirs.isEmpty) {
val configuredLocalDirs = getConfiguredLocalDirs(conf)
throw new IOException(
s"Failed to get a temp directory under [${configuredLocalDirs.mkString(",")}].")
+ } else {
+ localRootDirs(scala.util.Random.nextInt(localRootDirs.length))
}
}
@@ -809,13 +821,13 @@ private[spark] object Utils extends Logging {
* logic of locating the local directories according to deployment mode.
*/
def getConfiguredLocalDirs(conf: SparkConf): Array[String] = {
- val shuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false)
+ val shuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED)
if (isRunningInYarnContainer(conf)) {
// If we are in yarn mode, systems can have different disk layouts so we must set it
// to what Yarn on this system said was available. Note this assumes that Yarn has
// created the directories already, and that they are secured so that only the
// user has access to them.
- getYarnLocalDirs(conf).split(",")
+ randomizeInPlace(getYarnLocalDirs(conf).split(","))
} else if (conf.getenv("SPARK_EXECUTOR_DIRS") != null) {
conf.getenv("SPARK_EXECUTOR_DIRS").split(File.pathSeparator)
} else if (conf.getenv("SPARK_LOCAL_DIRS") != null) {
@@ -1374,7 +1386,7 @@ private[spark] object Utils extends Logging {
originalThrowable = cause
try {
logError("Aborting task", originalThrowable)
- TaskContext.get().asInstanceOf[TaskContextImpl].markTaskFailed(originalThrowable)
+ TaskContext.get().markTaskFailed(originalThrowable)
catchBlock
} catch {
case t: Throwable =>
@@ -1396,13 +1408,14 @@ private[spark] object Utils extends Logging {
}
}
+ // A regular expression to match classes of the internal Spark API's
+ // that we want to skip when finding the call site of a method.
+ private val SPARK_CORE_CLASS_REGEX =
+ """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r
+ private val SPARK_SQL_CLASS_REGEX = """^org\.apache\.spark\.sql.*""".r
+
/** Default filtering function for finding call sites using `getCallSite`. */
private def sparkInternalExclusionFunction(className: String): Boolean = {
- // A regular expression to match classes of the internal Spark API's
- // that we want to skip when finding the call site of a method.
- val SPARK_CORE_CLASS_REGEX =
- """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r
- val SPARK_SQL_CLASS_REGEX = """^org\.apache\.spark\.sql.*""".r
val SCALA_CORE_CLASS_PREFIX = "scala"
val isSparkClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined ||
SPARK_SQL_CLASS_REGEX.findFirstIn(className).isDefined
@@ -2038,6 +2051,30 @@ private[spark] object Utils extends Logging {
}
}
+ /**
+ * Implements the same logic as JDK `java.lang.String#trim` by removing leading and trailing
+ * non-printable characters less or equal to '\u0020' (SPACE) but preserves natural line
+ * delimiters according to [[java.util.Properties]] load method. The natural line delimiters are
+ * removed by JDK during load. Therefore any remaining ones have been specifically provided and
+ * escaped by the user, and must not be ignored
+ *
+ * @param str
+ * @return the trimmed value of str
+ */
+ private[util] def trimExceptCRLF(str: String): String = {
+ val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch =>
+ ch > ' ' || ch == '\r' || ch == '\n'
+ }
+
+ val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter)
+ val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter)
+ if (firstPos >= 0 && lastPos >= 0) {
+ str.substring(firstPos, lastPos + 1)
+ } else {
+ ""
+ }
+ }
+
/** Load properties present in the given file. */
def getPropertiesFromFile(filename: String): Map[String, String] = {
val file = new File(filename)
@@ -2048,8 +2085,10 @@ private[spark] object Utils extends Logging {
try {
val properties = new Properties()
properties.load(inReader)
- properties.stringPropertyNames().asScala.map(
- k => (k, properties.getProperty(k).trim)).toMap
+ properties.stringPropertyNames().asScala
+ .map { k => (k, trimExceptCRLF(properties.getProperty(k))) }
+ .toMap
+
} catch {
case e: IOException =>
throw new SparkException(s"Failed when loading Spark properties from $filename", e)
@@ -2781,6 +2820,36 @@ private[spark] object Utils extends Logging {
}
}
}
+
+ /**
+ * Regular expression matching full width characters.
+ *
+ * Looked at all the 0x0000-0xFFFF characters (unicode) and showed them under Xshell.
+ * Found all the full width characters, then get the regular expression.
+ */
+ private val fullWidthRegex = ("""[""" +
+ // scalastyle:off nonascii
+ """\u1100-\u115F""" +
+ """\u2E80-\uA4CF""" +
+ """\uAC00-\uD7A3""" +
+ """\uF900-\uFAFF""" +
+ """\uFE10-\uFE19""" +
+ """\uFE30-\uFE6F""" +
+ """\uFF00-\uFF60""" +
+ """\uFFE0-\uFFE6""" +
+ // scalastyle:on nonascii
+ """]""").r
+
+ /**
+ * Return the number of half widths in a given string. Note that a full width character
+ * occupies two half widths.
+ *
+ * For a string consisting of 1 million characters, the execution of this method requires
+ * about 50ms.
+ */
+ def stringHalfWidth(str: String): Int = {
+ if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size
+ }
}
private[util] object CallerContext extends Logging {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
index 5c6dd45ec58e3..19ff109b673e1 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala
@@ -80,7 +80,10 @@ class ExternalAppendOnlyMap[K, V, C](
this(createCombiner, mergeValue, mergeCombiners, serializer, blockManager, TaskContext.get())
}
- @volatile private var currentMap = new SizeTrackingAppendOnlyMap[K, C]
+ /**
+ * Exposed for testing
+ */
+ @volatile private[collection] var currentMap = new SizeTrackingAppendOnlyMap[K, C]
private val spilledMaps = new ArrayBuffer[DiskMapIterator]
private val sparkConf = SparkEnv.get.conf
private val diskBlockManager = blockManager.diskBlockManager
@@ -267,7 +270,7 @@ class ExternalAppendOnlyMap[K, V, C](
*/
def destructiveIterator(inMemoryIterator: Iterator[(K, C)]): Iterator[(K, C)] = {
readingIterator = new SpillableIterator(inMemoryIterator)
- readingIterator
+ readingIterator.toCompletionIterator
}
/**
@@ -280,8 +283,7 @@ class ExternalAppendOnlyMap[K, V, C](
"ExternalAppendOnlyMap.iterator is destructive and should only be called once.")
}
if (spilledMaps.isEmpty) {
- CompletionIterator[(K, C), Iterator[(K, C)]](
- destructiveIterator(currentMap.iterator), freeCurrentMap())
+ destructiveIterator(currentMap.iterator)
} else {
new ExternalIterator()
}
@@ -305,8 +307,8 @@ class ExternalAppendOnlyMap[K, V, C](
// Input streams are derived both from the in-memory map and spilled maps on disk
// The in-memory map is sorted in place, while the spilled maps are already in sorted order
- private val sortedMap = CompletionIterator[(K, C), Iterator[(K, C)]](destructiveIterator(
- currentMap.destructiveSortedIterator(keyComparator)), freeCurrentMap())
+ private val sortedMap = destructiveIterator(
+ currentMap.destructiveSortedIterator(keyComparator))
private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered)
inputStreams.foreach { it =>
@@ -565,16 +567,14 @@ class ExternalAppendOnlyMap[K, V, C](
}
}
- context.addTaskCompletionListener(context => cleanup())
+ context.addTaskCompletionListener[Unit](context => cleanup())
}
- private[this] class SpillableIterator(var upstream: Iterator[(K, C)])
+ private class SpillableIterator(var upstream: Iterator[(K, C)])
extends Iterator[(K, C)] {
private val SPILL_LOCK = new Object()
- private var nextUpstream: Iterator[(K, C)] = null
-
private var cur: (K, C) = readNext()
private var hasSpilled: Boolean = false
@@ -585,17 +585,24 @@ class ExternalAppendOnlyMap[K, V, C](
} else {
logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " +
s"it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory")
- nextUpstream = spillMemoryIteratorToDisk(upstream)
+ val nextUpstream = spillMemoryIteratorToDisk(upstream)
+ assert(!upstream.hasNext)
hasSpilled = true
+ upstream = nextUpstream
true
}
}
+ private def destroy(): Unit = {
+ freeCurrentMap()
+ upstream = Iterator.empty
+ }
+
+ def toCompletionIterator: CompletionIterator[(K, C), SpillableIterator] = {
+ CompletionIterator[(K, C), SpillableIterator](this, this.destroy)
+ }
+
def readNext(): (K, C) = SPILL_LOCK.synchronized {
- if (nextUpstream != null) {
- upstream = nextUpstream
- nextUpstream = null
- }
if (upstream.hasNext) {
upstream.next()
} else {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
index 176f84fa2a0d2..b159200d79222 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala
@@ -368,8 +368,8 @@ private[spark] class ExternalSorter[K, V, C](
val bufferedIters = iterators.filter(_.hasNext).map(_.buffered)
type Iter = BufferedIterator[Product2[K, C]]
val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] {
- // Use the reverse of comparator.compare because PriorityQueue dequeues the max
- override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1)
+ // Use the reverse order because PriorityQueue dequeues the max
+ override def compare(x: Iter, y: Iter): Int = comparator.compare(y.head._1, x.head._1)
})
heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true
new Iterator[Product2[K, C]] {
diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
index 60f6f537c1d54..8883e17bf3164 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala
@@ -28,9 +28,9 @@ import org.apache.spark.annotation.Private
* removed.
*
* The underlying implementation uses Scala compiler's specialization to generate optimized
- * storage for two primitive types (Long and Int). It is much faster than Java's standard HashSet
- * while incurring much less memory overhead. This can serve as building blocks for higher level
- * data structures such as an optimized HashMap.
+ * storage for four primitive types (Long, Int, Double, and Float). It is much faster than Java's
+ * standard HashSet while incurring much less memory overhead. This can serve as building blocks
+ * for higher level data structures such as an optimized HashMap.
*
* This OpenHashSet is designed to serve as building blocks for higher level data structures
* such as an optimized hash map. Compared with standard hash set implementations, this class
@@ -41,7 +41,7 @@ import org.apache.spark.annotation.Private
* to explore all spaces for each key (see http://en.wikipedia.org/wiki/Quadratic_probing).
*/
@Private
-class OpenHashSet[@specialized(Long, Int) T: ClassTag](
+class OpenHashSet[@specialized(Long, Int, Double, Float) T: ClassTag](
initialCapacity: Int,
loadFactor: Double)
extends Serializable {
@@ -77,6 +77,10 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag](
(new LongHasher).asInstanceOf[Hasher[T]]
} else if (mt == ClassTag.Int) {
(new IntHasher).asInstanceOf[Hasher[T]]
+ } else if (mt == ClassTag.Double) {
+ (new DoubleHasher).asInstanceOf[Hasher[T]]
+ } else if (mt == ClassTag.Float) {
+ (new FloatHasher).asInstanceOf[Hasher[T]]
} else {
new Hasher[T]
}
@@ -293,7 +297,7 @@ object OpenHashSet {
* A set of specialized hash function implementation to avoid boxing hash code computation
* in the specialized implementation of OpenHashSet.
*/
- sealed class Hasher[@specialized(Long, Int) T] extends Serializable {
+ sealed class Hasher[@specialized(Long, Int, Double, Float) T] extends Serializable {
def hash(o: T): Int = o.hashCode()
}
@@ -305,6 +309,17 @@ object OpenHashSet {
override def hash(o: Int): Int = o
}
+ class DoubleHasher extends Hasher[Double] {
+ override def hash(o: Double): Int = {
+ val bits = java.lang.Double.doubleToLongBits(o)
+ (bits ^ (bits >>> 32)).toInt
+ }
+ }
+
+ class FloatHasher extends Hasher[Float] {
+ override def hash(o: Float): Int = java.lang.Float.floatToIntBits(o)
+ }
+
private def grow1(newSize: Int) {}
private def move1(oldPos: Int, newPos: Int) { }
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
index 700ce56466c35..39f050f6ca5ad 100644
--- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala
@@ -17,17 +17,21 @@
package org.apache.spark.util.io
-import java.io.InputStream
+import java.io.{File, FileInputStream, InputStream}
import java.nio.ByteBuffer
-import java.nio.channels.WritableByteChannel
+import java.nio.channels.{FileChannel, WritableByteChannel}
+import java.nio.file.StandardOpenOption
+
+import scala.collection.mutable.ListBuffer
import com.google.common.primitives.UnsignedBytes
-import io.netty.buffer.{ByteBuf, Unpooled}
import org.apache.spark.SparkEnv
import org.apache.spark.internal.config
+import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
import org.apache.spark.network.util.ByteArrayWritableChannel
import org.apache.spark.storage.StorageUtils
+import org.apache.spark.util.Utils
/**
* Read-only byte buffer which is physically stored as multiple chunks rather than a single
@@ -81,10 +85,10 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
}
/**
- * Wrap this buffer to view it as a Netty ByteBuf.
+ * Wrap this in a custom "FileRegion" which allows us to transfer over 2 GB.
*/
- def toNetty: ByteBuf = {
- Unpooled.wrappedBuffer(chunks.length, getChunks(): _*)
+ def toNetty: ChunkedByteBufferFileRegion = {
+ new ChunkedByteBufferFileRegion(this, bufferWriteChunkSize)
}
/**
@@ -166,6 +170,38 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) {
}
+object ChunkedByteBuffer {
+ // TODO eliminate this method if we switch BlockManager to getting InputStreams
+ def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = {
+ data match {
+ case f: FileSegmentManagedBuffer =>
+ map(f.getFile, maxChunkSize, f.getOffset, f.getLength)
+ case other =>
+ new ChunkedByteBuffer(other.nioByteBuffer())
+ }
+ }
+
+ def map(file: File, maxChunkSize: Int): ChunkedByteBuffer = {
+ map(file, maxChunkSize, 0, file.length())
+ }
+
+ def map(file: File, maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = {
+ Utils.tryWithResource(FileChannel.open(file.toPath, StandardOpenOption.READ)) { channel =>
+ var remaining = length
+ var pos = offset
+ val chunks = new ListBuffer[ByteBuffer]()
+ while (remaining > 0) {
+ val chunkSize = math.min(remaining, maxChunkSize)
+ val chunk = channel.map(FileChannel.MapMode.READ_ONLY, pos, chunkSize)
+ pos += chunkSize
+ remaining -= chunkSize
+ chunks += chunk
+ }
+ new ChunkedByteBuffer(chunks.toArray)
+ }
+ }
+}
+
/**
* Reads data from a ChunkedByteBuffer.
*
diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala
new file mode 100644
index 0000000000000..9622d0ac05368
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala
@@ -0,0 +1,86 @@
+/*
+ * 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.util.io
+
+import java.nio.channels.WritableByteChannel
+
+import io.netty.channel.FileRegion
+import io.netty.util.AbstractReferenceCounted
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.network.util.AbstractFileRegion
+
+
+/**
+ * This exposes a ChunkedByteBuffer as a netty FileRegion, just to allow sending > 2gb in one netty
+ * message. This is because netty cannot send a ByteBuf > 2g, but it can send a large FileRegion,
+ * even though the data is not backed by a file.
+ */
+private[io] class ChunkedByteBufferFileRegion(
+ private val chunkedByteBuffer: ChunkedByteBuffer,
+ private val ioChunkSize: Int) extends AbstractFileRegion {
+
+ private var _transferred: Long = 0
+ // this duplicates the original chunks, so we're free to modify the position, limit, etc.
+ private val chunks = chunkedByteBuffer.getChunks()
+ private val size = chunks.foldLeft(0L) { _ + _.remaining() }
+
+ protected def deallocate: Unit = {}
+
+ override def count(): Long = size
+
+ // this is the "start position" of the overall Data in the backing file, not our current position
+ override def position(): Long = 0
+
+ override def transferred(): Long = _transferred
+
+ private var currentChunkIdx = 0
+
+ def transferTo(target: WritableByteChannel, position: Long): Long = {
+ assert(position == _transferred)
+ if (position == size) return 0L
+ var keepGoing = true
+ var written = 0L
+ var currentChunk = chunks(currentChunkIdx)
+ while (keepGoing) {
+ while (currentChunk.hasRemaining && keepGoing) {
+ val ioSize = Math.min(currentChunk.remaining(), ioChunkSize)
+ val originalLimit = currentChunk.limit()
+ currentChunk.limit(currentChunk.position() + ioSize)
+ val thisWriteSize = target.write(currentChunk)
+ currentChunk.limit(originalLimit)
+ written += thisWriteSize
+ if (thisWriteSize < ioSize) {
+ // the channel did not accept our entire write. We do *not* keep trying -- netty wants
+ // us to just stop, and report how much we've written.
+ keepGoing = false
+ }
+ }
+ if (keepGoing) {
+ // advance to the next chunk (if there are any more)
+ currentChunkIdx += 1
+ if (currentChunkIdx == chunks.size) {
+ keepGoing = false
+ } else {
+ currentChunk = chunks(currentChunkIdx)
+ }
+ }
+ }
+ _transferred += written
+ written
+ }
+}
diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
index d7d2d0b012bd3..a0664b30d6cc2 100644
--- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
+++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java
@@ -76,7 +76,7 @@ public void freeingPageSetsPageNumberToSpecialConstant() {
final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP);
final MemoryBlock dataPage = manager.allocatePage(256, c);
c.freePage(dataPage);
- Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.getPageNumber());
+ Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber);
}
@Test(expected = AssertionError.class)
diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
index 0d5c5ea7903e9..faa70f23b0ac6 100644
--- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
+++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java
@@ -233,6 +233,7 @@ public void writeEmptyIterator() throws Exception {
writer.write(Iterators.emptyIterator());
final Option mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
+ assertEquals(0, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile);
assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten());
@@ -252,6 +253,7 @@ public void writeWithoutSpilling() throws Exception {
writer.write(dataToWrite.iterator());
final Option mapStatus = writer.stop(true);
assertTrue(mapStatus.isDefined());
+ assertEquals(NUM_PARTITITONS, mapStatus.get().numberOfOutput());
assertTrue(mergedOutputFile.exists());
long sumOfPartitionSizes = 0;
diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
index 03cec8ed81b72..53a233f698c7a 100644
--- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
+++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java
@@ -379,7 +379,7 @@ public void iteratingOverDataPagesWithWastedSpace() throws Exception {
@Test
public void randomizedStressTest() {
- final int size = 65536;
+ final int size = 32768;
// Java arrays' hashCodes() aren't based on the arrays' contents, so we need to wrap arrays
// into ByteBuffers in order to use them as keys here.
final Map expected = new HashMap<>();
@@ -388,7 +388,7 @@ public void randomizedStressTest() {
// Fill the map to 90% full so that we can trigger probing
for (int i = 0; i < size * 0.9; i++) {
final byte[] key = getRandomByteArray(rand.nextInt(256) + 1);
- final byte[] value = getRandomByteArray(rand.nextInt(512) + 1);
+ final byte[] value = getRandomByteArray(rand.nextInt(256) + 1);
if (!expected.containsKey(ByteBuffer.wrap(key))) {
expected.put(ByteBuffer.wrap(key), value);
final BytesToBytesMap.Location loc = map.lookup(
diff --git a/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java b/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java
index 7e9cc70d8651f..0f489fb219010 100644
--- a/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java
+++ b/core/src/test/java/test/org/apache/spark/JavaSparkContextSuite.java
@@ -30,7 +30,7 @@
import org.apache.spark.*;
/**
- * Java apps can uses both Java-friendly JavaSparkContext and Scala SparkContext.
+ * Java apps can use both Java-friendly JavaSparkContext and Scala SparkContext.
*/
public class JavaSparkContextSuite implements Serializable {
diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
index 4fecf84db65a2..eea6f595efd2a 100644
--- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
index 4fecf84db65a2..7bc7f31be097b 100644
--- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134,
+ "lastUpdatedEpoch" : 0
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
new file mode 100644
index 0000000000000..9bf2086cc8e72
--- /dev/null
+++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json
@@ -0,0 +1,314 @@
+[ {
+ "id" : "driver",
+ "hostPort" : "node0033.grid.company.com:60749",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 0,
+ "maxTasks" : 0,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 1043437977,
+ "addTime" : "2018-04-19T23:55:05.107GMT",
+ "executorLogs" : { },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 1043437977,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 905801,
+ "JVMOffHeapMemory" : 205304696,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 905801,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 397602,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 629553808,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "7",
+ "hostPort" : "node6340.grid.company.com:5933",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:49.826GMT",
+ "executorLogs" : {
+ "stdout" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stdout?start=-4096",
+ "stderr" : "http://node6340.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000009/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "6",
+ "hostPort" : "node6644.grid.company.com:8445",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 0,
+ "totalTasks" : 0,
+ "totalDuration" : 0,
+ "totalGCTime" : 0,
+ "totalInputBytes" : 0,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 0,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:47.549GMT",
+ "executorLogs" : {
+ "stdout" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stdout?start=-4096",
+ "stderr" : "http://node6644.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000008/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "5",
+ "hostPort" : "node2477.grid.company.com:20123",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 9252,
+ "totalGCTime" : 920,
+ "totalInputBytes" : 36838295,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 355051,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:43.160GMT",
+ "executorLogs" : {
+ "stdout" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stdout?start=-4096",
+ "stderr" : "http://node2477.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000007/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ]
+}, {
+ "id" : "4",
+ "hostPort" : "node4243.grid.company.com:16084",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 3,
+ "totalTasks" : 3,
+ "totalDuration" : 15645,
+ "totalGCTime" : 405,
+ "totalInputBytes" : 87272855,
+ "totalShuffleRead" : 438675,
+ "totalShuffleWrite" : 26773039,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.278GMT",
+ "executorLogs" : {
+ "stdout" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096",
+ "stderr" : "http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 63104457,
+ "JVMOffHeapMemory" : 95657456,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 100853193,
+ "OnHeapExecutionMemory" : 37748736,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 126261,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 518613056,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "3",
+ "hostPort" : "node0998.grid.company.com:45265",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 14491,
+ "totalGCTime" : 342,
+ "totalInputBytes" : 50409514,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 31362123,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.088GMT",
+ "executorLogs" : {
+ "stdout" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096",
+ "stderr" : "http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 69535048,
+ "JVMOffHeapMemory" : 90709624,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 69535048,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 87796,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 726805712,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "2",
+ "hostPort" : "node4045.grid.company.com:29262",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 1,
+ "totalTasks" : 1,
+ "totalDuration" : 14113,
+ "totalGCTime" : 326,
+ "totalInputBytes" : 50423423,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 22950296,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:12.471GMT",
+ "executorLogs" : {
+ "stdout" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096",
+ "stderr" : "http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 58468944,
+ "JVMOffHeapMemory" : 91208368,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 58468944,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 87796,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 595946552,
+ "OffHeapStorageMemory" : 0
+ }
+}, {
+ "id" : "1",
+ "hostPort" : "node1404.grid.company.com:34043",
+ "isActive" : true,
+ "rddBlocks" : 0,
+ "memoryUsed" : 0,
+ "diskUsed" : 0,
+ "totalCores" : 1,
+ "maxTasks" : 1,
+ "activeTasks" : 0,
+ "failedTasks" : 0,
+ "completedTasks" : 3,
+ "totalTasks" : 3,
+ "totalDuration" : 15665,
+ "totalGCTime" : 471,
+ "totalInputBytes" : 98905018,
+ "totalShuffleRead" : 0,
+ "totalShuffleWrite" : 20594744,
+ "isBlacklisted" : false,
+ "maxMemory" : 956615884,
+ "addTime" : "2018-04-19T23:55:11.695GMT",
+ "executorLogs" : {
+ "stdout" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096",
+ "stderr" : "http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"
+ },
+ "memoryMetrics" : {
+ "usedOnHeapStorageMemory" : 0,
+ "usedOffHeapStorageMemory" : 0,
+ "totalOnHeapStorageMemory" : 956615884,
+ "totalOffHeapStorageMemory" : 0
+ },
+ "blacklistedInStages" : [ ],
+ "peakMemoryMetrics" : {
+ "OnHeapStorageMemory" : 47962185,
+ "JVMOffHeapMemory" : 100519936,
+ "OffHeapExecutionMemory" : 0,
+ "OnHeapUnifiedMemory" : 47962185,
+ "OnHeapExecutionMemory" : 0,
+ "OffHeapUnifiedMemory" : 0,
+ "DirectPoolMemory" : 98230,
+ "MappedPoolMemory" : 0,
+ "JVMHeapMemory" : 755008624,
+ "OffHeapStorageMemory" : 0
+ }
+} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
index 79950b0dc6486..9e1e65a358815 100644
--- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
@@ -28,19 +43,4 @@
"startTimeEpoch" : 1515492942372,
"endTimeEpoch" : 1515493477606
} ]
-}, {
- "id" : "app-20161116163331-0000",
- "name" : "Spark shell",
- "attempts" : [ {
- "startTime" : "2016-11-16T22:33:29.916GMT",
- "endTime" : "2016-11-16T22:33:40.587GMT",
- "lastUpdated" : "",
- "duration" : 10671,
- "sparkUser" : "jose",
- "completed" : true,
- "appSparkVersion" : "2.1.0-SNAPSHOT",
- "lastUpdatedEpoch" : 0,
- "startTimeEpoch" : 1479335609916,
- "endTimeEpoch" : 1479335620587
- } ]
} ]
diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
index 7d60977dcd4fe..28c6bf1b3e01e 100644
--- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
index dfbfd8aedcc23..f547b79f47e1a 100644
--- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
+++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json
@@ -1,4 +1,19 @@
[ {
+ "id" : "application_1506645932520_24630151",
+ "name" : "Spark shell",
+ "attempts" : [ {
+ "startTime" : "2018-04-19T23:54:42.734GMT",
+ "endTime" : "2018-04-19T23:56:29.134GMT",
+ "lastUpdated" : "",
+ "duration" : 106400,
+ "sparkUser" : "edlu",
+ "completed" : true,
+ "appSparkVersion" : "2.4.0-SNAPSHOT",
+ "lastUpdatedEpoch" : 0,
+ "startTimeEpoch" : 1524182082734,
+ "endTimeEpoch" : 1524182189134
+ } ]
+}, {
"id" : "application_1516285256255_0012",
"name" : "Spark shell",
"attempts" : [ {
@@ -101,4 +116,4 @@
"startTimeEpoch" : 1430917380880,
"endTimeEpoch" : 1430917380890
} ]
-} ]
\ No newline at end of file
+} ]
diff --git a/core/src/test/resources/spark-events/application_1506645932520_24630151 b/core/src/test/resources/spark-events/application_1506645932520_24630151
new file mode 100644
index 0000000000000..c48ed741c56e0
--- /dev/null
+++ b/core/src/test/resources/spark-events/application_1506645932520_24630151
@@ -0,0 +1,63 @@
+{"Event":"SparkListenerLogStart","Spark Version":"2.4.0-SNAPSHOT"}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"node0033.grid.company.com","Port":60749},"Maximum Memory":1043437977,"Timestamp":1524182105107,"Maximum Onheap Memory":1043437977,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_31/jre","Java Version":"1.8.0_31 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.jars.ivySettings":"/export/apps/spark/commonconf/ivysettings.xml","spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.driver.host":"node0033.grid.company.com","spark.dynamicAllocation.sustainedSchedulerBacklogTimeout":"5","spark.eventLog.enabled":"true","spark.ui.port":"0","spark.driver.port":"57705","spark.shuffle.service.enabled":"true","spark.ui.acls.enable":"true","spark.reducer.maxSizeInFlight":"48m","spark.yarn.queue":"spark_default","spark.repl.class.uri":"spark://node0033.grid.company.com:57705/classes","spark.jars":"","spark.yarn.historyServer.address":"clustersh01.grid.company.com:18080","spark.memoryOverhead.multiplier.percent":"10","spark.repl.class.outputDir":"/grid/a/mapred/tmp/spark-21b68b4b-c1db-460e-a228-b87545d870f1/repl-58778a76-04c1-434d-bfb7-9a9b83afe718","spark.dynamicAllocation.cachedExecutorIdleTimeout":"1200","spark.yarn.access.namenodes":"hdfs://clusternn02.grid.company.com:9000","spark.app.name":"Spark shell","spark.dynamicAllocation.schedulerBacklogTimeout":"5","spark.yarn.security.credentials.hive.enabled":"false","spark.yarn.am.cores":"1","spark.memoryOverhead.min":"384","spark.scheduler.mode":"FIFO","spark.driver.memory":"2G","spark.executor.instances":"4","spark.isolated.classloader.additional.classes.prefix":"com_company_","spark.logConf":"true","spark.ui.showConsoleProgress":"true","spark.user.priority.jars":"*********(redacted)","spark.isolated.classloader":"true","spark.sql.sources.schemaStringLengthThreshold":"40000","spark.yarn.secondary.jars":"spark-avro_2.11-3.2.0.21.jar,grid-topology-1.0.jar","spark.reducer.maxBlocksInFlightPerAddress":"100","spark.dynamicAllocation.maxExecutors":"900","spark.yarn.appMasterEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.executor.id":"driver","spark.yarn.am.memory":"2G","spark.driver.cores":"1","spark.search.packages":"com.company.dali:dali-data-spark,com.company.spark-common:spark-common","spark.min.mem.vore.ratio":"5","spark.sql.sources.partitionOverwriteMode":"DYNAMIC","spark.submit.deployMode":"client","spark.yarn.maxAppAttempts":"1","spark.master":"yarn","spark.default.packages":"com.company.dali:dali-data-spark:8.+?classifier=all,com.company.spark-common:spark-common_2.10:0.+?","spark.isolated.classloader.default.jar":"*dali-data-spark*","spark.authenticate":"true","spark.eventLog.usexattr":"true","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.memory":"2G","spark.home":"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51","spark.reducer.maxReqsInFlight":"10","spark.eventLog.dir":"hdfs://clusternn02.grid.company.com:9000/system/spark-history","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.isolated.classes":"org.apache.hadoop.hive.ql.io.CombineHiveInputFormat$CombineHiveInputSplit","spark.eventLog.compress":"true","spark.executor.cores":"1","spark.version":"2.1.0","spark.driver.appUIAddress":"http://node0033.grid.company.com:8364","spark.repl.local.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"clusterwp01.grid.company.com","spark.min.memory-gb.size":"10","spark.dynamicAllocation.minExecutors":"1","spark.dynamicAllocation.initialExecutors":"3","spark.expressionencoder.org.apache.avro.specific.SpecificRecord":"com.databricks.spark.avro.AvroEncoder$","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://clusterwp01.grid.company.com:8080/proxy/application_1506645932520_24630151","spark.executorEnv.LD_LIBRARY_PATH":"/export/apps/hadoop/latest/lib/native","spark.dynamicAllocation.executorIdleTimeout":"150","spark.shell.auto.node.labeling":"true","spark.yarn.dist.jars":"file:///export/home/edlu/spark-avro_2.11-3.2.0.21.jar,file:///export/apps/hadoop/site/lib/grid-topology-1.0.jar","spark.app.id":"application_1506645932520_24630151","spark.ui.view.acls":"*"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"*********(redacted)","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_31/jre/lib/amd64","user.dir":"*********(redacted)","java.library.path":"/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.31-b07","java.endorsed.dirs":"/usr/java/jdk1.8.0_31/jre/lib/endorsed","java.runtime.version":"1.8.0_31-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_31/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_31/jre/lib/resources.jar:/usr/java/jdk1.8.0_31/jre/lib/rt.jar:/usr/java/jdk1.8.0_31/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_31/jre/lib/jsse.jar:/usr/java/jdk1.8.0_31/jre/lib/jce.jar:/usr/java/jdk1.8.0_31/jre/lib/charsets.jar:/usr/java/jdk1.8.0_31/jre/lib/jfr.jar:/usr/java/jdk1.8.0_31/jre/classes","file.encoding":"UTF-8","user.timezone":"*********(redacted)","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"2.6.32-504.16.2.el6.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"*********(redacted)","sun.jnu.encoding":"UTF-8","user.language":"*********(redacted)","java.vendor.url":"*********(redacted)","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"*********(redacted)","user.name":"*********(redacted)","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master yarn --deploy-mode client --class org.apache.spark.repl.Main --name Spark shell --jars /export/home/edlu/spark-avro_2.11-3.2.0.21.jar,/export/apps/hadoop/site/lib/grid-topology-1.0.jar --num-executors 4 spark-shell","java.home":"/usr/java/jdk1.8.0_31/jre","java.version":"1.8.0_31","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-servlet-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/derby-10.12.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-reflect-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-rdbms-3.2.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-graphx_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/api-util-1.0.0-M20.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/base64-2.3.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-auth-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/validation-api-1.1.0.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-api-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/objenesis-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/conf/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpclient-4.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/kryo-shaded-3.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-library-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-net-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xz-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-annotations-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/activation-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arpack_combined_all-0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libthrift-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aircompressor-0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-jackson-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/asm-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-hive_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ivy-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/export/apps/hadoop/site/etc/hadoop/":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-java-1.1.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-format-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-all-4.1.17.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-ipc-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xmlenc-0.52.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jdo-api-3.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-client-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-runtime-3.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/pyrolite-4.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-catalyst_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-collections-3.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-api-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stream-2.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-format-2.3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-vector-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-web-proxy-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sketch_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-common-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hppc-0.7.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-sql_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/univocity-parsers-2.5.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-math3-3.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compiler-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-1.7.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/java-xmlbuilder-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.inject-1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-annotations-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/netty-3.9.9.Final.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/zookeeper-3.4.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guice-3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-compiler-2.11.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/eigenbase-properties-1.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/aopalliance-1.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-yarn_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/JavaEWAH-0.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsr305-1.3.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/libfb303-0.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.annotation-api-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-server-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-digester-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-jvm-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-framework-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/paranamer-2.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/janino-3.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-core-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-server-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-core-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsch-0.1.42.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-unsafe_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-codec-1.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jtransforms-2.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/lz4-java-1.4.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-core-3.2.10.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-exec-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/core-1.1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/leveldbjni-all-1.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-dbcp-1.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang3-3.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill-java-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jodd-core-3.5.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-pool-1.5.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/minlog-1.3.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/gson-2.2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/py4j-0.10.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-streaming_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-core-2.6.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/machinist_2.11-0.6.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/avro-1.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/snappy-0.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-app-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-graphite-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-core-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib-local_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/arrow-memory-0.8.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-guava-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-client-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xercesImpl-2.9.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-tags_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javolution-5.5.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/joda-time-2.9.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr-2.7.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-jobclient-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-lang-2.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/compress-lzf-1.0.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-crypto-1.0.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-core-1.9.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/curator-recipes-2.7.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/guava-14.0.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-core_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-sslengine-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-common_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-launcher_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/antlr4-runtime-4.7.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jetty-util-6.1.26.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jaxb-api-2.2.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-io-2.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-encoding-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/httpcore-4.4.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jackson-xc-1.9.13.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/protobuf-java-2.5.0.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-mllib_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-configuration-1.6.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-compress-1.4.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/orc-mapreduce-1.4.3-nohive.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/ST4-4.0.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/calcite-core-1.2.0-incubating.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-mapreduce-client-shuffle-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-repl_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/opencsv-2.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-logging-1.1.3.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-cli-1.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-client-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-yarn-common-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hadoop-hdfs-2.7.4.51.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/log4j-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-column-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/hive-metastore-1.2.1.spark2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/chill_2.11-0.8.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stringtemplate-3.2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/parquet-common-1.8.2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-network-shuffle_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/spark-kvstore_2.11-2.4.0-SNAPSHOT.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/stax-api-1.0-2.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jta-1.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/javassist-3.18.1-GA.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/commons-httpclient-3.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jets3t-0.9.4.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/apache-log4j-extras-1.2.17.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/metrics-json-3.1.5.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bcprov-jdk15on-1.58.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/oro-2.0.8.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/bonecp-0.8.0.RELEASE.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/jsp-api-2.1.jar":"System Classpath","/export/home/edlu/spark-2.4.0-SNAPSHOT-bin-2.7.4.51/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath"}}
+{"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"application_1506645932520_24630151","Timestamp":1524182082734,"User":"edlu"}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182111695,"Executor ID":"1","Executor Info":{"Host":"node1404.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stdout?start=-4096","stderr":"http://node1404.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000002/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"node1404.grid.company.com","Port":34043},"Maximum Memory":956615884,"Timestamp":1524182111795,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112088,"Executor ID":"3","Executor Info":{"Host":"node0998.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stdout?start=-4096","stderr":"http://node0998.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000005/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"node0998.grid.company.com","Port":45265},"Maximum Memory":956615884,"Timestamp":1524182112208,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112278,"Executor ID":"4","Executor Info":{"Host":"node4243.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stdout?start=-4096","stderr":"http://node4243.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000006/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"node4243.grid.company.com","Port":16084},"Maximum Memory":956615884,"Timestamp":1524182112408,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"SparkListenerExecutorAdded","Timestamp":1524182112471,"Executor ID":"2","Executor Info":{"Host":"node4045.grid.company.com","Total Cores":1,"Log Urls":{"stdout":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stdout?start=-4096","stderr":"http://node4045.grid.company.com:8042/node/containerlogs/container_e05_1523494505172_1552404_01_000004/edlu/stderr?start=-4096"}}}
+{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"node4045.grid.company.com","Port":29262},"Maximum Memory":956615884,"Timestamp":1524182112578,"Maximum Onheap Memory":956615884,"Maximum Offheap Memory":0}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":0,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line44.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line44.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line44.$read$$iw$$iw$$iw$$iw.(:59)\n$line44.$read$$iw$$iw$$iw.(:61)\n$line44.$read$$iw$$iw.(:63)\n$line44.$read$$iw.(:65)\n$line44.$read.(:67)\n$line44.$read$.(:71)\n$line44.$read$.()\n$line44.$eval$.$print$lzycompute(:7)\n$line44.$eval$.$print(:6)\n$line44.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `apps`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182125829}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":0,"time":1524182125832}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":1,"description":"createOrReplaceTempView at :40","details":"org.apache.spark.sql.Dataset.createOrReplaceTempView(Dataset.scala:3033)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line48.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line48.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line48.$read$$iw$$iw$$iw$$iw.(:59)\n$line48.$read$$iw$$iw$$iw.(:61)\n$line48.$read$$iw$$iw.(:63)\n$line48.$read$$iw.(:65)\n$line48.$read.(:67)\n$line48.$read$.(:71)\n$line48.$read$.()\n$line48.$eval$.$print$lzycompute(:7)\n$line48.$eval$.$print(:6)\n$line48.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nCreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Physical Plan ==\nExecute CreateViewCommand\n +- CreateViewCommand `sys_props`, false, true, LocalTempView\n +- AnalysisBarrier\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro","sparkPlanInfo":{"nodeName":"Execute CreateViewCommand","simpleString":"Execute CreateViewCommand","children":[],"metrics":[]},"time":1524182128463}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd","executionId":1,"time":1524182128463}
+{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart","executionId":2,"description":"show at :40","details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","physicalPlanDescription":"== Parsed Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- AnalysisBarrier\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Analyzed Logical Plan ==\nappId: string, attemptId: string, name: string, mode: string, completed: string, duration: string, endTime: string, endTimeEpoch: string, lastUpdated: string, lastUpdatedEpoch: string, sparkUser: string, startTime: string, startTimeEpoch: string, appSparkVersion: string, endDate: string, azkaban.link.workflow.url: string, azkaban.link.execution.url: string, azkaban.link.job.url: string, user.name: string\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [cast(appId#0 as string) AS appId#397, cast(attemptId#1 as string) AS attemptId#398, cast(name#2 as string) AS name#399, cast(mode#3 as string) AS mode#400, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, cast(endTime#6 as string) AS endTime#403, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, cast(lastUpdated#8 as string) AS lastUpdated#405, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, cast(sparkUser#10 as string) AS sparkUser#407, cast(startTime#11 as string) AS startTime#408, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, cast(appSparkVersion#13 as string) AS appSparkVersion#410, cast(endDate#28 as string) AS endDate#411, cast(azkaban.link.workflow.url#159 as string) AS azkaban.link.workflow.url#412, cast(azkaban.link.execution.url#161 as string) AS azkaban.link.execution.url#413, cast(azkaban.link.job.url#163 as string) AS azkaban.link.job.url#414, cast(user.name#165 as string) AS user.name#415]\n +- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- Join LeftOuter, (appId#0 = appId#137)\n :- Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- Relation[appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] avro\n +- Aggregate [appId#137], [appId#137, first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else cast(null as string), true) AS azkaban.link.workflow.url#159, first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else cast(null as string), true) AS azkaban.link.execution.url#161, first(if ((key#148 <=> azkaban.link.job.url)) value#149 else cast(null as string), true) AS azkaban.link.job.url#163, first(if ((key#148 <=> user.name)) value#149 else cast(null as string), true) AS user.name#165]\n +- Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Project [appId#137, col#145]\n +- Generate explode(systemProperties#135), false, [col#145]\n +- Relation[runtime#133,sparkProperties#134,systemProperties#135,classpathEntries#136,appId#137,attemptId#138] avro\n\n== Optimized Logical Plan ==\nGlobalLimit 21\n+- LocalLimit 21\n +- Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>\n\n== Physical Plan ==\nCollectLimit 21\n+- *(1) LocalLimit 21\n +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]\n +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n +- *(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n +- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)\n : +- *(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n : +- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct azkaban.link.workflow.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165])\n +- *(4) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- Exchange hashpartitioning(appId#137, 200)\n +- SortAggregate(key=[appId#137], functions=[partial_first(if ((key#148 <=> azkaban.link.workflow.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.execution.url)) value#149 else null, true), partial_first(if ((key#148 <=> azkaban.link.job.url)) value#149 else null, true), partial_first(if ((key#148 <=> user.name)) value#149 else null, true)], output=[appId#137, first#273, valueSet#274, first#275, valueSet#276, first#277, valueSet#278, first#279, valueSet#280])\n +- *(3) Sort [appId#137 ASC NULLS FIRST], false, 0\n +- *(3) Project [appId#137, col#145.key AS key#148, col#145.value AS value#149]\n +- Generate explode(systemProperties#135), [appId#137], false, [col#145]\n +- *(2) FileScan avro [systemProperties#135,appId#137] Batched: false, Format: com.databricks.spark.avro.DefaultSource@485d3d1, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct>,appId:string>","sparkPlanInfo":{"nodeName":"CollectLimit","simpleString":"CollectLimit 21","children":[{"nodeName":"WholeStageCodegen","simpleString":"WholeStageCodegen","children":[{"nodeName":"LocalLimit","simpleString":"LocalLimit 21","children":[{"nodeName":"Project","simpleString":"Project [appId#0, attemptId#1, name#2, mode#3, cast(completed#4 as string) AS completed#401, cast(duration#5L as string) AS duration#402, endTime#6, cast(endTimeEpoch#7L as string) AS endTimeEpoch#404, lastUpdated#8, cast(lastUpdatedEpoch#9L as string) AS lastUpdatedEpoch#406, sparkUser#10, startTime#11, cast(startTimeEpoch#12L as string) AS startTimeEpoch#409, appSparkVersion#13, cast(endDate#28 as string) AS endDate#411, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]","children":[{"nodeName":"InputAdapter","simpleString":"InputAdapter","children":[{"nodeName":"InMemoryTableScan","simpleString":"InMemoryTableScan [appId#0, appSparkVersion#13, attemptId#1, azkaban.link.execution.url#161, azkaban.link.job.url#163, azkaban.link.workflow.url#159, completed#4, duration#5L, endDate#28, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, mode#3, name#2, sparkUser#10, startTime#11, startTimeEpoch#12L, user.name#165]","children":[],"metrics":[{"name":"number of output rows","accumulatorId":35,"metricType":"sum"},{"name":"scan time total (min, med, max)","accumulatorId":36,"metricType":"timing"}]}],"metrics":[]}],"metrics":[]}],"metrics":[]}],"metrics":[{"name":"duration total (min, med, max)","accumulatorId":34,"metricType":"timing"}]}],"metrics":[]},"time":1524182129952}
+{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1524182130194,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"FileScanRDD","Scope":"{\"id\":\"24\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"19\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"23\",\"name\":\"Generate\"}","Callsite":"cache at :41","Parent IDs":[10],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"18\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:54)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:56)\n$line49.$read$$iw$$iw$$iw$$iw$$iw.(:58)\n$line49.$read$$iw$$iw$$iw$$iw.(:60)\n$line49.$read$$iw$$iw$$iw.(:62)\n$line49.$read$$iw$$iw.(:64)\n$line49.$read$$iw.(:66)\n$line49.$read.(:68)\n$line49.$read$.(:72)\n$line49.$read$.()\n$line49.$eval$.$print$lzycompute(:7)\n$line49.$eval$.$print(:6)\n$line49.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"show at :40","Number of Tasks":1,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"33\",\"name\":\"map\"}","Callsite":"show at :40","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"32\",\"name\":\"mapPartitionsInternal\"}","Callsite":"show at :40","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"8\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"WholeStageCodegen\"}","Callsite":"show at :40","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":20,"Name":"*(5) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28, azkaban.link.workflow.url#159, azkaban.link.execution.url#161, azkaban.link.job.url#163, user.name#165]\n+- SortMergeJoin [appId#0], [appId#137], LeftOuter\n :- *(1) Sort [appId#0 ASC NULLS FIRST], false, 0\n : +- Exchange hashpartitioning(appId#0, 200)\n : +- InMemoryTableScan [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28]\n : +- InMemoryRelation [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, endDate#28], true, 10000, StorageLevel(disk, memory, deserialized, 1 rep...","Scope":"{\"id\":\"26\",\"name\":\"mapPartitionsInternal\"}","Callsite":"cache at :41","Parent IDs":[19],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"31\",\"name\":\"InMemoryTableScan\"}","Callsite":"show at :40","Parent IDs":[22],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":18,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"7\",\"name\":\"SortMergeJoin\"}","Callsite":"cache at :41","Parent IDs":[8,17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"13\",\"name\":\"SortAggregate\"}","Callsite":"cache at :41","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"ShuffledRowRDD","Scope":"{\"id\":\"17\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[14],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"4\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :41","Parent IDs":[18],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":200,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[0,1],"Details":"org.apache.spark.sql.Dataset.show(Dataset.scala:691)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:40)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:45)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:47)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:49)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:51)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:53)\n$line50.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:55)\n$line50.$read$$iw$$iw$$iw$$iw$$iw.(:57)\n$line50.$read$$iw$$iw$$iw$$iw.(:59)\n$line50.$read$$iw$$iw$$iw.(:61)\n$line50.$read$$iw$$iw.(:63)\n$line50.$read$$iw.(:65)\n$line50.$read.(:67)\n$line50.$read$.(:71)\n$line50.$read$.()\n$line50.$eval$.$print$lzycompute(:7)\n$line50.$eval$.$print(:6)\n$line50.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)","Accumulables":[]}],"Stage IDs":[0,1,2],"Properties":{"spark.sql.execution.id":"2"}}
+{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"cache at :41","Number of Tasks":4,"RDD Info":[{"RDD ID":6,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"Exchange\"}","Callsite":"cache at :41","Parent IDs":[5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"FileScanRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"*(1) Project [appId#0, attemptId#1, name#2, mode#3, completed#4, duration#5L, endTime#6, endTimeEpoch#7L, lastUpdated#8, lastUpdatedEpoch#9L, sparkUser#10, startTime#11, startTimeEpoch#12L, appSparkVersion#13, cast(endTime#6 as date) AS endDate#28]\n+- *(1) FileScan avro [appId#0,attemptId#1,name#2,mode#3,completed#4,duration#5L,endTime#6,endTimeEpoch#7L,lastUpdated#8,lastUpdatedEpoch#9L,sparkUser#10,startTime#11,startTimeEpoch#12L,appSparkVersion#13] Batched: false, Format: com.databricks.spark.avro.DefaultSource@7006b304, Location: InMemoryFileIndex[hdfs://clusternn01.grid.company.com:9000/data/hadoopdev/sparkmetrics/ltx1-..., PartitionFilters: [], PushedFilters: [], ReadSchema: struct:39","Parent IDs":[1],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"0\",\"name\":\"WholeStageCodegen\"}","Callsite":"cache at :39","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[4],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"InMemoryTableScan\"}","Callsite":"cache at :41","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":4,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.sql.Dataset.cache(Dataset.scala:2912)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:41)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:46)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:48)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:50)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:52)\n$line49.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.