This is an automated email from the ASF dual-hosted git repository.

vanzin pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new f4a17e9  [SPARK-26726] Synchronize the amount of memory used by the 
broadcast variable to the UI display
f4a17e9 is described below

commit f4a17e916b729f9dc46e859b50a416db1e37b92e
Author: 韩田田00222924 <han.tiant...@zte.com.cn>
AuthorDate: Thu Jan 31 09:17:33 2019 -0800

    [SPARK-26726] Synchronize the amount of memory used by the broadcast 
variable to the UI display
    
    …not synchronized to the UI display
    
    ## What changes were proposed in this pull request?
    The amount of memory used by the broadcast variable is not synchronized to 
the UI display.
    I added the case for BroadcastBlockId and updated the memory usage.
    
    ## How was this patch tested?
    
    We can test this patch with unit tests.
    
    Closes #23649 from httfighter/SPARK-26726.
    
    Lead-authored-by: 韩田田00222924 <han.tiant...@zte.com.cn>
    Co-authored-by: han.tiant...@zte.com.cn <han.tiant...@zte.com.cn>
    Signed-off-by: Marcelo Vanzin <van...@cloudera.com>
---
 .../apache/spark/status/AppStatusListener.scala    | 44 +++++++++++++++++-----
 .../spark/status/AppStatusListenerSuite.scala      | 18 +++++++++
 2 files changed, 53 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index f69c7dd..3089f05 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -829,6 +829,7 @@ private[spark] class AppStatusListener(
     event.blockUpdatedInfo.blockId match {
       case block: RDDBlockId => updateRDDBlock(event, block)
       case stream: StreamBlockId => updateStreamBlock(event, stream)
+      case broadcast: BroadcastBlockId => updateBroadcastBlock(event, 
broadcast)
       case _ =>
     }
   }
@@ -887,15 +888,7 @@ private[spark] class AppStatusListener(
     // Update the executor stats first, since they are used to calculate the 
free memory
     // on tracked RDD distributions.
     maybeExec.foreach { exec =>
-      if (exec.hasMemoryInfo) {
-        if (storageLevel.useOffHeap) {
-          exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta)
-        } else {
-          exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta)
-        }
-      }
-      exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta)
-      exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta)
+      updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta)
     }
 
     // Update the block entry in the RDD info, keeping track of the deltas 
above so that we
@@ -997,6 +990,39 @@ private[spark] class AppStatusListener(
     }
   }
 
+  private def updateBroadcastBlock(
+      event: SparkListenerBlockUpdated,
+      broadcast: BroadcastBlockId): Unit = {
+    val executorId = event.blockUpdatedInfo.blockManagerId.executorId
+    liveExecutors.get(executorId).foreach { exec =>
+      val now = System.nanoTime()
+      val storageLevel = event.blockUpdatedInfo.storageLevel
+
+      // Whether values are being added to or removed from the existing 
accounting.
+      val diskDelta = event.blockUpdatedInfo.diskSize * (if 
(storageLevel.useDisk) 1 else -1)
+      val memoryDelta = event.blockUpdatedInfo.memSize * (if 
(storageLevel.useMemory) 1 else -1)
+
+      updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta)
+      maybeUpdate(exec, now)
+    }
+  }
+
+  private def updateExecutorMemoryDiskInfo(
+      exec: LiveExecutor,
+      storageLevel: StorageLevel,
+      memoryDelta: Long,
+      diskDelta: Long): Unit = {
+    if (exec.hasMemoryInfo) {
+      if (storageLevel.useOffHeap) {
+        exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta)
+      } else {
+        exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta)
+      }
+    }
+    exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta)
+    exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta)
+  }
+
   private def getOrCreateStage(info: StageInfo): LiveStage = {
     val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber),
       new Function[(Int, Int), LiveStage]() {
diff --git 
a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index 356e6d1..9469a46 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -938,6 +938,24 @@ class AppStatusListenerSuite extends SparkFunSuite with 
BeforeAndAfter {
     intercept[NoSuchElementException] {
       check[StreamBlockData](stream1.name) { _ => () }
     }
+
+    // Update a BroadcastBlock.
+    val broadcast1 = BroadcastBlockId(1L)
+    listener.onBlockUpdated(SparkListenerBlockUpdated(
+      BlockUpdatedInfo(bm1, broadcast1, level, 1L, 1L)))
+
+    check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+      assert(exec.info.memoryUsed === 1L)
+      assert(exec.info.diskUsed === 1L)
+    }
+
+    // Drop a BroadcastBlock.
+    listener.onBlockUpdated(SparkListenerBlockUpdated(
+      BlockUpdatedInfo(bm1, broadcast1, StorageLevel.NONE, 1L, 1L)))
+    check[ExecutorSummaryWrapper](bm1.executorId) { exec =>
+      assert(exec.info.memoryUsed === 0)
+      assert(exec.info.diskUsed === 0)
+    }
   }
 
   test("eviction of old data") {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to