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
code review comments: hid array implementation of executor metrics, a…
…nd add ExecutorMetrics, with getMetricValue()

method for accessing executor metric values. Rename MetricGetter to ExecutorMetricType.

Should ExecutorMetricType be moved to executor package, or ExecutorMetrics be moved to metrics package?
Should Json (de)serialization functions be moved from api.scala to ExecutorMetrics?
  • Loading branch information
edwinalu committed Jul 25, 2018
commit 20799d2af7b70334534be913f7defea6d6b79ffb
3 changes: 2 additions & 1 deletion 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 @@ -38,7 +39,7 @@ private[spark] case class Heartbeat(
executorId: String,
accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])], // taskId -> accumulator updates
blockManagerId: BlockManagerId,
executorUpdates: Array[Long]) // executor level updates
executorUpdates: ExecutorMetrics) // executor level updates

/**
* An event that SparkContext uses to notify HeartbeatReceiver that SparkContext.taskScheduler is
Expand Down
8 changes: 5 additions & 3 deletions core/src/main/scala/org/apache/spark/Heartbeater.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,10 @@ 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.MetricGetter
import org.apache.spark.metrics.ExecutorMetricType
import org.apache.spark.util.{ThreadUtils, Utils}

/**
Expand Down Expand Up @@ -62,8 +63,9 @@ private[spark] class Heartbeater(
* Get the current executor level metrics. These are returned as an array, with the index
* determined by MetricGetter.values
*/
def getCurrentMetrics(): Array[Long] = {
MetricGetter.values.map(_.getMetricValue(memoryManager)).toArray
def getCurrentMetrics(): ExecutorMetrics = {
val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray
new ExecutorMetrics(metrics)
}
}

Original file line number Diff line number Diff line change
@@ -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: Boolean = false

(0 until ExecutorMetricType.values.length).foreach { idx =>
if ( executorMetrics.metrics(idx) > metrics(idx)) {
updated = true
metrics(idx) = executorMetrics.metrics(idx)
}
}
updated
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,64 +21,70 @@ import javax.management.ObjectName

import org.apache.spark.memory.MemoryManager

