You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by edwinalu <gi...@git.apache.org> on 2018/05/02 23:57:28 UTC

[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

GitHub user edwinalu opened a pull request:

    https://github.com/apache/spark/pull/21221

    [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API

    The original PR #20940 is messed up, and the dif shows changes not related to SPARK-23429. This is a cleaned up version of that pull request.
    
    Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage memory, on/off heap unified memory, direct memory, and mapped memory), and expose via the executors REST API. This information will help provide insight into how executor and driver JVM memory is used, and for the different memory regions. It can be used to help determine good values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction.
    
    ## What changes were proposed in this pull request?
    
    An ExecutorMetrics class is added, with jvmUsedHeapMemory, jvmUsedNonHeapMemory, onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory and mappedMemory. The new ExecutorMetrics is sent by executors to the driver as part of the Heartbeat. A heartbeat is added for the driver as well, to collect these metrics for the driver.
    
    The EventLoggingListener store information about the peak values for each metric, per active stage and executor. When a StageCompleted event is seen, and ExecutorMetricsUpdate event will be logged for each executor, with peal values for the stage. Only the ExecutorMetrics will be logged, and not the TaskMetrics, to minimize additional logging.
    
    The AppStatusListener records the peak values for each memory metric.
    
    The new memory metrics are added to the executors REST API.
    
    ## How was this patch tested?
    
    New unit tests have been added. This was also tested on our cluster.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/edwinalu/spark SPARK-23429.2

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/21221.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #21221
    
----
commit c8e8abedbdfec6e92b0c63e90f3c2c5755fd8978
Author: Edwina Lu <ed...@...>
Date:   2018-03-09T23:39:36Z

    SPARK-23429: Add executor memory metrics to heartbeat and expose in executors REST API
    
    Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage
    memory), and expose via the executors REST API. This information will help provide insight into how executor
    and driver JVM memory is used, and for the different memory regions. It can be used to help determine good
    values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction.
    
    Add an ExecutorMetrics class, with jvmUsedMemory, onHeapExecutionMemory, offHeapExecutionMemory,
    onHeapStorageMemory, and offHeapStorageMemory. The new ExecutorMetrics will be sent by executors to the
    driver as part of Heartbeat. A heartbeat will be added for the driver as well, to collect these metrics
    for the driver.
    
    Modify the EventLoggingListener to log ExecutorMetricsUpdate events if there is a new peak value for any
    of the memory metrics for an executor and stage. Only the ExecutorMetrics will be logged, and not the
    TaskMetrics, to minimize additional logging.
    
    Modify the AppStatusListener to record the peak values for each memory metric.
    
    Add the new memory metrics to the executors REST API.

commit 5d6ae1c34bf6618754e4b8b2e756a9a7b4bad987
Author: Edwina Lu <ed...@...>
Date:   2018-04-02T02:13:41Z

    modify MimaExcludes.scala to filter changes to SparkListenerExecutorMetricsUpdate

