Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
36 commits
Select commit Hold shift + click to select a range
c8e8abe
SPARK-23429: Add executor memory metrics to heartbeat and expose in e…
edwinalu Mar 9, 2018
5d6ae1c
modify MimaExcludes.scala to filter changes to SparkListenerExecutorM…
edwinalu Apr 2, 2018
ad10d28
Address code review comments, change event logging to stage end.
edwinalu Apr 22, 2018
10ed328
Add configuration parameter spark.eventLog.logExecutorMetricsUpdates.…
edwinalu May 15, 2018
2d20367
wip on enum based metrics
squito May 23, 2018
f904f1e
wip ... has both enum and non-enum version
squito May 23, 2018
c502ec4
case objects, mostly complete
squito May 23, 2018
7879e66
Merge pull request #1 from squito/metric_enums
edwinalu Jun 3, 2018
2662f6f
Address comments (move heartbeater from DAGScheduler to SparkContext,…
edwinalu Jun 10, 2018
2871335
SPARK-23429: Add executor memory metrics to heartbeat and expose in e…
edwinalu Mar 9, 2018
da83f2e
modify MimaExcludes.scala to filter changes to SparkListenerExecutorM…
edwinalu Apr 2, 2018
f25a44b
Address code review comments, change event logging to stage end.
edwinalu Apr 22, 2018
ca85c82
Add configuration parameter spark.eventLog.logExecutorMetricsUpdates.…
edwinalu May 15, 2018
8b74ba8
wip on enum based metrics
squito May 23, 2018
036148c
wip ... has both enum and non-enum version
squito May 23, 2018
91fb1db
case objects, mostly complete
squito May 23, 2018
2d8894a
Address comments (move heartbeater from DAGScheduler to SparkContext,…
edwinalu Jun 10, 2018
99044e6
Merge branch 'SPARK-23429.2' of https://github.com/edwinalu/spark int…
edwinalu Jun 14, 2018
263c8c8
code review comments
edwinalu Jun 14, 2018
812fdcf
code review comments:
edwinalu Jun 22, 2018
7ed42a5
Address code review comments. Also make executorUpdates in SparkListe…
edwinalu Jun 28, 2018
8d9acdf
Revert and make executorUpdates in SparkListenerExecutorMetricsUpdate…
edwinalu Jun 29, 2018
20799d2
code review comments: hid array implementation of executor metrics, a…
edwinalu Jul 25, 2018
8905d23
merge with master
edwinalu Jul 25, 2018
04875b8
Integration of ProcessTreeMetrics with PR 21221
Jul 26, 2018
a0eed11
address code review comments
edwinalu Aug 5, 2018
162b9b2
Merge branch 'SPARK-23429.2' of https://github.com/edwinalu/spark int…
Aug 6, 2018
29a44c7
Changing the position of ptree and also make the computation configur…
Aug 7, 2018
3671427
Seperate metrics for jvm, python and others and update the tests
Aug 8, 2018
03cd5bc
code review comments
edwinalu Aug 13, 2018
c79b5ab
Merge branch 'SPARK-23429.2' of https://github.com/edwinalu/spark int…
Aug 14, 2018
10e7f15
Merge branch 'master' into SPARK-23429.2
edwinalu Aug 14, 2018
a14b82a
merge conflicts
edwinalu Aug 14, 2018
2897281
disable stage executor metrics logging by default
edwinalu Aug 16, 2018
8f97b50
Merge branch 'SPARK-23429.2' of https://github.com/rezasafi/spark int…
Aug 17, 2018
b14cebc
Update JsonProtocolSuite with new metrics.
Aug 17, 2018
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Address code review comments, change event logging to stage end.
  • Loading branch information
edwinalu committed Jun 14, 2018
commit f25a44b95e4e6a8532c6541ee985789dff5bc7de
56 changes: 50 additions & 6 deletions core/src/main/scala/org/apache/spark/executor/Executor.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,13 @@ package org.apache.spark.executor

import java.io.{File, NotSerializableException}
import java.lang.Thread.UncaughtExceptionHandler
import java.lang.management.ManagementFactory
import java.lang.management.{BufferPoolMXBean, ManagementFactory}
import java.net.{URI, URL}
import java.nio.ByteBuffer
import java.util.Properties
import java.util.concurrent._
import javax.annotation.concurrent.GuardedBy
import javax.management.ObjectName

import scala.collection.JavaConverters._
import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
Expand All @@ -36,7 +37,7 @@ import org.apache.spark._
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config._
import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.memory.{MemoryManager, SparkOutOfMemoryError, TaskMemoryManager}
import org.apache.spark.rpc.RpcTimeout
import org.apache.spark.scheduler.{DirectTaskResult, IndirectTaskResult, Task, TaskDescription}
import org.apache.spark.shuffle.FetchFailedException
Expand Down Expand Up @@ -71,6 +72,12 @@ private[spark] class Executor(

private val conf = env.conf

// BufferPoolMXBean for direct memory
private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME)

// BufferPoolMXBean for mapped memory
private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME)

// No ip or host:port - just hostname
Utils.checkHost(executorHostname)
// must not have port specified.
Expand Down Expand Up @@ -788,10 +795,8 @@ private[spark] class Executor(
val curGCTime = computeTotalGcTime()

// get executor level memory metrics
val executorUpdates = new ExecutorMetrics(System.currentTimeMillis(),
ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(),
env.memoryManager.onHeapExecutionMemoryUsed, env.memoryManager.offHeapExecutionMemoryUsed,
env.memoryManager.onHeapStorageMemoryUsed, env.memoryManager.offHeapStorageMemoryUsed)
val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager,
directBufferPool, mappedBufferPool)

for (taskRunner <- runningTasks.values().asScala) {
if (taskRunner.task != null) {
Expand Down Expand Up @@ -829,4 +834,43 @@ private[spark] object Executor {
// task is fully deserialized. When possible, the TaskContext.getLocalProperty call should be
// used instead.
val taskDeserializationProps: ThreadLocal[Properties] = new ThreadLocal[Properties]

val DIRECT_BUFFER_POOL_NAME = "direct"
val MAPPED_BUFFER_POOL_NAME = "mapped"

/** Get the BufferPoolMXBean for the specified buffer pool. */
def getBufferPool(pool: String): BufferPoolMXBean = {
val name = new ObjectName("java.nio:type=BufferPool,name=" + pool)
ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer,
name.toString, classOf[BufferPoolMXBean])
}

/**
* Get the current executor level memory metrics.
*
* @param memoryManager the memory manager
* @param direct the direct memory buffer pool
* @param mapped the mapped memory buffer pool
* @return the executor memory metrics
*/
def getCurrentExecutorMetrics(
memoryManager: MemoryManager,
direct: BufferPoolMXBean,
mapped: BufferPoolMXBean) : ExecutorMetrics = {
val onHeapExecutionMemoryUsed = memoryManager.onHeapExecutionMemoryUsed
val offHeapExecutionMemoryUsed = memoryManager.offHeapExecutionMemoryUsed
val onHeapStorageMemoryUsed = memoryManager.onHeapStorageMemoryUsed
val offHeapStorageMemoryUsed = memoryManager.offHeapStorageMemoryUsed
new ExecutorMetrics(System.currentTimeMillis(),
ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(),
ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed(),
onHeapExecutionMemoryUsed,
offHeapExecutionMemoryUsed,
onHeapStorageMemoryUsed,
offHeapStorageMemoryUsed,
onHeapExecutionMemoryUsed + onHeapStorageMemoryUsed, // on heap unified memory
offHeapExecutionMemoryUsed + offHeapStorageMemoryUsed, // off heap unified memory
direct.getMemoryUsed,
mapped.getMemoryUsed)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,18 +26,29 @@ import org.apache.spark.annotation.DeveloperApi
* This is sent to the driver periodically (on executor heartbeat), to provide
* information about each executor's metrics.
*
* @param timestamp the time the metrics were collected
* @param jvmUsedMemory the amount of JVM used memory for the executor
* @param timestamp the time the metrics were collected, or -1 for Spark history
* log events which are logged when a stage has completed
* @param jvmUsedHeapMemory the amount of JVM used heap memory for the executor
* @param jvmUsedNonHeapMemory the amount of JVM used non-heap memory for the executor
* @param onHeapExecutionMemory the amount of on heap execution memory used
* @param offHeapExecutionMemory the amount of off heap execution memory used
* @param onHeapStorageMemory the amount of on heap storage memory used
* @param offHeapStorageMemory the amount of off heap storage memory used
* @param onHeapUnifiedMemory the amount of on heap unified region memory used
* @param offHeapUnifiedMemory the amount of off heap unified region memory used
* @param directMemory the amount of direct memory used
* @param mappedMemory the amount of mapped memory used
*/
@DeveloperApi
class ExecutorMetrics private[spark] (
val timestamp: Long,
val jvmUsedMemory: Long,
val jvmUsedHeapMemory: Long,
val jvmUsedNonHeapMemory: Long,
val onHeapExecutionMemory: Long,
val offHeapExecutionMemory: Long,
val onHeapStorageMemory: Long,
val offHeapStorageMemory: Long) extends Serializable
val offHeapStorageMemory: Long,
val onHeapUnifiedMemory: Long,
val offHeapUnifiedMemory: Long,
val directMemory: Long,
val mappedMemory: Long) extends Serializable
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ import org.apache.commons.lang3.SerializationUtils

import org.apache.spark._
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
import org.apache.spark.executor.{Executor, ExecutorMetrics, TaskMetrics}
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config
import org.apache.spark.network.util.JavaUtils
Expand Down Expand Up @@ -214,6 +214,12 @@ class DAGScheduler(
private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat,
sc.conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))

/** BufferPoolMXBean for direct memory */
private val directBufferPool = Executor.getBufferPool(Executor.DIRECT_BUFFER_POOL_NAME)

/** BufferPoolMXBean for mapped memory */
private val mappedBufferPool = Executor.getBufferPool(Executor.MAPPED_BUFFER_POOL_NAME)

/**
* Called by the TaskSetManager to report task's starting.
*/
Expand Down Expand Up @@ -1764,12 +1770,8 @@ class DAGScheduler(
/** Reports heartbeat metrics for the driver. */
private def reportHeartBeat(): Unit = {
// get driver memory metrics
val driverUpdates = new ExecutorMetrics(System.currentTimeMillis(),
ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed(),
sc.env.memoryManager.onHeapExecutionMemoryUsed,
sc.env.memoryManager.offHeapExecutionMemoryUsed,
sc.env.memoryManager.onHeapStorageMemoryUsed,
sc.env.memoryManager.offHeapStorageMemoryUsed)
val driverUpdates = Executor.getCurrentExecutorMetrics(
sc.env.memoryManager, directBufferPool, mappedBufferPool)
val accumUpdates = new Array[(Long, Int, Int, Seq[AccumulableInfo])](0)
listenerBus.post(SparkListenerExecutorMetricsUpdate("driver", accumUpdates,
Some(driverUpdates)))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,9 @@ private[spark] class EventLoggingListener(
// Visible for tests only.
private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)

// Peak metric values for each executor
private var peakExecutorMetrics = new mutable.HashMap[String, PeakExecutorMetrics]()
// map of live stages, to peak executor metrics for the stage
private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int),
mutable.HashMap[String, PeakExecutorMetrics]]()

/**
* Creates the log file in the configured log directory.
Expand Down Expand Up @@ -162,7 +163,8 @@ private[spark] class EventLoggingListener(
override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = {
logEvent(event)
// clear the peak metrics when a new stage starts
peakExecutorMetrics.values.foreach(_.reset())
liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()),
new mutable.HashMap[String, PeakExecutorMetrics]())
}

override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event)
Expand All @@ -177,6 +179,27 @@ private[spark] class EventLoggingListener(

// Events that trigger a flush
override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
// log the peak executor metrics for the stage, for each executor
val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]()
val executorMap = liveStageExecutorMetrics.remove(
(event.stageInfo.stageId, event.stageInfo.attemptNumber()))
executorMap.foreach {
executorEntry => {
for ((executorId, peakExecutorMetrics) <- executorEntry) {
val executorMetrics = new ExecutorMetrics(-1, peakExecutorMetrics.jvmUsedHeapMemory,
peakExecutorMetrics.jvmUsedNonHeapMemory, peakExecutorMetrics.onHeapExecutionMemory,
peakExecutorMetrics.offHeapExecutionMemory, peakExecutorMetrics.onHeapStorageMemory,
peakExecutorMetrics.offHeapStorageMemory, peakExecutorMetrics.onHeapUnifiedMemory,
peakExecutorMetrics.offHeapUnifiedMemory, peakExecutorMetrics.directMemory,
peakExecutorMetrics.mappedMemory)
val executorUpdate = new SparkListenerExecutorMetricsUpdate(
executorId, accumUpdates, Some(executorMetrics))
logEvent(executorUpdate)
}
}
}

// log stage completed event
logEvent(event, flushLogger = true)
}

Expand Down Expand Up @@ -205,12 +228,10 @@ private[spark] class EventLoggingListener(
}
override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = {
logEvent(event, flushLogger = true)
peakExecutorMetrics.put(event.executorId, new PeakExecutorMetrics())
}

override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = {
logEvent(event, flushLogger = true)
peakExecutorMetrics.remove(event.executorId)
}

override def onExecutorBlacklisted(event: SparkListenerExecutorBlacklisted): Unit = {
Expand Down Expand Up @@ -244,19 +265,13 @@ private[spark] class EventLoggingListener(
}
}

/**
* Log if there is a new peak value for one of the memory metrics for the given executor.
* Metrics are cleared out when a new stage is started in onStageSubmitted, so this will
* log new peak memory metric values per executor per stage.
*/
override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = {
var log: Boolean = false
// For the active stages, record any new peak values for the memory metrics for the executor
event.executorUpdates.foreach { executorUpdates =>
val peakMetrics = peakExecutorMetrics.getOrElseUpdate(event.execId, new PeakExecutorMetrics())
if (peakMetrics.compareAndUpdate(executorUpdates)) {
val accumUpdates = new ArrayBuffer[(Long, Int, Int, Seq[AccumulableInfo])]()
logEvent(new SparkListenerExecutorMetricsUpdate(event.execId, accumUpdates,
event.executorUpdates), flushLogger = true)
liveStageExecutorMetrics.values.foreach { peakExecutorMetrics =>
val peakMetrics = peakExecutorMetrics.getOrElseUpdate(
event.execId, new PeakExecutorMetrics())
peakMetrics.compareAndUpdate(executorUpdates)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,17 +21,40 @@ import org.apache.spark.executor.ExecutorMetrics
import org.apache.spark.status.api.v1.PeakMemoryMetrics

/**
* Records the peak values for executor level metrics. If jvmUsedMemory is -1, then no values have
* been recorded yet.
* Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no
* values have been recorded yet.
*/
private[spark] class PeakExecutorMetrics {
private var jvmUsedMemory = -1L;
private var onHeapExecutionMemory = 0L
private var offHeapExecutionMemory = 0L
private var onHeapStorageMemory = 0L
private var offHeapStorageMemory = 0L
private var onHeapUnifiedMemory = 0L
private var offHeapUnifiedMemory = 0L
private var _jvmUsedHeapMemory = -1L;
private var _jvmUsedNonHeapMemory = 0L;
private var _onHeapExecutionMemory = 0L
private var _offHeapExecutionMemory = 0L
private var _onHeapStorageMemory = 0L
private var _offHeapStorageMemory = 0L
private var _onHeapUnifiedMemory = 0L
private var _offHeapUnifiedMemory = 0L
private var _directMemory = 0L
private var _mappedMemory = 0L

def jvmUsedHeapMemory: Long = _jvmUsedHeapMemory

def jvmUsedNonHeapMemory: Long = _jvmUsedNonHeapMemory

def onHeapExecutionMemory: Long = _onHeapExecutionMemory

def offHeapExecutionMemory: Long = _offHeapExecutionMemory

def onHeapStorageMemory: Long = _onHeapStorageMemory

def offHeapStorageMemory: Long = _offHeapStorageMemory

def onHeapUnifiedMemory: Long = _onHeapUnifiedMemory

def offHeapUnifiedMemory: Long = _offHeapUnifiedMemory

def directMemory: Long = _directMemory

def mappedMemory: Long = _mappedMemory

/**
* Compare the specified memory values with the saved peak executor memory
Expand All @@ -43,36 +66,44 @@ private[spark] class PeakExecutorMetrics {
def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = {
var updated: Boolean = false

if (executorMetrics.jvmUsedMemory > jvmUsedMemory) {
jvmUsedMemory = executorMetrics.jvmUsedMemory
if (executorMetrics.jvmUsedHeapMemory > _jvmUsedHeapMemory) {
_jvmUsedHeapMemory = executorMetrics.jvmUsedHeapMemory
updated = true
}
if (executorMetrics.jvmUsedNonHeapMemory > _jvmUsedNonHeapMemory) {
_jvmUsedNonHeapMemory = executorMetrics.jvmUsedNonHeapMemory
updated = true
}
if (executorMetrics.onHeapExecutionMemory > _onHeapExecutionMemory) {
_onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory
updated = true
}
if (executorMetrics.onHeapExecutionMemory > onHeapExecutionMemory) {
onHeapExecutionMemory = executorMetrics.onHeapExecutionMemory
if (executorMetrics.offHeapExecutionMemory > _offHeapExecutionMemory) {
_offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
updated = true
}
if (executorMetrics.offHeapExecutionMemory > offHeapExecutionMemory) {
offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) {
_onHeapStorageMemory = executorMetrics.onHeapStorageMemory
updated = true
}
if (executorMetrics.onHeapStorageMemory > onHeapStorageMemory) {
onHeapStorageMemory = executorMetrics.onHeapStorageMemory
if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) {
_offHeapStorageMemory = executorMetrics.offHeapStorageMemory
updated = true
}
if (executorMetrics.offHeapStorageMemory > offHeapStorageMemory) {
offHeapStorageMemory = executorMetrics.offHeapStorageMemory
if (executorMetrics.onHeapUnifiedMemory > _onHeapUnifiedMemory) {
_onHeapUnifiedMemory = executorMetrics.onHeapUnifiedMemory
updated = true
}
val newOnHeapUnifiedMemory = (executorMetrics.onHeapExecutionMemory +
executorMetrics.onHeapStorageMemory)
if (newOnHeapUnifiedMemory > onHeapUnifiedMemory) {
onHeapUnifiedMemory = newOnHeapUnifiedMemory
if (executorMetrics.offHeapUnifiedMemory > _offHeapUnifiedMemory) {
_offHeapUnifiedMemory = executorMetrics.offHeapUnifiedMemory
updated = true
}
val newOffHeapUnifiedMemory = (executorMetrics.offHeapExecutionMemory +
executorMetrics.offHeapStorageMemory)
if ( newOffHeapUnifiedMemory > offHeapUnifiedMemory) {
offHeapUnifiedMemory = newOffHeapUnifiedMemory
if (executorMetrics.directMemory > _directMemory) {
_directMemory = executorMetrics.directMemory
updated = true
}
if (executorMetrics.mappedMemory > _mappedMemory) {
_mappedMemory = executorMetrics.mappedMemory
updated = true
}

Expand All @@ -84,23 +115,13 @@ private[spark] class PeakExecutorMetrics {
* values set.
*/
def getPeakMemoryMetrics: Option[PeakMemoryMetrics] = {
if (jvmUsedMemory < 0) {
if (_jvmUsedHeapMemory < 0) {
None
} else {
Some(new PeakMemoryMetrics(jvmUsedMemory, onHeapExecutionMemory,
offHeapExecutionMemory, onHeapStorageMemory, offHeapStorageMemory,
onHeapUnifiedMemory, offHeapUnifiedMemory))
Some(new PeakMemoryMetrics(_jvmUsedHeapMemory, _jvmUsedNonHeapMemory,
_onHeapExecutionMemory, _offHeapExecutionMemory, _onHeapStorageMemory,
_offHeapStorageMemory, _onHeapUnifiedMemory, _offHeapUnifiedMemory,
_directMemory, _mappedMemory))
}
}

/** Clears/resets the saved peak values. */
def reset(): Unit = {
jvmUsedMemory = -1L;
onHeapExecutionMemory = 0L
offHeapExecutionMemory = 0L
onHeapStorageMemory = 0L
offHeapStorageMemory = 0L
onHeapUnifiedMemory = 0L
offHeapUnifiedMemory = 0L
}
}
Loading