private[spark] sealed trait MetricGetter {
def getMetricValue(memoryManager: MemoryManager): Long
val name = getClass().getName().stripSuffix("$").split("""\.""").last
/**
* 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 MemoryManagerMetricGetter(
f: MemoryManager => Long) extends MetricGetter {
override def getMetricValue(memoryManager: MemoryManager): Long = {
private[spark] abstract class MemoryManagerExecutorMetricType(
f: MemoryManager => Long) extends ExecutorMetricType {
override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
f(memoryManager)
}
}

private[spark]abstract class MBeanMetricGetter(mBeanName: String)
extends MetricGetter {
val bean = ManagementFactory.newPlatformMXBeanProxy(ManagementFactory.getPlatformMBeanServer,
private[spark]abstract class MBeanExecutorMetricType(mBeanName: String)
extends ExecutorMetricType {
private val bean = ManagementFactory.newPlatformMXBeanProxy(
ManagementFactory.getPlatformMBeanServer,
new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean])

override def getMetricValue(memoryManager: MemoryManager): Long = {
override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
bean.getMemoryUsed
}
}

private[spark] case object JVMHeapMemory extends MetricGetter {
override def getMetricValue(memoryManager: MemoryManager): Long = {
case object JVMHeapMemory extends ExecutorMetricType {
override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed()
}
}

private[spark] case object JVMOffHeapMemory extends MetricGetter {
override def getMetricValue(memoryManager: MemoryManager): Long = {
case object JVMOffHeapMemory extends ExecutorMetricType {
override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = {
ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed()
}
}

private[spark] case object OnHeapExecutionMemory extends MemoryManagerMetricGetter(
case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType(
_.onHeapExecutionMemoryUsed)

private[spark] case object OffHeapExecutionMemory extends MemoryManagerMetricGetter(
case object OffHeapExecutionMemory extends MemoryManagerExecutorMetricType(
_.offHeapExecutionMemoryUsed)

private[spark] case object OnHeapStorageMemory extends MemoryManagerMetricGetter(
case object OnHeapStorageMemory extends MemoryManagerExecutorMetricType(
_.onHeapStorageMemoryUsed)

private[spark] case object OffHeapStorageMemory extends MemoryManagerMetricGetter(
case object OffHeapStorageMemory extends MemoryManagerExecutorMetricType(
_.offHeapStorageMemoryUsed)

private[spark] case object OnHeapUnifiedMemory extends MemoryManagerMetricGetter(
case object OnHeapUnifiedMemory extends MemoryManagerExecutorMetricType(
(m => m.onHeapExecutionMemoryUsed + m.onHeapStorageMemoryUsed))

private[spark] case object OffHeapUnifiedMemory extends MemoryManagerMetricGetter(
case object OffHeapUnifiedMemory extends MemoryManagerExecutorMetricType(
(m => m.offHeapExecutionMemoryUsed + m.offHeapStorageMemoryUsed))

private[spark] case object DirectPoolMemory extends MBeanMetricGetter(
case object DirectPoolMemory extends MBeanExecutorMetricType(
"java.nio:type=BufferPool,name=direct")
private[spark] case object MappedPoolMemory extends MBeanMetricGetter(

case object MappedPoolMemory extends MBeanExecutorMetricType(
"java.nio:type=BufferPool,name=mapped")

private[spark] object MetricGetter {
private[spark] object ExecutorMetricType {
// List of all executor metric types
val values = IndexedSeq(
JVMHeapMemory,
JVMOffHeapMemory,
Expand All @@ -92,5 +98,7 @@ private[spark] object MetricGetter {
MappedPoolMemory
)

val idxAndValues = values.zipWithIndex.map(_.swap)
// Map of executor metric type to its index in values.
val metricIdxMap =
Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*)
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ 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
Expand Down Expand Up @@ -248,7 +248,7 @@ class DAGScheduler(
accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])],
blockManagerId: BlockManagerId,
// executor metrics indexed by MetricGetter.values
executorUpdates: Array[Long]): Boolean = {
executorUpdates: ExecutorMetrics): Boolean = {
listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates,
Some(executorUpdates)))
blockManagerMaster.driverEndpoint.askSync[Boolean](
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets
import java.util.EnumSet
import java.util.Locale

import scala.collection.mutable
import scala.collection.mutable.{ArrayBuffer, HashMap}

import org.apache.hadoop.conf.Configuration
Expand All @@ -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
Expand Down Expand Up @@ -96,7 +96,7 @@ private[spark] class EventLoggingListener(
private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)

// map of (stageId, stageAttempt), to peak executor metrics for the stage
private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]()
private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, ExecutorMetrics]]()

/**
* Creates the log file in the configured log directory.
Expand Down Expand Up @@ -165,7 +165,7 @@ private[spark] class EventLoggingListener(
if (shouldLogStageExecutorMetrics) {
// record the peak metrics for the new stage
liveStageExecutorMetrics.put((event.stageInfo.stageId, event.stageInfo.attemptNumber()),
new HashMap[String, PeakExecutorMetrics]())
new HashMap[String, ExecutorMetrics]())
}
}

Expand Down Expand Up @@ -195,7 +195,7 @@ private[spark] class EventLoggingListener(
executorOpt.foreach { execMap =>
execMap.foreach { case (executorId, peakExecutorMetrics) =>
logEvent(new SparkListenerStageExecutorMetrics(executorId, event.stageInfo.stageId,
event.stageInfo.attemptNumber(), peakExecutorMetrics.metrics))
event.stageInfo.attemptNumber(), peakExecutorMetrics))
}
}
}
Expand Down Expand Up @@ -272,8 +272,8 @@ private[spark] class EventLoggingListener(
event.executorUpdates.foreach { executorUpdates =>
liveStageExecutorMetrics.values.foreach { peakExecutorMetrics =>
val peakMetrics = peakExecutorMetrics.getOrElseUpdate(
event.execId, new PeakExecutorMetrics())
peakMetrics.compareAndUpdate(executorUpdates)
event.execId, new ExecutorMetrics())
peakMetrics.compareAndUpdatePeakValues(executorUpdates)
}
}
}
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -166,7 +166,7 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends
case class SparkListenerExecutorMetricsUpdate(
execId: String,
accumUpdates: Seq[(Long, Int, Int, Seq[AccumulableInfo])],
executorUpdates: Option[Array[Long]] = None)
executorUpdates: Option[ExecutorMetrics] = None)
extends SparkListenerEvent

/**
Expand All @@ -182,7 +182,7 @@ case class SparkListenerStageExecutorMetrics(
execId: String,
stageId: Int,
stageAttemptId: Int,
executorMetrics: Array[Long])
executorMetrics: ExecutorMetrics)
extends SparkListenerEvent

@DeveloperApi
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -76,7 +77,7 @@ private[spark] trait TaskScheduler {
execId: String,
accumUpdates: Array[(Long, Seq[AccumulatorV2[_, _]])],
blockManagerId: BlockManagerId,
executorUpdates: Array[Long]): Boolean
executorUpdates: ExecutorMetrics): Boolean

/**
* Get an application ID associated with the job.
Expand Down
Loading