commit ad10d2814bbfbaf8c21fcbb1abe83ef7a8e9ffe7
Author: Edwina Lu <ed...@...>
Date:   2018-04-22T00:02:57Z

    Address code review comments, change event logging to stage end.

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190990562
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -81,7 +84,7 @@ private[spark] class EventLoggingListener(
       private val compressionCodecName = compressionCodec.map { c =>
         CompressionCodec.getShortName(c.getClass.getName)
       }
    -
    +logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates)
    --- End diff --
    
    Removed. Thanks, I hadn't meant to push that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190990593
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +96,10 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int),
    +    mutable.HashMap[String, PeakExecutorMetrics]]()
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91811 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91811/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190363619
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    --- End diff --
    
    I found this pretty confusing at first.  I suggest renaming `i` to `logIdx` and including a comment about the `j` loop.  Also we tend to use `(1 to 2).foreach`.  eg.
    
    ```scala
    // just before the SparkListenerStageCompleted gets logged, we expect to get a 
    // SparkListenerExecutorMetricsUpdate for each executor
    (1 to 2).foreach { _ =>
      checkExecutorMetricsUpdate(lines(logIdx), stageCompleted.stageInfo.stageId,
        expectedMetricsEvents)
      logIdx += 1
    }
    // also check that we get the expected SparkListenerStageCompleted
    checkEvent(lines(logIdx), event)
    logIdx += 1
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #92275 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92275/testReport)** for PR 21221 at commit [`812fdcf`](https://github.com/apache/spark/commit/812fdcf3961bae2a4fa20b4f60e739b45233fcd0).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #93570 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93570/testReport)** for PR 21221 at commit [`8905d23`](https://github.com/apache/spark/commit/8905d231c3a959f70266223d3546b17a655cee39).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195542018
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    +    metricsMap match {
    +      case Some(metrics) =>
    +        Some(MetricGetter.values.map { m =>
    +          metrics.getOrElse (m.name, 0L) match {
    +            case intVal: Int => intVal.toLong
    +            case longVal: Long => longVal
    +          }
    +        }.toArray)
    +      case None => None
    +    }
    +  }
    +}
    +
    +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */
    +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] {
    +  override def serialize(
    +      metrics: Option[Array[Long]],
    +      jsonGenerator: JsonGenerator,
    +      serializerProvider: SerializerProvider): Unit = {
    +    metrics match {
    +      case Some(m) =>
    +        val metricsMap = (0 until MetricGetter.values.length).map { idx =>
    --- End diff --
    
    It's still being used in JsonProtocol.executorMetricsToJson -- let me know if you'd like me to convert that to use values instead.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207037118
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    --- End diff --
    
    yeah, but you're talking about both a `foreach` *and* an `if` together.
    
    A long time back we discussed using `option.fold` for this, as it is all in one function, but we rejected it as being pretty confusing for most developers.
    
    ```scala
    scala> def foo(x: Option[String]) = x.fold("nada")("some " + _)
    foo: (x: Option[String])String
    
    scala> foo(None)
    res0: String = nada
    
    scala> foo(Some("blah"))
    res1: String = some blah
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209717523
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) {
    +          maybeUpdate(exec, now)
    +        }
    +      }
    +    }
    +  }
    +
    +  override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = {
    +    val now = System.nanoTime()
    --- End diff --
    
    Should we use a `Clock` instance here for testing?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191489062
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * 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 _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
    +   * values, and update if there is a new peak value.
    +   *
    +   * @param executorMetrics the executor metrics to compare
    +   * @return if there is a new peak value for any metric
    +   */
    +  def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = {
    +    var updated: Boolean = false
    +
    +    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.offHeapExecutionMemory > _offHeapExecutionMemory) {
    +      _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
    +      updated = true
    +    }
    +    if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) {
    +      _onHeapStorageMemory = executorMetrics.onHeapStorageMemory
    +      updated = true
    +    }
    +    if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) {
    +      _offHeapStorageMemory = executorMetrics.offHeapStorageMemory
    --- End diff --
    
    The more you can take it over from here, the better :) But let me know if there is anything which is confusing, or if the TODOs that I've left actually don't seem possible etc. and I can take a closer look.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722887
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    +    }
    +  }
    +}
    +/** 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 match {
    +      case Some(m) =>
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r210691276
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -69,6 +69,11 @@ package object config {
         .bytesConf(ByteUnit.KiB)
         .createWithDefaultString("100k")
     
    +  private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS =
    +    ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled")
    +      .booleanConf
    +      .createWithDefault(true)
    --- End diff --
    
    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.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209711557
  
    --- Diff: 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)
    --- End diff --
    
    Unclear - if we expose these metrics to some external consumer via an API for example, then we almost certainly want to have a schema labelling these fields for consumption by e.g. dashboards. I think what we have here is fine for now.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682917
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -264,6 +282,11 @@ private[spark] trait SparkListenerInterface {
        */
       def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit
     
    +  /**
    +   * Called when the driver reads stage executor metrics from the history log.
    --- End diff --
    
    Updated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    @squito For PeakMemoryMetrics in api.scala, changing to the array gives REST API output of:
    
      "peakMemoryMetrics" : {
        "metrics" : [ 755008624, 100519936, 0, 0, 47962185, 0, 47962185, 0, 98230, 0 ]
      }
    
    instead of:
    
      "peakMemoryMetrics" : {
        "jvmUsedHeapMemory" : 629553808,
        "jvmUsedNonHeapMemory" : 205304696,
        "onHeapExecutionMemory" : 0,
        "offHeapExecutionMemory" : 0,
        "onHeapStorageMemory" : 905801,
        "offHeapStorageMemory" : 0,
        "onHeapUnifiedMemory" : 905801,
        "offHeapUnifiedMemory" : 0,
        "directMemory" : 397602,
        "mappedMemory" : 0
      }
    
    Would it be OK to revert back to the original version of PeakMemoryMetrics, where each field is listed as a separate element? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94715 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94715/testReport)** for PR 21221 at commit [`03cd5bc`](https://github.com/apache/spark/commit/03cd5bceddc7867a90918430b23bf9fa3771edfd).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94715/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195773554
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L,
    +          Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L,
    +          Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L,
    +          Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L,
    +          Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L,
    +          Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L,
    +          Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))),
    +      createStageSubmittedEvent(1),
    --- End diff --
    
    another comment:
    
    now start stage 1, one more metric update for each executor, and a new peak for exec X
    
    (if there is a peak update, that is ...)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92488/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Jenkins, retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195886625
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    This is confusing, especially since the current code does not have the stage level metrics, just executor level. 
    
    The -1 wouldn't be replaced. PeakExecutorMetrics only tracks the peak metric values for each executor (and later each executor per stage) and doesn't have a timestamp.
    
    If there is a local max (500), which is the max between T3 and T5, it would be logged at time T5, even if it happens at T3.5.
    
    In actual event order, what the driver sees when the application is running:
    T1: start of stage 1
    T2: value of 1000 for metric m1
    T3: start of stage 2
    T3.5: peak value of 500 for metric m1
    T4: stage 1 ends
    T5: stage 2 ends
    
    Suppose that 1000 (seen at T2) is the peak value of m1 between T1 and T4, so it is the peak value seen while stage 1 is running. The m1=1000 value will be dumped as the max value in an executorMetricsUpdate event right before the stage 1 end event is logged. Also suppose that 500 (seen at T3.5 is the peak value of m1 between T3 and T5, so it is the peak value seen while stage 2 is running. The m1=500 value will be dumped as the max value in an executorMetricsUpdate right before the stage 2 end event is logged.
    
    The generated Spark history log would have the following order of events:
    
    Start of stage 1
    Start of stage 2
    executorMetricsUpdate with m1=1000
    end of stage 1
    executorMetricsUpdate with m1=500
    end of stage 2
    
    When the Spark history server is reading the log, it will will create the peakExecutorMetrics for stage 2 when stage 2 starts, which is before it sees the executorMetricsUpdate with m1=1000, and so will store m1=1000 as the current peak value. When it later sees the executorMetricsUpdate with m1=500, it needs to overwrite the m1 value (and set to 500), not compare and update to the max value (which would be 1000). The -1 would indicate that the event is coming from the Spark history log, is a peak value for the stage just about to complete, and should overwrite any previous values.
    
    If the timestamp is set to a positive value, then it will do a compare and update to the max value instead.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190990997
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +183,35 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 executor
    --- End diff --
    
    Yes, it's all running executors, and does not filter based on if they have tasks for the stage. I've updated the comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187506958
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,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(
    --- End diff --
    
    Yes, it's safer to clean up earlier attempts -- I can add some code to iterate through earlier attemptIDs.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r210690505
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -216,8 +217,7 @@ private[spark] class Executor(
     
       def stop(): Unit = {
         env.metricsSystem.report()
    -    heartbeater.shutdown()
    -    heartbeater.awaitTermination(10, TimeUnit.SECONDS)
    +    heartbeater.stop()
    --- End diff --
    
    Added.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r200826235
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    We can change back to using an ExecutorMetrics class in this case.
    
    The plan was for any new metrics to be added to the end, so that there wouldn't be any change in ordering, and executorMetrics could be changed to immutable Seq[Long], but there would still be the issue of having to reference MetricGetter to find out how the metrics are indexed. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190345745
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -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 = {
    --- End diff --
    
    does it make more sense to move this inside `Heartbeater`?  Then you don't need to pass in any BufferPoolMXBeans.  also rename to "getCurrentMemoryMetrics"


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198684121
  
    --- Diff: project/MimaExcludes.scala ---
    @@ -89,7 +89,13 @@ object MimaExcludes {
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"),
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"),
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="),
    -    ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol")
    +    ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"),
    +
    +    // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"),
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"),
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"),
    +    ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$")
    --- End diff --
    
    Will move up.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195290564
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -234,8 +272,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 (shouldLogExecutorMetricsUpdates) {
    +      // 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 PeakExecutorMetrics())
    +          peakMetrics.compareAndUpdate(executorUpdates)
    --- End diff --
    
    couldn't you get the right timestamp here to log, as you do for updating the live entity?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94757/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94757 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94757/testReport)** for PR 21221 at commit [`10e7f15`](https://github.com/apache/spark/commit/10e7f15dcec0ed21bb6ec4c972f80d7db1ce21cf).
     * This patch **fails from timeout after a configured wait of \`340m\`**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723173
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserializer for peakMemoryMetrics: convert map to ExecutorMetrics */
    +private[spark] class ExecutorMetricsJsonDeserializer
    +  extends JsonDeserializer[Option[ExecutorMetrics]] {
    --- End diff --
    
    Done.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Thanks, I think this looks good. With a prior +1 from @felixcheung and @squito I'm going to merge this now. Let us know if there are any further concerns and we can follow up.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195290854
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    --- End diff --
    
    I think you might able to do
    ```
    jsonParser.readValueAs(classOf[Option[Map[String, java.lang.Long]]])
    ```
    and then everything will get read as a long which simplifies the code below ... but I'm not 100% sure


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190346995
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +96,10 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int),
    +    mutable.HashMap[String, PeakExecutorMetrics]]()
    --- End diff --
    
    you could just import mutable.HashMap (added bonus -- fits on one line)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209770440
  
    --- Diff: 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: Boolean = false
    +
    +    (0 until ExecutorMetricType.values.length).foreach { idx =>
    +       if ( executorMetrics.metrics(idx) > metrics(idx)) {
    --- End diff --
    
    Fixed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198611872
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no
    + * values have been recorded yet.
    + */
    +private[spark] class PeakExecutorMetrics {
    --- End diff --
    
    ping


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209715208
  
    --- Diff: 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)
    --- End diff --
    
    Put a space after `[spark]`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682884
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +181,28 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 executorMap = liveStageExecutorMetrics.remove(
    +        (event.stageInfo.stageId, event.stageInfo.attemptNumber()))
    +      executorMap.foreach {
    +       executorEntry => {
    +          for ((executorId, peakExecutorMetrics) <- executorEntry) {
    --- End diff --
    
    Yes, the naming is confusing. Changed to the 1st option.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195289751
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    why is the timestamp -1 here?  if we're always logging it as -1, it doesn't seem very useful


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/21221


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91802 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91802/testReport)** for PR 21221 at commit [`2662f6f`](https://github.com/apache/spark/commit/2662f6f9c6a7c34cea34b748f6735eb1625b73cb).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207004345
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    --- End diff --
    
    same here as matt's comments, use `metricsMap.map {}`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    well, I think you should change the way PeakExecutorMetrics gets converted to json, so that it uses a name from the relevant `MetricGetter`.  You should be able to customize the way it gets converted to json here:
    
    https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala#L50


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #92488 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92488/testReport)** for PR 21221 at commit [`7ed42a5`](https://github.com/apache/spark/commit/7ed42a5d0eb0b93bb9ddecf14d9461c80dfe1ea0).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91983 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91983/testReport)** for PR 21221 at commit [`263c8c8`](https://github.com/apache/spark/commit/263c8c846265b6bdfdce471e44c163ab85b930a3).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r188136553
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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 intervalMs the interval between heartbeats.
    + */
    +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) {
    +  // Executor for the heartbeat task
    +  private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater")
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207004094
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserializer for peakMemoryMetrics: convert map to ExecutorMetrics */
    +private[spark] class ExecutorMetricsJsonDeserializer
    +  extends JsonDeserializer[Option[ExecutorMetrics]] {
    --- End diff --
    
    nit: double indent the `extends` line


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #92275 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92275/testReport)** for PR 21221 at commit [`812fdcf`](https://github.com/apache/spark/commit/812fdcf3961bae2a4fa20b4f60e739b45233fcd0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187823298
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,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) {
    --- End diff --
    
    I revisited the code, I think you're right. My mistake, sorry.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195533972
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1751,7 +1753,7 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    -  }
    +   }
    --- End diff --
    
    fixed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207073149
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    --- End diff --
    
    Ok, this can stay as is.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207002859
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    --- End diff --
    
    From the [Scaladoc](https://www.scala-lang.org/api/2.10.2/index.html#scala.Option):
    
    > The most idiomatic way to use an scala.Option instance is to treat it as a collection or monad and use map,flatMap, filter, or foreach
    
    It's probably better to follow the Scala conventions here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206333086
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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 = HashMap[(Int, Int), HashMap[String, ExecutorMetrics]]()
    --- End diff --
    
    Do these have to be `HashMap`s or can they be just `m,utable.Map`s? Can instantiate with `mutable.Map.empty[..., ...]()`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195773177
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    --- End diff --
    
    its hard to follow at a high level the purpose of what you're setting up here, so I'd drop in a few comments like
    
    receive 3 metric updates from each executor with just stage 0 running, with different peak updates for each executor


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94240 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94240/testReport)** for PR 21221 at commit [`a0eed11`](https://github.com/apache/spark/commit/a0eed1187e93e80803af9ee4c2010c6d90cf1d31).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722892
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    +    }
    +  }
    +}
    +/** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */
    +private[spark] class ExecutorMetricsJsonSerializer
    +  extends JsonSerializer[Option[ExecutorMetrics]] {
    --- End diff --
    
    It doesn't serialize -- nothing is added to the JSON.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209771443
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,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: ExecutorMetrics =>
    --- End diff --
    
    No, these don't need to be explicitly defined.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209716819
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,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: ExecutorMetrics =>
    --- End diff --
    
    Does the type need to be explicitly defined here and in the next line?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/90422/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195543999
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    --- End diff --
    
    Let me know if there's something else to try -- I'd also rather avoid the Map[String, Object].


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91804/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    @squito , I'm modifying ExecutorMetrics to take in the metrics array -- this will be easier for tests where we pass in set values, and seems fine for the actual code. It will check that the length of the passed in array is the same as MetricGetter.values.length. Let me know if you have any concerns.
    
    @felixcheung , I'll finish the current changes, then rebase. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198624819
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +102,48 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs[Option[Map[String, Long]]](
    +      new TypeReference[Option[Map[String, java.lang.Long]]] {})
    +    metricsMap match {
    +      case Some(metrics) =>
    +        Some(MetricGetter.values.map(m => metrics.getOrElse(m.name, 0L)).toArray)
    +      case None => None
    +    }
    +  }
    +}
    +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */
    +class PeakMemoryMetricsSerializer private[spark] extends JsonSerializer[Option[Array[Long]]] {
    --- End diff --
    
    same here


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723205
  
    --- Diff: 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)
    --- End diff --
    
    Is it likely that users would want to access the individual fields, rather than iterating through all? The 1st option would be a bit nicer if so. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198610162
  
    --- Diff: core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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
    +
    +sealed trait MetricGetter {
    --- End diff --
    
    `private[spark]` on all of these classes


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91802 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91802/testReport)** for PR 21221 at commit [`2662f6f`](https://github.com/apache/spark/commit/2662f6f9c6a7c34cea34b748f6735eb1625b73cb).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:
      * `class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] `
      * `class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #92488 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92488/testReport)** for PR 21221 at commit [`7ed42a5`](https://github.com/apache/spark/commit/7ed42a5d0eb0b93bb9ddecf14d9461c80dfe1ea0).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198624068
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    +    metricsMap match {
    +      case Some(metrics) =>
    +        Some(MetricGetter.values.map { m =>
    +          metrics.getOrElse (m.name, 0L) match {
    +            case intVal: Int => intVal.toLong
    +            case longVal: Long => longVal
    +          }
    +        }.toArray)
    +      case None => None
    +    }
    +  }
    +}
    +
    +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */
    +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] {
    +  override def serialize(
    +      metrics: Option[Array[Long]],
    +      jsonGenerator: JsonGenerator,
    +      serializerProvider: SerializerProvider): Unit = {
    +    metrics match {
    +      case Some(m) =>
    +        val metricsMap = (0 until MetricGetter.values.length).map { idx =>
    --- End diff --
    
    I don't really feel strongly.  If you don't find it helpful here, then we might as well get rid of it, its just a convenience meant for cases like this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198624800
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +102,48 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Option[Array[Long]]] {
    --- End diff --
    
    `private[spark] class ...`
    
    I'm not sure what `private[spark]` does in the place you have it -- it might mark the constructor.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187507940
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +94,10 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int),
    --- End diff --
    
    This is tracking peak metric values for executors for each stage, so that the peak values for the stage can be dumped at stage end. The purpose is to reduce the amount of logging, to only number of stages * number of executors ExecutorMetricsUpdate events.
    
    I originally tried logging for new peak values, resetting when a new stage begins -- this is simpler, but can lead to more events being logged.
    
    Having stage level information is useful for users trying to identify which stages are more memory intensive. This information could be useful they are trying to reduce the amount of memory used, since they would know which stages (and the relevant code) to focus on.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    For the other reviewers, this was merged to master (not 2.4)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195892278
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    --- End diff --
    
    Adding comments.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r210492513
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -69,6 +69,11 @@ package object config {
         .bytesConf(ByteUnit.KiB)
         .createWithDefaultString("100k")
     
    +  private[spark] val EVENT_LOG_STAGE_EXECUTOR_METRICS =
    +    ConfigBuilder("spark.eventLog.logStageExecutorMetrics.enabled")
    +      .booleanConf
    +      .createWithDefault(true)
    --- End diff --
    
    should this be "false" for now until we could test this out more, just to be on the safe side?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187238940
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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 intervalMs the interval between heartbeats.
    + */
    +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) {
    +  // Executor for the heartbeat task
    +  private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater")
    --- End diff --
    
    I'm wondering should the prefix name of heartbeater thread be `"executor-heartbeater"` ?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #95801 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95801/testReport)** for PR 21221 at commit [`571285b`](https://github.com/apache/spark/commit/571285beace1a0c1df92d9f5127828ed8955c93f).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r215899049
  
    --- Diff: core/src/main/java/org/apache/spark/SparkFirehoseListener.java ---
    @@ -103,6 +103,12 @@ public final void onExecutorMetricsUpdate(
         onEvent(executorMetricsUpdate);
       }
     
    +  @Override
    +  public final void onStageExecutorMetrics(
    +          SparkListenerStageExecutorMetrics executorMetrics) {
    --- End diff --
    
    nit: remove extra spaces for better indent


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682787
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -1922,6 +1928,12 @@ class SparkContext(config: SparkConf) extends Logging {
         Utils.tryLogNonFatalError {
           _eventLogger.foreach(_.stop())
         }
    +    if(_heartbeater != null) {
    --- End diff --
    
    Added.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92275/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91846 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91846/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/92490/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93557/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198683846
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,217 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    --- End diff --
    
    Woops, that was left over from when it was ExecutorMetricsUpdated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #90422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90422/testReport)** for PR 21221 at commit [`ad10d28`](https://github.com/apache/spark/commit/ad10d2814bbfbaf8c21fcbb1abe83ef7a8e9ffe7).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206335296
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala ---
    @@ -217,7 +218,12 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp
     
         // Verify the same events are replayed in the same order
         assert(sc.eventLogger.isDefined)
    -    val originalEvents = sc.eventLogger.get.loggedEvents
    +    val originalEvents = sc.eventLogger.get.loggedEvents.filter { e =>
    +      JsonProtocol.sparkEventFromJson(e) match {
    --- End diff --
    
    `isInstanceOf` may be cleaner here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94865 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94865/testReport)** for PR 21221 at commit [`2897281`](https://github.com/apache/spark/commit/2897281a384d25556609a17be21f926cb5d68dd6).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r196236364
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    Just discussed with @squito -- thanks! Logging -1 for timestamp is confusing and hacky. 
    
    Some items discussed:
    
    For ExecutorMetrics, timestamp can be optional, or it can be removed completely and replaced by Array[Long], with comments explaining how the metrics work.
    
    For logging, stage ID could be added as part of the executorMetrics in SparkListenerExecutorMetricsUpdate, but this is awkward, since this information isn't used as part of the Heartbeat, and only for logging. While the application is running, there could be multiple stages running when the metrics are gathered, so specifying 1 stage ID doesn't make sense. For logging, the metrics are the peak values for a particular stage, so are associated with a stage.
    
    Another option is to add the information to SparkListenerStageCompleted, but this would bloat the event if there are many executors.
    
    A third option is to create a new event, SparkListenerStageExecutorMetrics, which would have the executor ID, stage ID and attempt, and peak metrics. 
    
    I'll give the 3rd option a try, and will add details to the design doc once this is more finalized.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723114
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala ---
    @@ -217,7 +218,12 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp
     
         // Verify the same events are replayed in the same order
         assert(sc.eventLogger.isDefined)
    -    val originalEvents = sc.eventLogger.get.loggedEvents
    +    val originalEvents = sc.eventLogger.get.loggedEvents.filter { e =>
    +      JsonProtocol.sparkEventFromJson(e) match {
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191012697
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * 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 _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
    +   * values, and update if there is a new peak value.
    +   *
    +   * @param executorMetrics the executor metrics to compare
    +   * @return if there is a new peak value for any metric
    +   */
    +  def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = {
    +    var updated: Boolean = false
    +
    +    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.offHeapExecutionMemory > _offHeapExecutionMemory) {
    +      _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
    +      updated = true
    +    }
    +    if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) {
    +      _onHeapStorageMemory = executorMetrics.onHeapStorageMemory
    +      updated = true
    +    }
    +    if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) {
    +      _offHeapStorageMemory = executorMetrics.offHeapStorageMemory
    --- End diff --
    
    Thanks! This is cleaner, and will make it easier to add new metrics. It is very easy to have a copy/paste error. I can merge and make the test changes -- let me know if that sounds good, or if you'd like to make some more changes first. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91642/testReport)** for PR 21221 at commit [`7879e66`](https://github.com/apache/spark/commit/7879e66eed22cfd4dff2367c0ee3138369243711).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195289142
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +96,9 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]()
    --- End diff --
    
    map of (stageId, stageAttempt) for live stages, to peak executor metrics for the stage


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r200805499
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    We cannot expose an array of long's in a developer api (mutability).
    In addition, we cannot have users needing to reference private spark api's to understand the meaning of it - particularly when the ordering can be subject to change in subsequent versions of spark.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722674
  
    --- Diff: 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
    --- End diff --
    
    Let's stick with the current version for now, and revisit if we end up adding more metric types.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209712159
  
    --- Diff: core/src/main/scala/org/apache/spark/memory/MemoryManager.scala ---
    @@ -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
    --- End diff --
    
    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.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94715 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94715/testReport)** for PR 21221 at commit [`03cd5bc`](https://github.com/apache/spark/commit/03cd5bceddc7867a90918430b23bf9fa3771edfd).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/90613/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191494940
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * 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 _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
    +   * values, and update if there is a new peak value.
    +   *
    +   * @param executorMetrics the executor metrics to compare
    +   * @return if there is a new peak value for any metric
    +   */
    +  def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = {
    +    var updated: Boolean = false
    +
    +    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.offHeapExecutionMemory > _offHeapExecutionMemory) {
    +      _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
    +      updated = true
    +    }
    +    if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) {
    +      _onHeapStorageMemory = executorMetrics.onHeapStorageMemory
    +      updated = true
    +    }
    +    if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) {
    +      _offHeapStorageMemory = executorMetrics.offHeapStorageMemory
    --- End diff --
    
    Will do. Thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207000099
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,214 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test stage executor metrics logging functionality. This checks that peak
    +   * values from SparkListenerExecutorMetricsUpdate events during a stage are
    +   * logged in a StageExecutorMetrics event for each executor at stage completion.
    +   */
    +  private def testStageExecutorMetricsEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "stageExecutorMetrics-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected StageExecutorMetrics, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] =
    --- End diff --
    
    super nit: I'd move these expectations after the events to post, it follows a bit more naturally.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195773700
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L,
    +          Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L,
    +          Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L,
    +          Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L,
    +          Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L,
    +          Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L,
    +          Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L,
    +          Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L,
    +          Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))),
    +      createStageCompletedEvent(0),
    --- End diff --
    
    complete stage 0, and 3 more update for each executor with just stage 1 running, and another peak for exec Y


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91424/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209772320
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) {
    +          maybeUpdate(exec, now)
    +        }
    +      }
    +    }
    +  }
    +
    +  override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = {
    +    val now = System.nanoTime()
    --- End diff --
    
    The rest of the file is using System.nanoTime() -- it seems more consistent to keep it the same. Clock has getTimeMillis(), although we could always multiply by 1000, not sure if the precision would matter.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    We're going to delay on merging this until after the 2.4 branch is cut. We can include this in Spark 2.5.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187501157
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1753,9 +1766,21 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    +    heartbeater.stop()
    +  }
    +
    +  /** Reports heartbeat metrics for the driver. */
    +  private def reportHeartBeat(): Unit = {
    --- End diff --
    
    With cluster mode, including YARN, there isn't a local executor, so the metrics for the driver would not be collected. Perhaps this could be modified to skip this step for local mode.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r203489913
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    I am completely sold on the idea of enum-like.
    My main concern was around avoiding `MatchError`'s in scala and the other potential failures you elaborated on above.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187824094
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1753,9 +1766,21 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    +    heartbeater.stop()
    +  }
    +
    +  /** Reports heartbeat metrics for the driver. */
    +  private def reportHeartBeat(): Unit = {
    --- End diff --
    
    > With cluster mode, including YARN, there isn't a local executor, so the metrics for the driver would not be collected.
    
    Yes. But the problem is can we use `executor`'s  `getCurrentExecutorMetrics()` method for collecting memory metrics for `driver` ? IIRC, `driver`  do not acqurie memory from execution memory pool at least.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195542491
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -304,6 +305,11 @@ class SparkContext(config: SparkConf) extends Logging {
         _dagScheduler = ds
       }
     
    +  private[spark] def heartbeater: Heartbeater = _heartbeater
    +  private[spark] def heartbeater_=(hb: Heartbeater): Unit = {
    --- End diff --
    
    These aren't used -- I'll remove.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195955081
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    I can see how this would work, but it also seems far more confusing than necessary.  My understanding was that you'd always log the last timestamp which replaced the peak value for *any* metric.  Are you ever logging something other than -1 for the timestamp?  If not, we just shouldn't put any timestamp in the log.
    
    It might be helpful to step back a bit , and rather than focusing on the mechanics of what you're doing now, discuss the desired end behavior in the history server and the live UI based on the timestamp.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190351033
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +183,35 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 executor
    --- End diff --
    
    I'd add a comment here that this will log metrics for all executors that were alive while the stage was running, whether or not they ran any tasks for that stage (I think that's what it will do here, right?)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #90613 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90613/testReport)** for PR 21221 at commit [`10ed328`](https://github.com/apache/spark/commit/10ed328bfcf160711e7619aac23472f97bf1c976).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r203319952
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    +1 on enum's @squito !
    The only concern would be evolving the enum's in a later release - changing enum could result in source incompatibility.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195892263
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    --- End diff --
    
    That works and is much cleaner, thanks!


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91642/testReport)** for PR 21221 at commit [`7879e66`](https://github.com/apache/spark/commit/7879e66eed22cfd4dff2367c0ee3138369243711).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:
      * `sealed trait MetricGetter `
      * `abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter `
      * `abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #90613 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90613/testReport)** for PR 21221 at commit [`10ed328`](https://github.com/apache/spark/commit/10ed328bfcf160711e7619aac23472f97bf1c976).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91846 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91846/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    @squito , I've replaced PeakMemoryMetrics with just an Array[Long], and added a custom serializer an deserializer for it.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191008473
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    --- End diff --
    
    Changed for both.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198613341
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -264,6 +282,11 @@ private[spark] trait SparkListenerInterface {
        */
       def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit
     
    +  /**
    +   * Called when the driver reads stage executor metrics from the history log.
    --- End diff --
    
    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.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198618559
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,29 @@ private[spark] class AppStatusListener(
             }
           }
         }
    +    event.executorUpdates.foreach { updates: Array[Long] =>
    +      // check if there is a new peak value for any of the executor level memory metrics
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        if (exec.peakExecutorMetrics.compareAndUpdate(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
    --- End diff --
    
    do you need this here *and* in `onExecutorMetricsUpdate`?  I guess you do, because one is for reading from the logs, and the other is for the live UI?  If so its worth putting in a comment about that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Jenkins, retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94759 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94759/testReport)** for PR 21221 at commit [`a14b82a`](https://github.com/apache/spark/commit/a14b82a39fa00c43fa60c245f62a4fb0c154bd9a).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206334599
  
    --- Diff: core/src/main/scala/org/apache/spark/status/LiveEntity.scala ---
    @@ -302,10 +305,10 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
           Option(removeReason),
           executorLogs,
           memoryMetrics,
    -      blacklistedInStages)
    +      blacklistedInStages,
    +      if (peakExecutorMetrics.isSet()) Some(peakExecutorMetrics) else None)
    --- End diff --
    
    `Some(peakExecutorMetrics).filter(_.isSet)`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190346570
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -81,7 +84,7 @@ private[spark] class EventLoggingListener(
       private val compressionCodecName = compressionCodec.map { c =>
         CompressionCodec.getShortName(c.getClass.getName)
       }
    -
    +logInfo("spark.eventLog.logExecutorMetricsUpdates.enabled is " + shouldLogExecutorMetricsUpdates)
    --- End diff --
    
    doesn't really seem necessary at all, definitely not at INFO level (and indentation is wrong).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    probably need to be rebased


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r203122722
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    Adding getMetricValue() would abstract away the array implementation. Should MetricGetter/ExecutorMetricType be public?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723141
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,214 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test stage executor metrics logging functionality. This checks that peak
    +   * values from SparkListenerExecutorMetricsUpdate events during a stage are
    +   * logged in a StageExecutorMetrics event for each executor at stage completion.
    +   */
    +  private def testStageExecutorMetricsEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "stageExecutorMetrics-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected StageExecutorMetrics, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] =
    --- End diff --
    
    Moved to after the replay.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206312790
  
    --- Diff: core/src/main/scala/org/apache/spark/memory/MemoryManager.scala ---
    @@ -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
    --- End diff --
    
    Do these have to be `synchronized`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    @edwinalu - this can merge now that Spark 2.4's release branch has been cut, but there's conflicting files now. Can we clear the conflicts and then we can merge this?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r205095575
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,215 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test stage executor metrics logging functionality. This checks that peak
    +   * values from SparkListenerExecutorMetricsUpdate events during a stage are
    +   * logged in a StageExecutorMetrics event for each executor at stage completion.
    +   */
    +  private def testStageExecutorMetricsEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "stageExecutorMetrics-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected StageExecutorMetrics, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] =
    +      Map(
    +        ((0, "1"),
    +          new SparkListenerStageExecutorMetrics("1", 0, 0,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          new SparkListenerStageExecutorMetrics("2", 0, 0,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          new SparkListenerStageExecutorMetrics("1", 1, 0,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          new SparkListenerStageExecutorMetrics("2", 1, 0,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      // receive 3 metric updates from each executor with just stage 0 running,
    +      // with different peak updates for each executor
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 5, 7
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      // now start stage 1, one more metric update for each executor, and new
    +      // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks
    +      createStageSubmittedEvent(1),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7
    --- End diff --
    
    Stage 0 is still running, and these are new peaks for that stage. It is also initializing all the stage 1 metric values, since these are the first executor metrics seen for stage 1 (I'll add this to the comments).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Jenkins, test this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190353891
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -209,6 +210,16 @@ class DAGScheduler(
       private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this)
       taskScheduler.setDAGScheduler(this)
     
    +  /** driver heartbeat for collecting metrics */
    +  private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater",
    --- End diff --
    
    lets not put this in the DAGScheduler please -- this class is fragile enough as it is :)
    
    I think this should just go in SparkContext.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191007795
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -209,6 +210,16 @@ class DAGScheduler(
       private[scheduler] val eventProcessLoop = new DAGSchedulerEventProcessLoop(this)
       taskScheduler.setDAGScheduler(this)
     
    +  /** driver heartbeat for collecting metrics */
    +  private val heartbeater: Heartbeater = new Heartbeater(reportHeartBeat, "driver-heartbeater",
    --- End diff --
    
    Moved.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207016169
  
    --- Diff: 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
    --- End diff --
    
    Yup, on my end this is a low conviction suggestion - we might start feeling pain around this as we add more metric types, but for a first pass this is probably fine.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195539837
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // clear out any previous attempts, that did not have a stage completed event
    --- End diff --
    
    Tracking task start and end would be some amount of overhead. If it's a relatively unlikely corner case, and unlikely to have much impact on the numbers, it may be better to leave as is. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195751572
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    --- End diff --
    
    ok I remember now -- the problem is that `classOf` just has the erased type.  But jackson has something to get around this, with its `TypeReference`:
    
    ```scala
    val json = """{"a":1,"b":2147483648}"""
    val parsedWithTRef = mapper.readValue[Option[Map[String, Long]]](json, new TypeReference[Option[Map[String, java.lang.Long]]] {})
    val parsedWithClass = mapper.readValue(json, classOf[Option[Map[String, java.lang.Long]]])
    scala> parsedWithTRef.get.get("a").get
    res26: Long = 1
    scala> parsedWithClass.get.get("a").get
    java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Long
      ... 48 elided
    ```
    
    I think there is another scala-specific api to avoid repeating the type twice ... but also wouldn't really worry about it


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190363971
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    +        case _ =>
    +          checkEvent(lines(i), event)
    +          i += 1
    +        }
    +      }
    +    } finally {
    +      logData.close()
    +    }
    +  }
    +
    +  /** Create a stage submitted event for the specified stage Id. */
    +  private def createStageSubmittedEvent(stageId: Int) = {
    +    SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create a stage completed event for the specified stage Id. */
    +  private def createStageCompletedEvent(stageId: Int) = {
    +    SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorAddedEvent(executorId: Int) = {
    +    SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorRemovedEvent(executorId: Int) = {
    +    SparkListenerExecutorRemoved(0L, executorId.toString, "test")
    +  }
    +
    +  /** Create an executor metrics update event, with the specified executor metrics values. */
    +  private def createExecutorMetricsUpdateEvent(
    +      executorId: Int,
    +      executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = {
    +    val taskMetrics = TaskMetrics.empty
    +    taskMetrics.incDiskBytesSpilled(111)
    +    taskMetrics.incMemoryBytesSpilled(222)
    +    val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo)))
    +    SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics))
    +  }
    +
    +  /** Check that the two ExecutorMetrics match */
    +  private def checkExecutorMetrics(
    +      executorMetrics1: Option[ExecutorMetrics],
    +      executorMetrics2: Option[ExecutorMetrics]) = {
    +    (executorMetrics1, executorMetrics2) match {
    +      case (Some(e1), Some(e2)) =>
    +        assert(e1.timestamp === e2.timestamp)
    +        assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory)
    +        assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory)
    +        assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory)
    +        assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory)
    +        assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory)
    +        assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory)
    +        assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory)
    +        assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory)
    +        assert(e1.directMemory === e2.directMemory)
    +        assert(e1.mappedMemory === e2.mappedMemory)
    +      case (None, None) =>
    +      case _ =>
    +        assert(false)
    +    }
    +  }
    +
    +  /** Check that the Spark history log line matches the expected event. */
    +  private def checkEvent(line: String, event: SparkListenerEvent): Unit = {
    +    assert(line.contains(event.getClass.toString.split("\\.").last))
    +    event match {
    +      case executorMetrics: SparkListenerExecutorMetricsUpdate =>
    +        JsonProtocol.sparkEventFromJson(parse(line)) match {
    --- End diff --
    
    you can pull `JsonProtocol.sparkEventFromJson(parse(line))` out to avoid repeating, along with the type comparison.
    
    ```scala
    val parsed = JsonProtocol.sparkEventFromJson(parse(line))
    assert(parsed.getClass === event.getClass)
    event match {
     ...
    ```
    
    (also `assertTypeError` does something else entirely: http://doc.scalatest.org/2.2.6/#org.scalatest.Assertions)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187507139
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no
    + * values have been recorded yet.
    + */
    +private[spark] class PeakExecutorMetrics {
    --- End diff --
    
    I got some errors when trying to add methods to ExecutorMetrics. I don't remember the details, but can try this again.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206333224
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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()),
    +        new HashMap[String, ExecutorMetrics]())
    --- End diff --
    
    `mutable.Map.empty[String, ExecutorMetrics]` if the above signature is changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195534314
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +96,9 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = HashMap[(Int, Int), HashMap[String, PeakExecutorMetrics]]()
    --- End diff --
    
    modified.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207006413
  
    --- Diff: 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
    --- End diff --
    
    I appreciate the comparison with this alternative, but I don't see a big problem w/ the initial version.  If there were many different types for `T` that were really different, then I think I'd be more inclined to do that -- might still come to that as more types are added.
    
    (I'm also the one who suggested the current version, so I'm biased)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94842 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94842/testReport)** for PR 21221 at commit [`a14b82a`](https://github.com/apache/spark/commit/a14b82a39fa00c43fa60c245f62a4fb0c154bd9a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209770605
  
    --- Diff: 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)
    --- End diff --
    
    Added.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94759 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94759/testReport)** for PR 21221 at commit [`a14b82a`](https://github.com/apache/spark/commit/a14b82a39fa00c43fa60c245f62a4fb0c154bd9a).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #90422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90422/testReport)** for PR 21221 at commit [`ad10d28`](https://github.com/apache/spark/commit/ad10d2814bbfbaf8c21fcbb1abe83ef7a8e9ffe7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Jenkins, retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r188136532
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1753,9 +1766,21 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    +    heartbeater.stop()
    +  }
    +
    +  /** Reports heartbeat metrics for the driver. */
    +  private def reportHeartBeat(): Unit = {
    --- End diff --
    
    It's a bit redundant for fields that aren't used by the driver -- for the driver, execution memory gets set to 0.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94240/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #93557 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93557/testReport)** for PR 21221 at commit [`20799d2`](https://github.com/apache/spark/commit/20799d2af7b70334534be913f7defea6d6b79ffb).
     * This patch **fails Spark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195291809
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -304,6 +305,11 @@ class SparkContext(config: SparkConf) extends Logging {
         _dagScheduler = ds
       }
     
    +  private[spark] def heartbeater: Heartbeater = _heartbeater
    +  private[spark] def heartbeater_=(hb: Heartbeater): Unit = {
    --- End diff --
    
    I don't think you're using this getter and setter at all?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206334329
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    --- End diff --
    
    Don't case compare with `Some` and `None` - for options prefer the functional equivalents and occasionally can use isEmpty / isPresent. We can do this here:
    
    ```
    val opt = liveExecutors.get(...).orElse(...)
    opt.foreach { ...}
    if (opt.isEmpty) {
      logWarning(...)
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209715001
  
    --- Diff: core/src/main/scala/org/apache/spark/memory/MemoryManager.scala ---
    @@ -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
    +
    +  /**
    +   *  Off heap execution memory currently in use, in bytes.
    +   */
    +  final def offHeapExecutionMemoryUsed: Long = offHeapExecutionMemoryPool.memoryUsed
    --- End diff --
    
    `synchronized` here also and in the below two methods.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206331772
  
    --- Diff: 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
    --- End diff --
    
    A few minor flags with this design could be:
    - When passing the memory manager to the bean-sourced metrics provider, the memory manager ends up being unused
    - There's an inconsistency with the fact that the metric types that use the memory manager accept the memory manager as an argument, but the metric types that use the memory beans construct the beans as singletons.
    
    The following is a proposal that doesn't have the above inconsistencies, but is a bit more complex. Open to discussion on if the extra complexity is worthwhile here.
    
    ```
    sealed trait ExecutorMetricType[T] {
      private[spark] def getMetricValue(src: T): Long
      private[spark] val name = ...
    }
    
    case class MBeanMetricType(mbeanName: String) extends ExecutorMetricType[BufferPoolMXBean] {
      override def name(): String = // derive some name from the bean name, most likely
      override def getMetricValue(bean: BufferPoolMXBean): Long = {
        bean.getMemoryUsed
      }
    }
    
    case object OnHeapExecutionMemory extends ExecutorMetricType[MemoryManager] {
      override def name(): String = "OnHeapExecution"
      override def getMetricValue(memoryManager: MemoryManager): Long = memoryManager. onHeapExecutionMemoryUsed
    }
    
    private[spark] object ExecutorMetricType {
      val memoryManagerMetrics = IndexedSeq(
          OnHeapExecutionMemory,
          OffHeapExecutionMemory,
          // Add all subtypes of ExecutorMetricType[MemoryManager] here.
        )
      private val BUFFER_POOL_BEAN_NAME = "java.nio:type=BufferPool,name=direct"
      private val MAPPED_POOL_BEAN_NAME = "java.nio:type=BufferPool,name=mapped"
      val mbeanMetrics = IndexedSeq(
            (MBeanMetricType(BUFFER_POOL_BEAN_NAME), getBean(BUFFER_POOL_BEAN_NAME),
            MBeanMetricType(MAPPED_POOL_BEAN_NAME), getBean(MAPPED_POOL_BEAN_NAME))
    
      val values = memoryManagerMetrics ++ mBeanMetrics.map(_._1) // Just for length? There might be a simpler way
    
      def getMetricsSummary(memoryManager: MemoryManager): ExecutorMetrics = {
        val allMetrics = new Array[Long](values.length)
        memoryManagerMetrics.zipWithIndex.forEach { case (index, metric) => allMetrics(index) = metrics.getMetricValue(memoryManager)
        
        mbeanMetrics.zipWithIndex.foreach { case (index, (metric, bean)) => allMetrics(index + memoryManagerMetrics.length) = metric.getMetricValue(bean)
        return new ExecutorMetrics(allMetrics)
      }
    
      private def getBean(beanName: String): BufferPoolMxBean = {
        ManagementFactory.newPlatformMXBeanProxy(
          ManagementFactory.getPlatformMBeanServer,
          new ObjectName(beanName).toString,
          classOf[BufferPoolMXBean])
      }
    }


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195770816
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no
    + * values have been recorded yet.
    + */
    +private[spark] class PeakExecutorMetrics {
    --- End diff --
    
    can you revisit this given the other refactoring that has taken place?
    
    and if you do need this extra class, please include a comment here explaining the `metrics` array and referencing `MetricGetter`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209770404
  
    --- Diff: core/src/main/scala/org/apache/spark/memory/MemoryManager.scala ---
    @@ -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
    --- End diff --
    
    That's true -- I'll change the methods to synchronized.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682980
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,29 @@ private[spark] class AppStatusListener(
             }
           }
         }
    +    event.executorUpdates.foreach { updates: Array[Long] =>
    +      // check if there is a new peak value for any of the executor level memory metrics
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        if (exec.peakExecutorMetrics.compareAndUpdate(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
    --- End diff --
    
    Unfortunately, yes. I've added some comments.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94842/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723165
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    +         if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) {
    +          maybeUpdate(exec, now)
    --- End diff --
    
    Yes, this is called on replay. I've removed the "maybeUpdate".


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91811 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91811/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91424 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91424/testReport)** for PR 21221 at commit [`7879e66`](https://github.com/apache/spark/commit/7879e66eed22cfd4dff2367c0ee3138369243711).
     * This patch **fails to build**.
     * This patch **does not merge cleanly**.
     * This patch adds the following public classes _(experimental)_:
      * `sealed trait MetricGetter `
      * `abstract class MemoryManagerMetricGetter(f: MemoryManager => Long) extends MetricGetter `
      * `abstract class MBeanMetricGetter(mBeanName: String) extends MetricGetter `


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206312334
  
    --- Diff: 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)
    --- End diff --
    
    Out of curiosity, why are we using an array here with index-based fetching? We could use a struct / case class to represent these metrics. But I suppose the size of the payload we send is smaller if we use an Array, and we don't want to pay serialization costs?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723098
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -691,7 +723,19 @@ 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") match {
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195289072
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1751,7 +1753,7 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    -  }
    +   }
    --- End diff --
    
    nit: old indentation was right


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #93557 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93557/testReport)** for PR 21221 at commit [`20799d2`](https://github.com/apache/spark/commit/20799d2af7b70334534be913f7defea6d6b79ffb).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/90087/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #93570 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93570/testReport)** for PR 21221 at commit [`8905d23`](https://github.com/apache/spark/commit/8905d231c3a959f70266223d3546b17a655cee39).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r196111195
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    there is no point in logging the timestamp, if we only log -1.  Better to just remove it.  Are you doing anything with it in the live UI?  Or should we just get rid of the timestamp field entirely?  (it can always be added later if there is a use for it.)
    
    I agree that having one timestamp for the peak across all metrics isn't very useful.  Its possible *all* the metrics would hit a peak at nearly the same time, but the more metrics we add, the less likely that becomes.  And even if we had the timestamp for each metric, what would we do with it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r210492311
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -216,8 +217,7 @@ private[spark] class Executor(
     
       def stop(): Unit = {
         env.metricsSystem.report()
    -    heartbeater.shutdown()
    -    heartbeater.awaitTermination(10, TimeUnit.SECONDS)
    +    heartbeater.stop()
    --- End diff --
    
    future: `try {} catch {  case NonFatal(e)`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187825561
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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 intervalMs the interval between heartbeats.
    + */
    +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) {
    +  // Executor for the heartbeat task
    +  private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater")
    --- End diff --
    
    "pass in the name to the constructor" is better(if we do need to do this for the driver)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682779
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.internal.Logging
    +import org.apache.spark.memory.MemoryManager
    +import org.apache.spark.metrics.MetricGetter
    +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. */
    +  private[spark] def start(): Unit = {
    --- End diff --
    
    Removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206334970
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    +    }
    +  }
    +}
    +/** serializer for peakMemoryMetrics: convert ExecutorMetrics to map with metric name as key */
    +private[spark] class ExecutorMetricsJsonSerializer
    +  extends JsonSerializer[Option[ExecutorMetrics]] {
    --- End diff --
    
    If this is empty does it serialize as `null` or does it not serialize at all? Or does it serialize some other token like `empty`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190364971
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    +        case _ =>
    +          checkEvent(lines(i), event)
    +          i += 1
    +        }
    +      }
    +    } finally {
    +      logData.close()
    +    }
    +  }
    +
    +  /** Create a stage submitted event for the specified stage Id. */
    +  private def createStageSubmittedEvent(stageId: Int) = {
    +    SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create a stage completed event for the specified stage Id. */
    +  private def createStageCompletedEvent(stageId: Int) = {
    +    SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorAddedEvent(executorId: Int) = {
    +    SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorRemovedEvent(executorId: Int) = {
    +    SparkListenerExecutorRemoved(0L, executorId.toString, "test")
    +  }
    +
    +  /** Create an executor metrics update event, with the specified executor metrics values. */
    +  private def createExecutorMetricsUpdateEvent(
    +      executorId: Int,
    +      executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = {
    +    val taskMetrics = TaskMetrics.empty
    +    taskMetrics.incDiskBytesSpilled(111)
    +    taskMetrics.incMemoryBytesSpilled(222)
    +    val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo)))
    +    SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics))
    +  }
    +
    +  /** Check that the two ExecutorMetrics match */
    +  private def checkExecutorMetrics(
    +      executorMetrics1: Option[ExecutorMetrics],
    +      executorMetrics2: Option[ExecutorMetrics]) = {
    +    (executorMetrics1, executorMetrics2) match {
    +      case (Some(e1), Some(e2)) =>
    +        assert(e1.timestamp === e2.timestamp)
    +        assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory)
    +        assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory)
    +        assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory)
    +        assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory)
    +        assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory)
    +        assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory)
    +        assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory)
    +        assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory)
    +        assert(e1.directMemory === e2.directMemory)
    +        assert(e1.mappedMemory === e2.mappedMemory)
    +      case (None, None) =>
    +      case _ =>
    +        assert(false)
    +    }
    +  }
    +
    +  /** Check that the Spark history log line matches the expected event. */
    +  private def checkEvent(line: String, event: SparkListenerEvent): Unit = {
    +    assert(line.contains(event.getClass.toString.split("\\.").last))
    +    event match {
    +      case executorMetrics: SparkListenerExecutorMetricsUpdate =>
    --- End diff --
    
    you're never using this w/ `SparkListenerExecutorMetricsUpdate`, right?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95801/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209714796
  
    --- Diff: 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: Boolean = false
    +
    +    (0 until ExecutorMetricType.values.length).foreach { idx =>
    +       if ( executorMetrics.metrics(idx) > metrics(idx)) {
    --- End diff --
    
    Nit: No space after the left bracket.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190995781
  
    --- Diff: core/src/main/scala/org/apache/spark/executor/Executor.scala ---
    @@ -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 = {
    --- End diff --
    
    Yes, and easier to share the code between driver and executor. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206995945
  
    --- Diff: 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)
    --- End diff --
    
    I suggested this earlier in the reviews.  Most of the operations for dealing with this data want to iterate over all the fields.  its much easier this way vs. having a bazillion
    
    ```scala
    if (x.fizz > y.fizz) { 
      y.fizz = x.fizz
    }
    if (x.buzz > y.buzz) {
      y.buzz = x.buzz
    }
    ...
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95776/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187248156
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -93,6 +94,10 @@ private[spark] class EventLoggingListener(
       // Visible for tests only.
       private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName)
     
    +  // map of live stages, to peak executor metrics for the stage
    +  private val liveStageExecutorMetrics = mutable.HashMap[(Int, Int),
    --- End diff --
    
    Why we should track executor's memory metrics for each stage?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722773
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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()),
    +        new HashMap[String, ExecutorMetrics]())
    --- End diff --
    
    Modified.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91811/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by rezasafi <gi...@git.apache.org>.
Github user rezasafi commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r204976606
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,215 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test stage executor metrics logging functionality. This checks that peak
    +   * values from SparkListenerExecutorMetricsUpdate events during a stage are
    +   * logged in a StageExecutorMetrics event for each executor at stage completion.
    +   */
    +  private def testStageExecutorMetricsEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "stageExecutorMetrics-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected StageExecutorMetrics, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] =
    +      Map(
    +        ((0, "1"),
    +          new SparkListenerStageExecutorMetrics("1", 0, 0,
    +              Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          new SparkListenerStageExecutorMetrics("2", 0, 0,
    +              Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          new SparkListenerStageExecutorMetrics("1", 1, 0,
    +              Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          new SparkListenerStageExecutorMetrics("2", 1, 0,
    +              Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      // receive 3 metric updates from each executor with just stage 0 running,
    +      // with different peak updates for each executor
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 5, 7
    +      createExecutorMetricsUpdateEvent(1,
    +          Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8
    +      createExecutorMetricsUpdateEvent(2,
    +          Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      // now start stage 1, one more metric update for each executor, and new
    +      // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks
    +      createStageSubmittedEvent(1),
    +      // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7
    --- End diff --
    
    Are this comment and the one in line 322 correct? Shouldn't it say stage 1?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by gatorsmile <gi...@git.apache.org>.
Github user gatorsmile commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    @mccheah When you merged the code, could you also leave the comments about which branches you did the merge? 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/94865/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195538848
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 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.metrics)
    --- End diff --
    
    We need to pass in a value for timestamp, but there isn't really one for the peak metrics, since times for each peak could be different. 
    
    When processing, -1 will help indicate that the event is coming from the history log, and contains the peak values for the stage that is just ending. When updating the stage executor peaks (peak executor values stored for each active stage), we can replace all of the peak executor metric values instead of updating with the max of current and new values for each metric. 
    
    As an example, suppose there is the following scenario:
    T1: start of stage 1
    T2: peak value of 1000 for metric m1
    T3: start of stage 2
    T4: stage 1 ends, and peak metric values for stage 1 are logged, including m1=1000
    T5: stage 2 ends, and peak metric values for stage 2 are logged.
    
    If values for m1 are < 1000 between T3 (start of stage 2) and T5 (end of stage 2), and say that the highest value for m1 during that period is 500, then we want the peak value for m1 for stage 2 to show as 500.
    
    There would be an ExecutorMetricsUpdate event logged (and then read) at T4 (end of stage 1), with m1=1000, which is after T3 (start of stage 2). If when reading the history log, we set the stage 2 peakExecutorMetrics to the max of current or new values from ExecutorMetricsUpdate, then the value for stage 2 would remain at 1000. However, we want it to be replaced by the value of 500 when it gets the ExecutorMetricsUpdate logged at T5 (end of stage 2). During processing of ExecutorMetricsUpdate, for the stage level metrics, it will replace all the peakExecutorMetrics if timestamp is -1.
    
    I can add some comments for this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206334704
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    +    }
    +  }
    +}
    +/** 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 match {
    +      case Some(m) =>
    --- End diff --
    
    Don't use `Some` and `None` case matching - prefer `foreach`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195291213
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    +    metricsMap match {
    +      case Some(metrics) =>
    +        Some(MetricGetter.values.map { m =>
    +          metrics.getOrElse (m.name, 0L) match {
    +            case intVal: Int => intVal.toLong
    +            case longVal: Long => longVal
    +          }
    +        }.toArray)
    +      case None => None
    +    }
    +  }
    +}
    +
    +/** serializer for peakMemoryMetrics: convert array to map with metric name as key */
    +class PeakMemoryMetricsSerializer extends JsonSerializer[Option[Array[Long]]] {
    +  override def serialize(
    +      metrics: Option[Array[Long]],
    +      jsonGenerator: JsonGenerator,
    +      serializerProvider: SerializerProvider): Unit = {
    +    metrics match {
    +      case Some(m) =>
    +        val metricsMap = (0 until MetricGetter.values.length).map { idx =>
    --- End diff --
    
    ```
    MetricGetter.idxAndValues.map { case (idx, getter) =>
      getter.name -> m(idx)
    }
    ```
    
    (or maybe we can get rid of `idxAndValues` if it doesn't really help ...)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198625661
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,217 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    --- End diff --
    
    i think you mean *executor* metrics, not *task* metrics, right?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722865
  
    --- Diff: core/src/main/scala/org/apache/spark/status/LiveEntity.scala ---
    @@ -302,10 +305,10 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE
           Option(removeReason),
           executorLogs,
           memoryMetrics,
    -      blacklistedInStages)
    +      blacklistedInStages,
    +      if (peakExecutorMetrics.isSet()) Some(peakExecutorMetrics) else None)
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187236701
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala ---
    @@ -1753,9 +1766,21 @@ class DAGScheduler(
         messageScheduler.shutdownNow()
         eventProcessLoop.stop()
         taskScheduler.stop()
    +    heartbeater.stop()
    +  }
    +
    +  /** Reports heartbeat metrics for the driver. */
    +  private def reportHeartBeat(): Unit = {
    --- End diff --
    
    Why we need this for `driver` ? If spark run in local mode, there's a local `executor`, which will report heartbeat.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195772234
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    --- End diff --
    
    it looks to me like you are not checking that you actually get everything in `expectedMetricsEvents` -- is that right?  can you add that check?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207004013
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,34 @@ 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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        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)) match {
    +      case Some(exec) =>
    +         if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) {
    +          maybeUpdate(exec, now)
    --- End diff --
    
    this is only for replaying from logs, right?  but then, `maybeUpdate` will be a no-op, as `live` is false.  So, what is the desired effect here?  But maybe I'm missing something, because this is covered by the some of the api tests.  I think maybe you don't need to call update at all here, and it'll just get written to the kvstore as part of the final `flush` call when the kvstore is closed after reading the whole log.
    
    also style nit: indentation is off


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209714883
  
    --- Diff: 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: Boolean = false
    --- End diff --
    
    No need to specifically label this as `Boolean`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195892271
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    --- End diff --
    
    Yes, changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206333488
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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 = new HashMap[String, CompressionCodec]
    --- End diff --
    
    `mutable.Map.empty` - prefer this everywhere.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91424 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91424/testReport)** for PR 21221 at commit [`7879e66`](https://github.com/apache/spark/commit/7879e66eed22cfd4dff2367c0ee3138369243711).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209773404
  
    --- Diff: core/src/main/scala/org/apache/spark/status/AppStatusListener.scala ---
    @@ -669,6 +686,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: ExecutorMetrics =>
    +      liveExecutors.get(event.execId).foreach { exec: LiveExecutor =>
    +        if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(updates)) {
    +          maybeUpdate(exec, now)
    +        }
    +      }
    +    }
    +  }
    +
    +  override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = {
    +    val now = System.nanoTime()
    --- End diff --
    
    Yup that's fine.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91804 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91804/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191012183
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    +        case _ =>
    +          checkEvent(lines(i), event)
    +          i += 1
    +        }
    +      }
    +    } finally {
    +      logData.close()
    +    }
    +  }
    +
    +  /** Create a stage submitted event for the specified stage Id. */
    +  private def createStageSubmittedEvent(stageId: Int) = {
    +    SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create a stage completed event for the specified stage Id. */
    +  private def createStageCompletedEvent(stageId: Int) = {
    +    SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorAddedEvent(executorId: Int) = {
    +    SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorRemovedEvent(executorId: Int) = {
    +    SparkListenerExecutorRemoved(0L, executorId.toString, "test")
    +  }
    +
    +  /** Create an executor metrics update event, with the specified executor metrics values. */
    +  private def createExecutorMetricsUpdateEvent(
    +      executorId: Int,
    +      executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = {
    +    val taskMetrics = TaskMetrics.empty
    +    taskMetrics.incDiskBytesSpilled(111)
    +    taskMetrics.incMemoryBytesSpilled(222)
    +    val accum = Array((333L, 1, 1, taskMetrics.accumulators().map(AccumulatorSuite.makeInfo)))
    +    SparkListenerExecutorMetricsUpdate(executorId.toString, accum, Some(executorMetrics))
    +  }
    +
    +  /** Check that the two ExecutorMetrics match */
    +  private def checkExecutorMetrics(
    +      executorMetrics1: Option[ExecutorMetrics],
    +      executorMetrics2: Option[ExecutorMetrics]) = {
    +    (executorMetrics1, executorMetrics2) match {
    +      case (Some(e1), Some(e2)) =>
    +        assert(e1.timestamp === e2.timestamp)
    +        assert(e1.jvmUsedHeapMemory === e2.jvmUsedHeapMemory)
    +        assert(e1.jvmUsedNonHeapMemory === e2.jvmUsedNonHeapMemory)
    +        assert(e1.onHeapExecutionMemory === e2.onHeapExecutionMemory)
    +        assert(e1.offHeapExecutionMemory === e2.offHeapExecutionMemory)
    +        assert(e1.onHeapStorageMemory === e2.onHeapStorageMemory)
    +        assert(e1.offHeapStorageMemory === e2.offHeapStorageMemory)
    +        assert(e1.onHeapUnifiedMemory === e2.onHeapUnifiedMemory)
    +        assert(e1.offHeapUnifiedMemory === e2.offHeapUnifiedMemory)
    +        assert(e1.directMemory === e2.directMemory)
    +        assert(e1.mappedMemory === e2.mappedMemory)
    +      case (None, None) =>
    +      case _ =>
    +        assert(false)
    +    }
    +  }
    +
    +  /** Check that the Spark history log line matches the expected event. */
    +  private def checkEvent(line: String, event: SparkListenerEvent): Unit = {
    +    assert(line.contains(event.getClass.toString.split("\\.").last))
    +    event match {
    +      case executorMetrics: SparkListenerExecutorMetricsUpdate =>
    +        JsonProtocol.sparkEventFromJson(parse(line)) match {
    --- End diff --
    
    Thanks, modified. 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198628058
  
    --- Diff: project/MimaExcludes.scala ---
    @@ -89,7 +89,13 @@ object MimaExcludes {
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"),
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"),
         ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="),
    -    ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol")
    +    ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"),
    +
    +    // [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.apply"),
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.copy"),
    +    ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate.this"),
    +    ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate$")
    --- End diff --
    
    minor: I hear we put new ones at the top now (so you don't have to add a comma to the previous one)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r191008115
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,233 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    +      Map(
    +        ((0, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L))),
    +        ((0, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L))),
    +        ((1, "1"),
    +          createExecutorMetricsUpdateEvent(1,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L))),
    +        ((1, "2"),
    +          createExecutorMetricsUpdateEvent(2,
    +            new ExecutorMetrics(-1L, 7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))
    +
    +    // Events to post.
    +    val events = Array(
    +      SparkListenerApplicationStart("executionMetrics", None,
    +        1L, "update", None),
    +      createExecutorAddedEvent(1),
    +      createExecutorAddedEvent(2),
    +      createStageSubmittedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(10L, 4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(10L, 1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(15L, 4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(15L, 2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(20L, 2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(20L, 3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L)),
    +      createStageSubmittedEvent(1),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(25L, 5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(25L, 7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L)),
    +      createStageCompletedEvent(0),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(30L, 6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(30L, 5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(35L, 7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L)),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(35L, 5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L)),
    +      createExecutorMetricsUpdateEvent(1,
    +        new ExecutorMetrics(40L, 5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L)),
    +      createExecutorRemovedEvent(1),
    +      createExecutorMetricsUpdateEvent(2,
    +        new ExecutorMetrics(40L, 4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L)),
    +      createStageCompletedEvent(1),
    +      SparkListenerApplicationEnd(1000L))
    +
    +    // play the events for the event logger
    +    eventLogger.start()
    +    listenerBus.start(Mockito.mock(classOf[SparkContext]), Mockito.mock(classOf[MetricsSystem]))
    +    listenerBus.addToEventLogQueue(eventLogger)
    +    events.foreach(event => listenerBus.post(event))
    +    listenerBus.stop()
    +    eventLogger.stop()
    +
    +    // Verify the log file contains the expected events.
    +    // Posted events should be logged, except for ExecutorMetricsUpdate events -- these
    +    // are consolidated, and the peak values for each stage are logged at stage end.
    +    val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem)
    +    try {
    +      val lines = readLines(logData)
    +      val logStart = SparkListenerLogStart(SPARK_VERSION)
    +      assert(lines.size === 14)
    +      assert(lines(0).contains("SparkListenerLogStart"))
    +      assert(lines(1).contains("SparkListenerApplicationStart"))
    +      assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart)
    +      var i = 1
    +      events.foreach {event =>
    +        event match {
    +          case metricsUpdate: SparkListenerExecutorMetricsUpdate =>
    +          case stageCompleted: SparkListenerStageCompleted =>
    +            for (j <- 1 to 2) {
    +              checkExecutorMetricsUpdate(lines(i), stageCompleted.stageInfo.stageId,
    +                expectedMetricsEvents)
    +                i += 1
    +             }
    +            checkEvent(lines(i), event)
    +            i += 1
    +        case _ =>
    +          checkEvent(lines(i), event)
    +          i += 1
    +        }
    +      }
    +    } finally {
    +      logData.close()
    +    }
    +  }
    +
    +  /** Create a stage submitted event for the specified stage Id. */
    +  private def createStageSubmittedEvent(stageId: Int) = {
    +    SparkListenerStageSubmitted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create a stage completed event for the specified stage Id. */
    +  private def createStageCompletedEvent(stageId: Int) = {
    +    SparkListenerStageCompleted(new StageInfo(stageId, 0, stageId.toString, 0,
    +      Seq.empty, Seq.empty, "details"))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    +  private def createExecutorAddedEvent(executorId: Int) = {
    +    SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty))
    +  }
    +
    +  /** Create an executor added event for the specified executor Id. */
    --- End diff --
    
    I'll remove -- they are pretty self-explanatory.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198682809
  
    --- Diff: core/src/main/scala/org/apache/spark/metrics/MetricGetter.scala ---
    @@ -0,0 +1,88 @@
    +/*
    + * 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
    +
    +sealed trait MetricGetter {
    --- End diff --
    
    Added.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by mccheah <gi...@git.apache.org>.
Github user mccheah commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r206335138
  
    --- Diff: core/src/main/scala/org/apache/spark/util/JsonProtocol.scala ---
    @@ -691,7 +723,19 @@ 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") match {
    --- End diff --
    
    `Option.map`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195543711
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    +  override def deserialize(
    +      jsonParser: JsonParser,
    +      deserializationContext: DeserializationContext): Option[Array[Long]] = {
    +    val metricsMap = jsonParser.readValueAs(classOf[Option[Map[String, Object]]])
    --- End diff --
    
    Nope, it's still reading it as an Int, even with java.lang.Long, otherwise that would have been cleaner.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187501194
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,52 @@
    +/*
    + * 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 intervalMs the interval between heartbeats.
    + */
    +private[spark] class Heartbeater(reportHeartbeat: () => Unit, intervalMs: Long) {
    +  // Executor for the heartbeat task
    +  private val heartbeater = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-heartbeater")
    --- End diff --
    
    Yes, this could be for the driver as well, so could be misleading. I can change to "heartbeater".


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #95801 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95801/testReport)** for PR 21221 at commit [`571285b`](https://github.com/apache/spark/commit/571285beace1a0c1df92d9f5127828ed8955c93f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by felixcheung <gi...@git.apache.org>.
Github user felixcheung commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    ok to test


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198611581
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +181,28 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // 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 executorMap = liveStageExecutorMetrics.remove(
    +        (event.stageInfo.stageId, event.stageInfo.attemptNumber()))
    +      executorMap.foreach {
    +       executorEntry => {
    +          for ((executorId, peakExecutorMetrics) <- executorEntry) {
    --- End diff --
    
    couple of minor things -- I wouldn't use `executorMap` for the `Option` and then `executorEntry` for the Map.  Also, we tend to prefer using `foreach` over scala's `for` loop, the one exception is that it can clean up extra nesting when you've got a bunch of loops (which you actually do here).
    
    So I'd go with
    
    ```scala
    val executorOpt = liveStageExecutorMetrics.remove(
      (event.stageInfo.stageId, event.stageInfo.attemptNumber()))
    executorOpt.foreach { execMap =>
      execMap.foreach { case (executorId, peakExecutorMetrics) =>
        ...
      }
    }
    ```
    
    or if you want to use the for loop, use it around everything:
    
    ```scala
    val execOpt = liveStageExecutorMetrics.remove(
        (event.stageInfo.stageId, event.stageInfo.attemptNumber()))
    for {
      execMap <- execOpt
      (executorId, peakExecutorMetrics) <- execMap
    } {
      ...
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207722724
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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 = HashMap[(Int, Int), HashMap[String, ExecutorMetrics]]()
    --- End diff --
    
    Yes, mutable Maps will work, and better to go with the more generic version.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723188
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -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 = new HashMap[String, CompressionCodec]
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195541136
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -234,8 +272,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 (shouldLogExecutorMetricsUpdates) {
    +      // 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 PeakExecutorMetrics())
    +          peakMetrics.compareAndUpdate(executorUpdates)
    --- End diff --
    
    What would be the right timestamp? Peaks for different metrics could have different timestamps.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198609741
  
    --- Diff: core/src/main/scala/org/apache/spark/Heartbeater.scala ---
    @@ -0,0 +1,69 @@
    +/*
    + * 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.internal.Logging
    +import org.apache.spark.memory.MemoryManager
    +import org.apache.spark.metrics.MetricGetter
    +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. */
    +  private[spark] def start(): Unit = {
    --- End diff --
    
    nit: whole class is `private[spark]` so you don't need to add this to individual methods


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195290278
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,6 +182,31 @@ private[spark] class EventLoggingListener(
     
       // Events that trigger a flush
       override def onStageCompleted(event: SparkListenerStageCompleted): Unit = {
    +    if (shouldLogExecutorMetricsUpdates) {
    +      // clear out any previous attempts, that did not have a stage completed event
    --- End diff --
    
    one potential issue here -- even though there is a stage completed event, you can still have tasks running from stage attempt (when there is a fetch failure, all existing tasks keep running).  Those leftover tasks will effect the memory usage for other tasks which run on those executors.
    
    that said, I dunno if we can do much better here.  the alternative would be to track the task start & end events for each stage attempt.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r209770476
  
    --- Diff: 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: Boolean = false
    --- End diff --
    
    Removed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187239219
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,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) {
    --- End diff --
    
    How about `case (executorId, peakExecutorMetrics) =>` ? It would be more readable.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r190405223
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * 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 _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
    +   * values, and update if there is a new peak value.
    +   *
    +   * @param executorMetrics the executor metrics to compare
    +   * @return if there is a new peak value for any metric
    +   */
    +  def compareAndUpdate(executorMetrics: ExecutorMetrics): Boolean = {
    +    var updated: Boolean = false
    +
    +    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.offHeapExecutionMemory > _offHeapExecutionMemory) {
    +      _offHeapExecutionMemory = executorMetrics.offHeapExecutionMemory
    +      updated = true
    +    }
    +    if (executorMetrics.onHeapStorageMemory > _onHeapStorageMemory) {
    +      _onHeapStorageMemory = executorMetrics.onHeapStorageMemory
    +      updated = true
    +    }
    +    if (executorMetrics.offHeapStorageMemory > _offHeapStorageMemory) {
    +      _offHeapStorageMemory = executorMetrics.offHeapStorageMemory
    --- End diff --
    
    I know spark has this kind of code all over the place already, but I really hate how error prone it is -- way too easy for a copy paste error to result in comparing the wrong two metrics, or updating the wrong value, or forgetting to update this when another metric is added, etc.
    
    I just opened this https://github.com/edwinalu/spark/pull/1 as another way to do this that would eliminate a ton of boilerplate IMO.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r207723177
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +103,50 @@ 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,
         val usedOffHeapStorageMemory: Long,
         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 match {
    +      case Some(metrics) =>
    +        Some(new ExecutorMetrics(metrics))
    +      case None => None
    --- End diff --
    
    Changed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #91804 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/91804/testReport)** for PR 21221 at commit [`99044e6`](https://github.com/apache/spark/commit/99044e6ec0cdc1b760c57dd5b7e74349384c6a98).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #90087 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/90087/testReport)** for PR 21221 at commit [`ad10d28`](https://github.com/apache/spark/commit/ad10d2814bbfbaf8c21fcbb1abe83ef7a8e9ffe7).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r203455379
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala ---
    @@ -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[Array[Long]] = 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: Array[Long])
    --- End diff --
    
    re: compatability -- the json already uses names, so is not dependent on enum order.  we always require the same code for the values sent between the driver & executor, so that isn't a concern.  For the user accessing these values with `getMetricValue()` -- deleting an enum would be the same as deleting a field, so yeah it would break compatibility.  technically allowed for a "developerapi" but something we should avoid.  Adding enums should be OK.  If the enums are re-ordered in the spark code, but the user compiles against an older version ... I *think* it should be OK, as we'd look up the index in the actual spark code at runtime.
    
    btw, I'm using "enum" loosely, I don't actually mean a java enum, I mean more the general concept, as its implemented in the current diff.  A fixed set of constants, with a helper to get all of them in order.  We could switch to using java enums -- I actually started that (https://github.com/apache/spark/pull/21221/commits/8b74ba8fff21b499e7cc9d93f9864831aa29773e), but changed it in the end.  honestly I forget why -- I think it was because I decided I wanted scala scoping constructs and more flexible code organization or something along those lines, and java's enum didn't really buy me much more.  The `executorMetrics` field here is basically an `EnumMap`, but it can actually do primitives.  That matters more for the internal messages, than here in the public spark listener api.
    
    anyway, I think there *does* need to be some developer api which exposes all of the MetricGetter/ExecutorMetricType values.  I don't really care whether that is a java enum, or the home-grown version here.  I'm flexible on specifics, but my suggestion: an `ExecutorMetricType` enum that is a developerapi; make the SparkListener api expose the values as `executorMetrics: EnumMap<ExecutorMetricType, Long>`; internally, still use `Array[Long]`, and have `MetricGetter` contain the code which knows how to take an executorMetricType and determine its current value.  that would make the public api more self-documenting and clean, keep things efficient and clean internally, and also allow us to separate out the public list of metrics from the internal logic to determine current values, without having to play too many games with package-private access.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187506780
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,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) {
    --- End diff --
    
    The for loop (line 187) is going through the hashmap entries of executorId to peakExecutorMetrics, so there are multiple values. Could you please provide more detail for how "case (executorId, peakExecutorMetrics) =>" would work? If the for loop is OK, then I can add some comments.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #94757 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/94757/testReport)** for PR 21221 at commit [`10e7f15`](https://github.com/apache/spark/commit/10e7f15dcec0ed21bb6ec4c972f80d7db1ce21cf).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91846/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/91802/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195892287
  
    --- Diff: core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala ---
    @@ -251,6 +261,222 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit
         }
       }
     
    +  /**
    +   * Test executor metrics update logging functionality. This checks that a
    +   * SparkListenerExecutorMetricsUpdate event is added to the Spark history
    +   * log if one of the executor metrics is larger than any previously
    +   * recorded value for the metric, per executor per stage. The task metrics
    +   * should not be added.
    +   */
    +  private def testExecutorMetricsUpdateEventLogging() {
    +    val conf = getLoggingConf(testDirPath, None)
    +    val logName = "executorMetricsUpdated-test"
    +    val eventLogger = new EventLoggingListener(logName, None, testDirPath.toUri(), conf)
    +    val listenerBus = new LiveListenerBus(conf)
    +
    +    // expected ExecutorMetricsUpdate, for the given stage id and executor id
    +    val expectedMetricsEvents: Map[(Int, String), SparkListenerExecutorMetricsUpdate] =
    --- End diff --
    
    I'll add a check.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by edwinalu <gi...@git.apache.org>.
Github user edwinalu commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r198683408
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +102,48 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer private[spark] extends JsonDeserializer[Option[Array[Long]]] {
    --- End diff --
    
    This is odd, but I can't seem to comment on your earlier comment. Regarding having a serializer/deserializer, I also don't have strong feelings -- it makes it more readable, but also takes up more space in the history log.
    
    Regarding this comment, thanks, I hadn't realized the placement meant that it marked the constructor. It's meant for the class, and I'll move.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187244792
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala ---
    @@ -169,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(
    --- End diff --
    
    Do we always post a `SparkListenerStageCompleted` event for failed satges (I can't rememer clearly)? If not, I think we should clean up other attempts of the same stage here.  


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by Ngone51 <gi...@git.apache.org>.
Github user Ngone51 commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r187247534
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/PeakExecutorMetrics.scala ---
    @@ -0,0 +1,127 @@
    +/*
    + * 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.executor.ExecutorMetrics
    +import org.apache.spark.status.api.v1.PeakMemoryMetrics
    +
    +/**
    + * Records the peak values for executor level metrics. If jvmUsedHeapMemory is -1, then no
    + * values have been recorded yet.
    + */
    +private[spark] class PeakExecutorMetrics {
    --- End diff --
    
    Do we really need this class? It seems `ExecutorMetrics` can already do the same work.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    **[Test build #92490 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/92490/testReport)** for PR 21221 at commit [`8d9acdf`](https://github.com/apache/spark/commit/8d9acdf32984c0c9c621a058b45805872bb9e4c5).
     * This patch passes all tests.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #21221: [SPARK-23429][CORE] Add executor memory metrics to heart...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/21221
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/93570/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #21221: [SPARK-23429][CORE] Add executor memory metrics t...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/21221#discussion_r195297024
  
    --- Diff: core/src/main/scala/org/apache/spark/status/api/v1/api.scala ---
    @@ -98,14 +101,53 @@ 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[PeakMemoryMetricsSerializer])
    +    @JsonDeserialize(using = classOf[PeakMemoryMetricsDeserializer])
    +    val peakMemoryMetrics: Option[Array[Long]])
     
     class MemoryMetrics private[spark](
         val usedOnHeapStorageMemory: Long,
         val usedOffHeapStorageMemory: Long,
         val totalOnHeapStorageMemory: Long,
         val totalOffHeapStorageMemory: Long)
     
    +/** deserialzer for peakMemoryMetrics: convert to array ordered by metric name */
    +class PeakMemoryMetricsDeserializer extends JsonDeserializer[Option[Array[Long]]] {
    --- End diff --
    
    can this be `private[spark]`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org