Skip to content
Closed
Show file tree
Hide file tree
Changes from 4 commits
Commits
Show all changes
31 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
a0eed11
address code review comments
edwinalu Aug 5, 2018
03cd5bc
code review comments
edwinalu Aug 13, 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
ee4aa1d
Merge branch 'master' into SPARK-23429.2
edwinalu Sep 6, 2018
571285b
fix indentation
edwinalu Sep 7, 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
8 changes: 5 additions & 3 deletions core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand All @@ -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
Expand Down Expand Up @@ -120,14 +122,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)
}
Expand Down
53 changes: 53 additions & 0 deletions core/src/main/scala/org/apache/spark/Heartbeater.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* 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.util.{ThreadUtils, Utils}

/**
* Creates a heartbeat thread which will call the specified reportHeartbeat function at
* intervals of intervalMs.
*
* @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(reportHeartbeat: () => Unit, name: String, intervalMs: Long) {
// Executor for the heartbeat task
private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor(name)

/** Schedules a task to report a heartbeat. */
private[spark] def start(): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: whole class is private[spark] so you don't need to add this to individual methods

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Removed.

// 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. */
private[spark] def stop(): Unit = {
heartbeater.shutdown()
heartbeater.awaitTermination(10, TimeUnit.SECONDS)
}
}

80 changes: 58 additions & 22 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 @@ -148,7 +155,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(reportHeartBeat, "executor-heartbeater",
conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s"))

// must be initialized before running startDriverHeartbeat()
private val heartbeatReceiverRef =
Expand All @@ -167,7 +175,7 @@ private[spark] class Executor(
*/
private var heartbeatFailures = 0

startDriverHeartbeater()
heartbeater.start()

private[executor] def numRunningTasks: Int = runningTasks.size()

Expand Down Expand Up @@ -216,8 +224,7 @@ private[spark] class Executor(

def stop(): Unit = {
env.metricsSystem.report()
heartbeater.shutdown()
heartbeater.awaitTermination(10, TimeUnit.SECONDS)
heartbeater.stop()
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

future: try {} catch { case NonFatal(e)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added.

threadPool.shutdown()
if (!isLocal) {
env.stop()
Expand Down Expand Up @@ -772,6 +779,10 @@ private[spark] class Executor(
val accumUpdates = new ArrayBuffer[(Long, Seq[AccumulatorV2[_, _]])]()
val curGCTime = computeTotalGcTime()

// get executor level memory metrics
val executorUpdates = Executor.getCurrentExecutorMetrics(env.memoryManager,
directBufferPool, mappedBufferPool)

for (taskRunner <- runningTasks.values().asScala) {
if (taskRunner.task != null) {
taskRunner.task.metrics.mergeShuffleReadMetrics()
Expand All @@ -780,7 +791,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"))
Expand All @@ -800,26 +812,50 @@ 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 {
// This is reserved for internal use by components that need to read task properties before a
// 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 = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

does it make more sense to move this inside Heartbeater? Then you don't need to pass in any BufferPoolMXBeans. also rename to "getCurrentMemoryMetrics"

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, and easier to share the code between driver and executor.

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
@@ -0,0 +1,54 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.executor

import org.apache.spark.annotation.DeveloperApi

/**
* :: DeveloperApi ::
* Executor level metrics.
*
* 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, 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 jvmUsedHeapMemory: Long,
val jvmUsedNonHeapMemory: Long,
val onHeapExecutionMemory: Long,
val offHeapExecutionMemory: Long,
val onHeapStorageMemory: Long,
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 @@ -69,6 +69,11 @@ package object config {
.bytesConf(ByteUnit.KiB)
.createWithDefaultString("100k")

private[spark] val EVENT_LOG_EXECUTOR_METRICS_UPDATES =
ConfigBuilder("spark.eventLog.logExecutorMetricsUpdates.enabled")
.booleanConf
.createWithDefault(true)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

should this be "false" for now until we could test this out more, just to be on the safe side?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That would be safer. I'll change to false, and we can change change to true after people have had a chance to test it out.


private[spark] val EVENT_LOG_OVERWRITE =
ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false)

Expand Down
20 changes: 20 additions & 0 deletions core/src/main/scala/org/apache/spark/memory/MemoryManager.scala
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,26 @@ private[spark] abstract class MemoryManager(
onHeapStorageMemoryPool.memoryUsed + offHeapStorageMemoryPool.memoryUsed
}

/**
* On heap execution memory currently in use, in bytes.
*/
final def onHeapExecutionMemoryUsed: Long = onHeapExecutionMemoryPool.memoryUsed
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do these have to be synchronized?

Copy link
Contributor Author

@edwinalu edwinalu Aug 5, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since it's accessing just one Long field, I don't think so. The other methods, which are synchronized are summing 2 fields.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It probably should be if only because the variable is annotated with @GuardedBy(this), so it makes the code more consistent to mark this as synchronized.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's true -- I'll change the methods to synchronized.


/**
* Off heap execution memory currently in use, in bytes.
*/
final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

synchronized here also and in the below two methods.


/**
* On heap storage memory currently in use, in bytes.
*/
final def onHeapStorageMemoryUsed: Long = onHeapStorageMemoryPool.memoryUsed

/**
* Off heap storage memory currently in use, in bytes.
*/
final def offHeapStorageMemoryUsed: Long = offHeapStorageMemoryPool.memoryUsed

/**
* Returns the execution memory consumption, in bytes, for the given task.
*/
Expand Down
Loading