From 39ba441df441c9903b9690cfc70a8a174924a7fc Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 17 Aug 2015 16:45:22 +0800 Subject: [PATCH 01/28] spark-9104 first draft version --- .../org/apache/spark/HeartbeatReceiver.scala | 7 +- .../org/apache/spark/executor/Executor.scala | 15 +++- .../spark/executor/ExecutorMetrics.scala | 65 ++++++++++++++ .../spark/network/BlockTransferService.scala | 8 +- .../netty/NettyBlockTransferService.scala | 39 +++++++- .../apache/spark/scheduler/DAGScheduler.scala | 11 +-- .../spark/scheduler/SparkListener.scala | 4 +- .../spark/scheduler/TaskScheduler.scala | 6 +- .../spark/scheduler/TaskSchedulerImpl.scala | 6 +- .../scala/org/apache/spark/ui/SparkUI.scala | 9 +- .../apache/spark/ui/memory/MemoryPage.scala | 84 ++++++++++++++++++ .../apache/spark/ui/memory/MemoryTab.scala | 88 +++++++++++++++++++ .../apache/spark/ui/storage/StorageTab.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../apache/spark/HeartbeatReceiverSuite.scala | 7 +- .../spark/scheduler/DAGSchedulerSuite.scala | 7 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- .../client/TransportClientFactory.java | 4 + .../spark/network/server/TransportServer.java | 7 +- 20 files changed, 345 insertions(+), 30 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ee60d697d8799..2f8b8b55cafe9 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -21,7 +21,7 @@ import java.util.concurrent.{ScheduledFuture, TimeUnit} import scala.collection.mutable -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEnv, RpcCallContext} import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler._ @@ -34,6 +34,7 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} */ private[spark] case class Heartbeat( executorId: String, + executorMetrics: ExecutorMetrics, taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId) @@ -118,14 +119,14 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) context.reply(true) // Messages received from executors - case heartbeat @ Heartbeat(executorId, taskMetrics, blockManagerId) => + case heartbeat @ Heartbeat(executorId, executorMetrics, taskMetrics, blockManagerId) => if (scheduler != null) { if (executorLastSeen.contains(executorId)) { executorLastSeen(executorId) = clock.getTimeMillis() eventLoopThread.submit(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { val unknownExecutor = !scheduler.executorHeartbeatReceived( - executorId, taskMetrics, blockManagerId) + executorId, executorMetrics, taskMetrics, blockManagerId) val response = HeartbeatResponse(reregisterBlockManager = unknownExecutor) context.reply(response) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5d78a9dc8885e..0b4468a9af1bf 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -85,6 +85,10 @@ private[spark] class Executor( env.blockManager.initialize(conf.getAppId) } + private val executorMetrics: ExecutorMetrics = new ExecutorMetrics + executorMetrics.setHostname(Utils.localHostName) + executorMetrics.setPort(env.blockTransferService.port) + // Create an RpcEndpoint for receiving RPCs from the driver private val executorEndpoint = env.rpcEnv.setupEndpoint( ExecutorEndpoint.EXECUTOR_ENDPOINT_NAME, new ExecutorEndpoint(env.rpcEnv, executorId)) @@ -439,7 +443,16 @@ private[spark] class Executor( } } - val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + env.blockTransferService.getMemMetrics(this.executorMetrics) + val executorMetrics = if (isLocal) { + Utils.deserialize[ExecutorMetrics](Utils.serialize(this.executorMetrics)) + } else { + this.executorMetrics + } + + val message = Heartbeat( + executorId, executorMetrics, tasksMetrics.toArray, env.blockManager.blockManagerId) + try { val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse](message) if (response.reregisterBlockManager) { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala new file mode 100644 index 0000000000000..172e2543767d4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.executor + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Metrics tracked during the execution of an executor. + * + * So, when adding new fields, take into consideration that the whole object can be serialized for + * shipping off at any time to consumers of the SparkListener interface. + */ +@DeveloperApi +class ExecutorMetrics extends Serializable { + + /** + * Host's name the executor runs on + */ + private var _hostname: String = _ + def hostname: String = _hostname + private[spark] def setHostname(value: String) = _hostname = value + + /** + * Host's port the executor runs on + */ + private var _port: Option[Int] = _ + def port: Option[Int] = _port + private[spark] def setPort(value: Int) = _port = Some(value) + + def hostPort: String = hostname + ":" + port.getOrElse(0) + + private var _transportMetrics: Option[TransportMetrics] = None + def transportMetrics: Option[TransportMetrics] = _transportMetrics + private[spark] def setTransportMetrics(value: TransportMetrics) = { + _transportMetrics = Some(value) + } +} + +/** + * :: DeveloperApi :: + * Metrics for network layer + */ +@DeveloperApi +case class TransportMetrics( + timeStamp: Long, + clientOnheapSize: Long, + clientDirectheapSize: Long, + serverOnheapSize: Long, + serverDirectheapSize: Long) \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index dcbda5a8515dd..9f0b4c7176af1 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -23,10 +23,11 @@ import java.nio.ByteBuffer import scala.concurrent.{Promise, Await, Future} import scala.concurrent.duration.Duration +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.Logging import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} import org.apache.spark.network.shuffle.{ShuffleClient, BlockFetchingListener} -import org.apache.spark.storage.{BlockManagerId, BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, StorageLevel} private[spark] abstract class BlockTransferService extends ShuffleClient with Closeable with Logging { @@ -37,6 +38,11 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo */ def init(blockDataManager: BlockDataManager) + /** + * Collect current executor memory metrics of transferService. + */ + def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = {} + /** * Tear down the transfer service. */ diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index d650d5fe73087..07b9b906ca95a 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,10 +17,15 @@ package org.apache.spark.network.netty +import java.util.List + import scala.collection.JavaConversions._ import scala.concurrent.{Future, Promise} +import io.netty.buffer._ + import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCallback, TransportClientFactory} @@ -29,8 +34,9 @@ import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.network.shuffle.protocol.UploadBlock import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.SparkEnv import org.apache.spark.storage.{BlockId, StorageLevel} -import org.apache.spark.util.Utils +import org.apache.spark.util.{Clock, Utils, SystemClock} /** * A BlockTransferService that uses Netty to fetch a set of blocks at at time. @@ -47,6 +53,37 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage private[this] var server: TransportServer = _ private[this] var clientFactory: TransportClientFactory = _ private[this] var appId: String = _ + private[this] var clock: Clock = new SystemClock() + + /** + * Use a different clock for this allocation manager. This is mainly used for testing. + */ + def setClock(newClock: Clock): Unit = { + clock = newClock + } + + override def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = { + val currentTime = clock.getTimeMillis() + val clientPooledAllocator = clientFactory.getPooledAllocator() + val serverAllocator = server.getAllocator() + val clientDirectHeapSize: Long = sumOfMetrics(clientPooledAllocator.directArenas()) + val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas()) + val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas()) + val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas()) + executorMetrics.setTransportMetrics(TransportMetrics( + currentTime, clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize)) + logDebug(s"current Netty directHeapSize is $clientDirectHeapSize, heapSize is $clientOnHeapSize, " + + s"current server directHeapsize is $serverDirectHeapSize, server heapsize is $serverOnHeapSize, " + + s"executer id is ${SparkEnv.get.blockManager.blockManagerId.executorId}") + } + + private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { + arenaMetricList.map { Arena => + Arena.chunkLists().map { chunk => + chunk.iterator().map(_.chunkSize()).sum + }.sum + }.sum + } override def init(blockDataManager: BlockDataManager): Unit = { val rpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index bb489c6b6e98f..ab7202b9073e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -33,7 +33,7 @@ import org.apache.commons.lang3.SerializationUtils import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.rpc.RpcTimeout @@ -175,15 +175,16 @@ class DAGScheduler( } /** - * Update metrics for in-progress tasks and let the master know that the BlockManager is still - * alive. Return true if the driver knows about the given block manager. Otherwise, return false, - * indicating that the block manager should re-register. + * Update metrics for live executor and in-progress tasks and let the master know that the + * BlockManager is still alive. Return true if the driver knows about the given block manager. + * Otherwise, return false, indicating that the block manager should re-register. */ def executorHeartbeatReceived( execId: String, + executorMetrics: ExecutorMetrics, taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) blockManagerId: BlockManagerId): Boolean = { - listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, executorMetrics, taskMetrics)) blockManagerMaster.driverEndpoint.askWithRetry[Boolean]( BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 896f1743332f1..5285bd9d34883 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} import org.apache.spark.util.{Distribution, Utils} @@ -104,11 +104,13 @@ case class SparkListenerBlockUpdated(blockUpdatedInfo: BlockUpdatedInfo) extends /** * Periodic updates from executors. * @param execId executor id + * @param executorMetrics metrics in executor level * @param taskMetrics sequence of (task id, stage id, stage attempt, metrics) */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, + executorMetrics: ExecutorMetrics, taskMetrics: Seq[(Long, Int, Int, TaskMetrics)]) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index f25f3ed0d9037..ead3207b30abb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.scheduler.SchedulingMode.SchedulingMode -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.storage.BlockManagerId /** @@ -64,8 +64,8 @@ private[spark] trait TaskScheduler { * alive. Return true if the driver knows about the given block manager. Otherwise, return false, * indicating that the block manager should re-register. */ - def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean + def executorHeartbeatReceived(execId: String, executorMetrics: ExecutorMetrics, + taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 1705e7f962de2..eaefb1cb89896 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -33,7 +33,7 @@ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.util.{ThreadUtils, Utils} -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.storage.BlockManagerId /** @@ -373,6 +373,7 @@ private[spark] class TaskSchedulerImpl( */ override def executorHeartbeatReceived( execId: String, + executorMetrics: ExecutorMetrics, taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId): Boolean = { @@ -383,7 +384,8 @@ private[spark] class TaskSchedulerImpl( } } } - dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) + dagScheduler.executorHeartbeatReceived( + execId, executorMetrics, metricsWithStageIds, blockManagerId) } def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long): Unit = synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 3788916cf39bb..e26ac3758b97e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -28,6 +28,7 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} +import org.apache.spark.ui.memory.{MemoryListener, MemoryTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.ui.scope.RDDOperationGraphListener @@ -43,6 +44,7 @@ private[spark] class SparkUI private ( val executorsListener: ExecutorsListener, val jobProgressListener: JobProgressListener, val storageListener: StorageListener, + val memoryListener: MemoryListener, val operationGraphListener: RDDOperationGraphListener, var appName: String, val basePath: String, @@ -63,6 +65,7 @@ private[spark] class SparkUI private ( attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) + attachTab(new MemoryTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler(ApiRootResource.getServletHandler(this)) @@ -179,16 +182,18 @@ private[spark] object SparkUI { val storageStatusListener = new StorageStatusListener val executorsListener = new ExecutorsListener(storageStatusListener) val storageListener = new StorageListener(storageStatusListener) + val memoryListener = new MemoryListener val operationGraphListener = new RDDOperationGraphListener(conf) listenerBus.addListener(environmentListener) listenerBus.addListener(storageStatusListener) listenerBus.addListener(executorsListener) listenerBus.addListener(storageListener) + listenerBus.addListener(memoryListener) listenerBus.addListener(operationGraphListener) new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, - executorsListener, _jobProgressListener, storageListener, operationGraphListener, - appName, basePath, startTime) + executorsListener, _jobProgressListener, storageListener, memoryListener, + operationGraphListener, appName, basePath, startTime) } } diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala new file mode 100644 index 0000000000000..f7ef0c2f60795 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala @@ -0,0 +1,84 @@ +/* + * 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.ui.memory + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.util.Utils + +private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { + private val listener = parent.listener + + def render(request: HttpServletRequest): Seq[Node] = { + + val executorIdToMem = listener.executorIdToMem + val memInfoSorted = executorIdToMem.toSeq.sortBy(_._1) + + val memTable = + + + + + + + + + + + {memInfoSorted.map(showRow(_))} + +
Executor IDAddressNet Memory (on-heap)Net Memory (direct-heap)Peak Net Memory (on-heap) / Happen TimePeak Net Read (direct-heap) / Happen Time
+ + val content = +
+
+ {memTable} +
+
; + + UIUtils.headerSparkPage("Memory Usage", content, parent) + } + + /** Render an HTML row representing an executor */ + private def showRow(info: (String, MemoryUIInfo)): Seq[Node] = { + + {info._1} + {info._2.executorAddress} + {if (info._2.transportInfo.isDefined) { + {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} + {Utils.bytesToString(info._2.transportInfo.get.directheapSize)} + + {Utils.bytesToString(info._2.transportInfo.get.peakOnheapSizeTime.memorySize)} / + {UIUtils.formatDate(info._2.transportInfo.get.peakOnheapSizeTime.timeStamp)} + + + {Utils.bytesToString(info._2.transportInfo.get.peakDirectheapSizeTime.memorySize)} / + {UIUtils.formatDate(info._2.transportInfo.get.peakDirectheapSizeTime.timeStamp)} + + } else { + N/A + N/A + N/A + N/A + }} + + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala new file mode 100644 index 0000000000000..c1cb2df29bbbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.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.ui.memory + +import scala.collection.mutable.HashMap + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} +import org.apache.spark.scheduler.{SparkListenerExecutorMetricsUpdate, SparkListener} +import org.apache.spark.ui.{SparkUITab, SparkUI} + +private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory") { + val listener = parent.memoryListener + attachPage(new MemoryPage(this)) +} + +/** + * :: DeveloperApi :: + * A SparkListener that prepares information to be displayed on the MemoryTab + */ +@DeveloperApi +class MemoryListener extends SparkListener { + type ExecutorId = String + val executorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] + + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { + val executorId = executorMetricsUpdate.execId + val executorMetrics = executorMetricsUpdate.executorMetrics + val memoryInfo = executorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) + memoryInfo.updateExecutorMetrics(executorMetrics) + } +} + +class MemoryUIInfo { + var executorAddress: String = _ + var transportInfo: Option[transportMemSize] = None + + def updateExecutorMetrics (execMetrics: ExecutorMetrics): Unit = { + transportInfo = transportInfo match { + case Some(transportMemSize) => transportInfo + case _ => Some(new transportMemSize) + } + executorAddress = execMetrics.hostPort + if (execMetrics.transportMetrics.isDefined) { + transportInfo.get.updateTransport(execMetrics.transportMetrics.get) + } + } +} + +class transportMemSize { + var onheapSize: Long = _ + var directheapSize: Long = _ + var peakOnheapSizeTime: MemTime = new MemTime() + var peakDirectheapSizeTime: MemTime = new MemTime() + def updateTransport(transportMetrics: TransportMetrics): Unit = { + val updatedOnheapSize = transportMetrics.clientOnheapSize + + transportMetrics.serverOnheapSize + val updatedDirectheapSize = transportMetrics.clientDirectheapSize + + transportMetrics.serverDirectheapSize + val updateTime: Long = transportMetrics.timeStamp + onheapSize = updatedOnheapSize + directheapSize = updatedDirectheapSize + if (updatedOnheapSize >= peakOnheapSizeTime.memorySize) { + peakOnheapSizeTime = MemTime(updatedOnheapSize, updateTime) + } + if (updatedDirectheapSize >= peakDirectheapSizeTime.memorySize) { + peakDirectheapSizeTime = MemTime(updatedDirectheapSize, updateTime) + } + } +} + +case class MemTime(memorySize: Long = 0L, timeStamp: Long = 0L) \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 22e2993b3b5bd..ebc855e326534 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { +private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { val listener = parent.storageListener attachPage(new StoragePage(this)) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index c600319d9ddb4..e5689d258e3f1 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -624,7 +624,7 @@ private[spark] object JsonProtocol { val metrics = taskMetricsFromJson(json \ "Task Metrics") (taskId, stageId, stageAttemptId, metrics) } - SparkListenerExecutorMetricsUpdate(execInfo, taskMetrics) + SparkListenerExecutorMetricsUpdate(execInfo, null, taskMetrics) } /** --------------------------------------------------------------------- * diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 139b8dc25f4b4..4ea3ca0dcfc92 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -74,7 +74,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock = new ManualClock heartbeatReceiver = new HeartbeatReceiver(sc, heartbeatReceiverClock) heartbeatReceiverRef = sc.env.rpcEnv.setupEndpoint("heartbeat", heartbeatReceiver) - when(scheduler.executorHeartbeatReceived(any(), any(), any())).thenReturn(true) + when(scheduler.executorHeartbeatReceived(any(), any(), any(), any())).thenReturn(true) } /** @@ -211,14 +211,15 @@ class HeartbeatReceiverSuite val metrics = new TaskMetrics val blockManagerId = BlockManagerId(executorId, "localhost", 12345) val response = heartbeatReceiverRef.askWithRetry[HeartbeatResponse]( - Heartbeat(executorId, Array(1L -> metrics), blockManagerId)) + Heartbeat(executorId, null, Array(1L -> metrics), blockManagerId)) if (executorShouldReregister) { assert(response.reregisterBlockManager) } else { assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), Matchers.eq(Array(1L -> metrics)), Matchers.eq(blockManagerId)) + Matchers.eq(executorId), null, Matchers.eq( + Array(1L -> metrics)), Matchers.eq(blockManagerId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 86dff8fb577d5..32b9494d50ce8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler) extends DAGSchedulerEventProcessLoop(dagScheduler) { @@ -82,8 +82,8 @@ class DAGSchedulerSuite override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def executorHeartbeatReceived(execId: String, executorMetrics: ExecutorMetrics, + taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -402,6 +402,7 @@ class DAGSchedulerSuite override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( execId: String, + executorMetrics: ExecutorMetrics, taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 56f7b9cf1f358..2d3f00ae1832c 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -302,7 +302,7 @@ class JobProgressListenerSuite extends SparkFunSuite with LocalSparkContext with listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1236L))) listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1237L))) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, null, Array( (1234L, 0, 0, makeTaskMetrics(0)), (1235L, 0, 0, makeTaskMetrics(100)), (1236L, 1, 0, makeTaskMetrics(200))))) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index dde95f3778434..f3b8cccc8d46d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -83,7 +83,7 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") - val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", Seq( + val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", null, Seq( (1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)))) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 4952ffb44bb8b..ecc50307d225d 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -88,6 +88,10 @@ public ClientPool(int size) { private EventLoopGroup workerGroup; private PooledByteBufAllocator pooledAllocator; + public PooledByteBufAllocator getPooledAllocator() { + return pooledAllocator; + } + public TransportClientFactory( TransportContext context, List clientBootstraps) { diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index f4fadb1ee3b8d..caf24ebb58840 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -53,8 +53,13 @@ public class TransportServer implements Closeable { private ServerBootstrap bootstrap; private ChannelFuture channelFuture; + private PooledByteBufAllocator allocator; private int port = -1; + public PooledByteBufAllocator getAllocator() { + return allocator; + } + /** Creates a TransportServer that binds to the given port, or to any available if 0. */ public TransportServer( TransportContext context, @@ -88,7 +93,7 @@ private void init(int portToBind) { NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); EventLoopGroup workerGroup = bossGroup; - PooledByteBufAllocator allocator = NettyUtils.createPooledByteBufAllocator( + allocator = NettyUtils.createPooledByteBufAllocator( conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); bootstrap = new ServerBootstrap() From 2101538ddf7d14ef90dc1097be51cf655eb75049 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 17 Aug 2015 17:37:18 +0800 Subject: [PATCH 02/28] show N/A for nio --- .../org/apache/spark/ui/memory/MemoryTab.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index c1cb2df29bbbd..2eb449d593fb8 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -51,14 +51,16 @@ class MemoryUIInfo { var executorAddress: String = _ var transportInfo: Option[transportMemSize] = None - def updateExecutorMetrics (execMetrics: ExecutorMetrics): Unit = { - transportInfo = transportInfo match { - case Some(transportMemSize) => transportInfo - case _ => Some(new transportMemSize) - } - executorAddress = execMetrics.hostPort + def updateExecutorMetrics(execMetrics: ExecutorMetrics): Unit = { if (execMetrics.transportMetrics.isDefined) { - transportInfo.get.updateTransport(execMetrics.transportMetrics.get) + transportInfo = transportInfo match { + case Some(transportMemSize) => transportInfo + case _ => Some(new transportMemSize) + } + executorAddress = execMetrics.hostPort + if (execMetrics.transportMetrics.isDefined) { + transportInfo.get.updateTransport(execMetrics.transportMetrics.get) + } } } } From 9ccaf884877a8caa549f7c0462bba8105bca71d7 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 18 Aug 2015 15:45:15 +0800 Subject: [PATCH 03/28] handle executor add and remove event for memotyTab --- .../apache/spark/ui/memory/MemoryTab.scala | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 2eb449d593fb8..eae1304112f1d 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -21,7 +21,8 @@ import scala.collection.mutable.HashMap import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} -import org.apache.spark.scheduler.{SparkListenerExecutorMetricsUpdate, SparkListener} +import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.ui.{SparkUITab, SparkUI} private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory") { @@ -38,19 +39,38 @@ class MemoryListener extends SparkListener { type ExecutorId = String val executorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] - override def onExecutorMetricsUpdate( - executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - val executorId = executorMetricsUpdate.execId - val executorMetrics = executorMetricsUpdate.executorMetrics + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + val executorId = event.execId + val executorMetrics = event.executorMetrics val memoryInfo = executorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) memoryInfo.updateExecutorMetrics(executorMetrics) } + + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { + val executorId = event.executorId + executorIdToMem.put(executorId, new MemoryUIInfo(event.executorInfo)) + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { + val executorId = event.executorId + executorIdToMem.remove(executorId) + } + + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { + val executorId = event.blockManagerId.executorId + executorIdToMem.remove(executorId) + } } class MemoryUIInfo { var executorAddress: String = _ var transportInfo: Option[transportMemSize] = None + def this(execInfo: ExecutorInfo) = { + this() + executorAddress = execInfo.executorHost + } + def updateExecutorMetrics(execMetrics: ExecutorMetrics): Unit = { if (execMetrics.transportMetrics.isDefined) { transportInfo = transportInfo match { @@ -70,6 +90,7 @@ class transportMemSize { var directheapSize: Long = _ var peakOnheapSizeTime: MemTime = new MemTime() var peakDirectheapSizeTime: MemTime = new MemTime() + def updateTransport(transportMetrics: TransportMetrics): Unit = { val updatedOnheapSize = transportMetrics.clientOnheapSize + transportMetrics.serverOnheapSize From 13c17fb9a110fcecac7411281b592a01a7b5d1cd Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 19 Aug 2015 10:56:01 +0800 Subject: [PATCH 04/28] show removed executors info on page --- .../apache/spark/ui/memory/MemoryPage.scala | 106 +++++++++++++----- .../apache/spark/ui/memory/MemoryTab.scala | 13 ++- 2 files changed, 85 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala index f7ef0c2f60795..fe936cafd8121 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui.memory import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.xml.{Node, NodeSeq} import org.apache.spark.ui.{UIUtils, WebUIPage} import org.apache.spark.util.Utils @@ -29,41 +29,89 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { - val executorIdToMem = listener.executorIdToMem - val memInfoSorted = executorIdToMem.toSeq.sortBy(_._1) - - val memTable = - - - - - - - - - - - {memInfoSorted.map(showRow(_))} - -
Executor IDAddressNet Memory (on-heap)Net Memory (direct-heap)Peak Net Memory (on-heap) / Happen TimePeak Net Read (direct-heap) / Happen Time
- - val content = -
-
- {memTable} -
-
; + val activeExecutorIdToMem = listener.activeExecutorIdToMem + val removedExecutorIdToMem = listener.removedExecutorIdToMem + val activeMemInfoSorted = activeExecutorIdToMem.toSeq.sortBy(_._1) + val removedMemInfoSorted = removedExecutorIdToMem.toSeq.sortBy(_._1) + val shouldShowActiveExecutors = activeExecutorIdToMem.nonEmpty + val shouldShowRemovedExecutors = removedExecutorIdToMem.nonEmpty + + val activeExecMemTable = new MemTableBase(activeMemInfoSorted, listener) + val removedExecMemTable = new MemTableBase(removedMemInfoSorted, listener) + + val summary: NodeSeq = +
+
    + { + if (shouldShowActiveExecutors) { +
  • + Active Executors: + {activeExecutorIdToMem.size} +
  • + } + } + { + if (shouldShowRemovedExecutors) { +
  • + Active Executors: + {removedExecutorIdToMem.size} +
  • + } + } +
+
+ + var content = summary + if (shouldShowActiveExecutors) { + content ++=

Active Executors ({activeExecutorIdToMem.size})

++ + activeExecMemTable.toNodeSeq + } + if (shouldShowRemovedExecutors) { + content ++=

Active Executors ({removedMemInfoSorted.size})

++ + removedExecMemTable.toNodeSeq + } UIUtils.headerSparkPage("Memory Usage", content, parent) } + +} + +private[ui] class MemTableBase( + memInfos: Seq[(String, MemoryUIInfo)], + listener: MemoryListener) { + + protected def columns: Seq[Node] = { + Executor ID + Address + Net Memory (on-heap) + Net Memory (direct-heap) + Peak Net Memory (on-heap) / Happen Time + Peak Net Read (direct-heap) / Happen Time + } + + def toNodeSeq: Seq[Node] = { + listener.synchronized { + memTable(showRow, memInfos) + } + } + + protected def memTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + {columns} + + {rows.map(r => makeRow(r))} + +
+ } + /** Render an HTML row representing an executor */ private def showRow(info: (String, MemoryUIInfo)): Seq[Node] = { {info._1} {info._2.executorAddress} {if (info._2.transportInfo.isDefined) { - {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} + {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} {Utils.bytesToString(info._2.transportInfo.get.directheapSize)} {Utils.bytesToString(info._2.transportInfo.get.peakOnheapSizeTime.memorySize)} / @@ -73,12 +121,12 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { {Utils.bytesToString(info._2.transportInfo.get.peakDirectheapSizeTime.memorySize)} / {UIUtils.formatDate(info._2.transportInfo.get.peakDirectheapSizeTime.timeStamp)} - } else { - N/A + } else { + N/A N/A N/A N/A - }} + }} } } \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index eae1304112f1d..d786cc7ec3ff7 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -37,28 +37,31 @@ private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory" @DeveloperApi class MemoryListener extends SparkListener { type ExecutorId = String - val executorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] + val activeExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] + val removedExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { val executorId = event.execId val executorMetrics = event.executorMetrics - val memoryInfo = executorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) + val memoryInfo = activeExecutorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) memoryInfo.updateExecutorMetrics(executorMetrics) } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { val executorId = event.executorId - executorIdToMem.put(executorId, new MemoryUIInfo(event.executorInfo)) + activeExecutorIdToMem.put(executorId, new MemoryUIInfo(event.executorInfo)) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { val executorId = event.executorId - executorIdToMem.remove(executorId) + val info = activeExecutorIdToMem.remove(executorId) + removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) } override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { val executorId = event.blockManagerId.executorId - executorIdToMem.remove(executorId) + val info = activeExecutorIdToMem.remove(executorId) + removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) } } From c9b44b170a79526506f7a20bc257825f72570e60 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 19 Aug 2015 23:25:27 +0800 Subject: [PATCH 05/28] add stage memory trace --- .../apache/spark/ui/memory/MemoryPage.scala | 110 ++++++-------- .../apache/spark/ui/memory/MemoryTab.scala | 42 ++++- .../apache/spark/ui/memory/MemoryTable.scala | 143 ++++++++++++++++++ .../spark/ui/memory/StageMemoryPage.scala | 52 +++++++ 4 files changed, 281 insertions(+), 66 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala index fe936cafd8121..62e4a5ea06ddd 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala @@ -22,22 +22,31 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.ui.{UIUtils, WebUIPage} -import org.apache.spark.util.Utils private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { - private val listener = parent.listener + private val memoryListener = parent.memoryListener + private val progressListener = parent.progressListener def render(request: HttpServletRequest): Seq[Node] = { - val activeExecutorIdToMem = listener.activeExecutorIdToMem - val removedExecutorIdToMem = listener.removedExecutorIdToMem + val activeExecutorIdToMem = memoryListener.activeExecutorIdToMem + val removedExecutorIdToMem = memoryListener.removedExecutorIdToMem + val completedStages = progressListener.completedStages.reverse.toSeq + val failedStages = progressListener.failedStages.reverse.toSeq + val numberCompletedStages = progressListener.numCompletedStages + val numberFailedStages = progressListener.numFailedStages val activeMemInfoSorted = activeExecutorIdToMem.toSeq.sortBy(_._1) val removedMemInfoSorted = removedExecutorIdToMem.toSeq.sortBy(_._1) val shouldShowActiveExecutors = activeExecutorIdToMem.nonEmpty val shouldShowRemovedExecutors = removedExecutorIdToMem.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty - val activeExecMemTable = new MemTableBase(activeMemInfoSorted, listener) - val removedExecMemTable = new MemTableBase(removedMemInfoSorted, listener) + val activeExecMemTable = new MemTableBase(activeMemInfoSorted, memoryListener) + val removedExecMemTable = new MemTableBase(removedMemInfoSorted, memoryListener) + val completedStagesTable = new stagesTableBase( + completedStages, parent.basePath, progressListener) + val failedStagesTable = new stagesTableBase(failedStages, parent.basePath, progressListener) val summary: NodeSeq =
@@ -51,12 +60,28 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { } } { - if (shouldShowRemovedExecutors) { -
  • - Active Executors: - {removedExecutorIdToMem.size} -
  • + if (shouldShowRemovedExecutors) { +
  • + Removed Executors: + {removedExecutorIdToMem.size} +
  • + } + } + { + if (shouldShowCompletedStages) { +
  • + Completed Stages: + {numberCompletedStages} +
  • + } } + { + if (shouldShowFailedStages) { +
  • + Failed Stages: + {numberFailedStages} +
  • + } }
    @@ -67,9 +92,17 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { activeExecMemTable.toNodeSeq } if (shouldShowRemovedExecutors) { - content ++=

    Active Executors ({removedMemInfoSorted.size})

    ++ + content ++=

    Removed Executors ({removedMemInfoSorted.size})

    ++ removedExecMemTable.toNodeSeq } + if (shouldShowCompletedStages) { + content ++=

    Completed Stages ({numberCompletedStages})

    ++ + completedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

    Failed Stages ({numberFailedStages})

    ++ + failedStagesTable.toNodeSeq + } UIUtils.headerSparkPage("Memory Usage", content, parent) } @@ -77,56 +110,3 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { } -private[ui] class MemTableBase( - memInfos: Seq[(String, MemoryUIInfo)], - listener: MemoryListener) { - - protected def columns: Seq[Node] = { - Executor ID - Address - Net Memory (on-heap) - Net Memory (direct-heap) - Peak Net Memory (on-heap) / Happen Time - Peak Net Read (direct-heap) / Happen Time - } - - def toNodeSeq: Seq[Node] = { - listener.synchronized { - memTable(showRow, memInfos) - } - } - - protected def memTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - {columns} - - {rows.map(r => makeRow(r))} - -
    - } - - /** Render an HTML row representing an executor */ - private def showRow(info: (String, MemoryUIInfo)): Seq[Node] = { - - {info._1} - {info._2.executorAddress} - {if (info._2.transportInfo.isDefined) { - {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} - {Utils.bytesToString(info._2.transportInfo.get.directheapSize)} - - {Utils.bytesToString(info._2.transportInfo.get.peakOnheapSizeTime.memorySize)} / - {UIUtils.formatDate(info._2.transportInfo.get.peakOnheapSizeTime.timeStamp)} - - - {Utils.bytesToString(info._2.transportInfo.get.peakDirectheapSizeTime.memorySize)} / - {UIUtils.formatDate(info._2.transportInfo.get.peakDirectheapSizeTime.timeStamp)} - - } else { - N/A - N/A - N/A - N/A - }} - - } -} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index d786cc7ec3ff7..cfb0e7de6b016 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -26,8 +26,10 @@ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.ui.{SparkUITab, SparkUI} private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory") { - val listener = parent.memoryListener + val memoryListener = parent.memoryListener + val progressListener = parent.jobProgressListener attachPage(new MemoryPage(this)) + attachPage(new StageMemoryPage(this)) } /** @@ -39,12 +41,27 @@ class MemoryListener extends SparkListener { type ExecutorId = String val activeExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] val removedExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] + // latestExecIdToExecMetrics include all executors that is active and removed + // this may consume a lot of memory when executors are changing frequently, e.g. in dynamical + // allocation mode. + val latestExecIdToExecMetrics = new HashMap[ExecutorId, ExecutorMetrics] + // stagesIdToMem a map maintains all executors memory information of each stage, + // the Map type is [(stageId, attemptId), Seq[(executorId, MemoryUIInfo)] + val activeStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] + val completedStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { val executorId = event.execId val executorMetrics = event.executorMetrics val memoryInfo = activeExecutorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) memoryInfo.updateExecutorMetrics(executorMetrics) + activeStagesToMem.map {stageToMem => + if (stageToMem._2.contains(executorId)) { + val memInfo = stageToMem._2.get(executorId).get + memInfo.updateExecutorMetrics(executorMetrics) + } + } + latestExecIdToExecMetrics.update(executorId, executorMetrics) } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { @@ -63,6 +80,29 @@ class MemoryListener extends SparkListener { val info = activeExecutorIdToMem.remove(executorId) removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) } + + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { + val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) + val memInfoMap = new HashMap[ExecutorId, MemoryUIInfo] + activeExecutorIdToMem.map(idToMem => memInfoMap.update(idToMem._1, new MemoryUIInfo)) + activeStagesToMem.update(stage, memInfoMap) + } + + override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { + val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) + val memInfoMap = activeStagesToMem.get(stage) + if (memInfoMap.isDefined) { + activeExecutorIdToMem.map { idToMem => + val executorId = idToMem._1 + val memInfo = memInfoMap.get.getOrElse(executorId, new MemoryUIInfo) + if (latestExecIdToExecMetrics.contains(executorId)) { + memInfo.updateExecutorMetrics(latestExecIdToExecMetrics.get(executorId).get) + } + memInfoMap.get.update(executorId, memInfo) + } + completedStagesToMem.put(stage, activeStagesToMem.remove(stage).get) + } + } } class MemoryUIInfo { diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala new file mode 100644 index 0000000000000..305ceee425f25 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -0,0 +1,143 @@ +/* + * 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.ui.memory + +import java.util.Date + +import scala.xml.Node + +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.ui.jobs.JobProgressListener +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.Utils + + +private[ui] class MemTableBase( + memInfos: Seq[(String, MemoryUIInfo)], + listener: MemoryListener) { + + protected def columns: Seq[Node] = { + Executor ID + Address + Net Memory (on-heap) + Net Memory (direct-heap) + Peak Net Memory (on-heap) / Happen Time + Peak Net Read (direct-heap) / Happen Time + } + + def toNodeSeq: Seq[Node] = { + listener.synchronized { + memTable(showRow, memInfos) + } + } + + protected def memTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + {columns} + + + {rows.map(r => makeRow(r))} + +
    + } + + /** Render an HTML row representing an executor */ + private def showRow(info: (String, MemoryUIInfo)): Seq[Node] = { + + + {info._1} + + + {info._2.executorAddress} + + {if (info._2.transportInfo.isDefined) { + + {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} + + + {Utils.bytesToString(info._2.transportInfo.get.directheapSize)} + + + {Utils.bytesToString(info._2.transportInfo.get.peakOnheapSizeTime.memorySize)} + / + {UIUtils.formatDate(info._2.transportInfo.get.peakOnheapSizeTime.timeStamp)} + + + {Utils.bytesToString(info._2.transportInfo.get.peakDirectheapSizeTime.memorySize)} + / + {UIUtils.formatDate(info._2.transportInfo.get.peakDirectheapSizeTime.timeStamp)} + + } else { + N/A + N/A + N/A + N/A + }} + + } +} + +private[ui] class stagesTableBase( + stageInfos: Seq[StageInfo], + basePath: String, + listener: JobProgressListener) { + protected def columns: Seq[Node] = { + Stage Id + Description + Submitted + } + + def toNodeSeq: Seq[Node] = { + listener.synchronized { + stagesTable(showRow, stageInfos) + } + } + + protected def stagesTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + + + {columns} + + + {rows.map(r => makeRow(r))} + +
    + } + + private def showRow(info: StageInfo): Seq[Node] = { + val submissionTime = info.submissionTime match { + case Some(t) => UIUtils.formatDate(new Date(t)) + case None => "Unknown" + } + + + {info.stageId} + {makeDescription(info)} + {submissionTime} + +} + + private def makeDescription(s: StageInfo): Seq[Node] = { + val basePathUri = UIUtils.prependBaseUri(basePath) + val nameLinkUri = s"$basePathUri/memory/stage?id=${s.stageId}&attempt=${s.attemptId}" +
    + {s.name} +
    + } +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala new file mode 100644 index 0000000000000..ca32d52a256ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.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.ui.memory + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUIPage} + +/** Page showing memory information for a given stage */ +private[ui] class StageMemoryPage(parent: MemoryTab) extends WebUIPage("stage") { + private val memoryListener = parent.memoryListener + + def render(request: HttpServletRequest): Seq[Node] = { + memoryListener.synchronized { + val parameterId = request.getParameter("id") + require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") + + val parameterAttempt = request.getParameter("attempt") + require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") + + val stage = (parameterId.toInt, parameterAttempt.toInt) + + val finishedStageToMem = memoryListener.completedStagesToMem + val content = if (finishedStageToMem.get(stage).isDefined) { + val executorIdToMem = finishedStageToMem.get(stage).get.toSeq.sortBy(_._1) + val execMemTable = new MemTableBase(executorIdToMem, memoryListener) +

    Executors ({executorIdToMem.size})

    ++ + execMemTable.toNodeSeq + } else { + Seq.empty + } + UIUtils.headerSparkPage("Stage Detail Memory Usage", content, parent) + } + } +} From 984feafa936c52ea880e4d5346405eb112011360 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 20 Aug 2015 11:14:11 +0800 Subject: [PATCH 06/28] add history support for heartbeat event --- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/executor/ExecutorMetrics.scala | 6 +-- .../netty/NettyBlockTransferService.scala | 4 +- .../scheduler/EventLoggingListener.scala | 4 +- .../apache/spark/ui/memory/MemoryTab.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 47 +++++++++++++++++-- 6 files changed, 52 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 9759e367d7cf4..a5a38d4556300 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -87,7 +87,7 @@ private[spark] class Executor( private val executorMetrics: ExecutorMetrics = new ExecutorMetrics executorMetrics.setHostname(Utils.localHostName) - executorMetrics.setPort(env.blockTransferService.port) + executorMetrics.setPort(Some(env.blockTransferService.port)) // Create an RpcEndpoint for receiving RPCs from the driver private val executorEndpoint = env.rpcEnv.setupEndpoint( diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 172e2543767d4..3850983243932 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -41,14 +41,14 @@ class ExecutorMetrics extends Serializable { */ private var _port: Option[Int] = _ def port: Option[Int] = _port - private[spark] def setPort(value: Int) = _port = Some(value) + private[spark] def setPort(value: Option[Int]) = _port = value def hostPort: String = hostname + ":" + port.getOrElse(0) private var _transportMetrics: Option[TransportMetrics] = None def transportMetrics: Option[TransportMetrics] = _transportMetrics - private[spark] def setTransportMetrics(value: TransportMetrics) = { - _transportMetrics = Some(value) + private[spark] def setTransportMetrics(value: Option[TransportMetrics]) = { + _transportMetrics = value } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 07b9b906ca95a..65f1cee16fd31 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -70,8 +70,8 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas()) val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas()) val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas()) - executorMetrics.setTransportMetrics(TransportMetrics( - currentTime, clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize)) + executorMetrics.setTransportMetrics(Some(TransportMetrics( + currentTime, clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize))) logDebug(s"current Netty directHeapSize is $clientDirectHeapSize, heapSize is $clientOnHeapSize, " + s"current server directHeapsize is $serverDirectHeapSize, server heapsize is $serverOnHeapSize, " + s"executer id is ${SparkEnv.get.blockManager.blockManagerId.executorId}") diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5a06ef02f5c57..9102a29a3705d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -203,7 +203,9 @@ private[spark] class EventLoggingListener( override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} // No-op because logging every update would be overkill - override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { } + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { + logEvent(event, flushLogger = false) + } /** * Stop logging events. The event log file will be renamed so that it loses the diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index cfb0e7de6b016..74d85bc306661 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -41,7 +41,7 @@ class MemoryListener extends SparkListener { type ExecutorId = String val activeExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] val removedExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] - // latestExecIdToExecMetrics include all executors that is active and removed + // latestExecIdToExecMetrics include all executors that is active and removed. // this may consume a lot of memory when executors are changing frequently, e.g. in dynamical // allocation mode. val latestExecIdToExecMetrics = new HashMap[ExecutorId, ExecutorMetrics] diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index e9d7bbb38c1e0..2815cc44cc87b 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -19,8 +19,6 @@ package org.apache.spark.util import java.util.{Properties, UUID} -import org.apache.spark.scheduler.cluster.ExecutorInfo - import scala.collection.JavaConverters._ import scala.collection.Map @@ -32,6 +30,7 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.rdd.RDDOperationScope import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage._ /** @@ -229,9 +228,11 @@ private[spark] object JsonProtocol { def executorMetricsUpdateToJson(metricsUpdate: SparkListenerExecutorMetricsUpdate): JValue = { val execId = metricsUpdate.execId val taskMetrics = metricsUpdate.taskMetrics + val executorMetrics = metricsUpdate.executorMetrics ("Event" -> Utils.getFormattedClassName(metricsUpdate)) ~ ("Executor ID" -> execId) ~ - ("Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => + ("Executor Metrics Updated" -> executorMetricsToJson(executorMetrics)) ~ + ("Task Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => ("Task ID" -> taskId) ~ ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ @@ -285,6 +286,22 @@ private[spark] object JsonProtocol { ("Value" -> accumulableInfo.value) } + def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { + val transportMetrics = executorMetrics.transportMetrics.map( + transportMetricsToJson).getOrElse(JNothing) + ("Executor Hostname" -> executorMetrics.hostname) ~ + ("Executor Port" -> executorMetrics.port.map(new JInt(_)).getOrElse(JNothing)) ~ + ("TransportMetrics" -> transportMetrics) + } + + def transportMetricsToJson(transportMetrics: TransportMetrics): JValue = { + ("TimeStamep" -> transportMetrics.timeStamp) ~ + ("ClientOnheapSize" -> transportMetrics.clientOnheapSize) ~ + ("ClientDirectheapSize" -> transportMetrics.clientDirectheapSize) ~ + ("ServerOnheapSize" -> transportMetrics.serverOnheapSize) ~ + ("ServerDirectheapSize" -> transportMetrics.serverDirectheapSize) + } + def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { val shuffleReadMetrics = taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) @@ -617,14 +634,15 @@ private[spark] object JsonProtocol { def executorMetricsUpdateFromJson(json: JValue): SparkListenerExecutorMetricsUpdate = { val execInfo = (json \ "Executor ID").extract[String] - val taskMetrics = (json \ "Metrics Updated").extract[List[JValue]].map { json => + val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics Updated") + val taskMetrics = (json \ "Task Metrics Updated").extract[List[JValue]].map { json => val taskId = (json \ "Task ID").extract[Long] val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] val metrics = taskMetricsFromJson(json \ "Task Metrics") (taskId, stageId, stageAttemptId, metrics) } - SparkListenerExecutorMetricsUpdate(execInfo, null, taskMetrics) + SparkListenerExecutorMetricsUpdate(execInfo, executorMetrics, taskMetrics) } /** --------------------------------------------------------------------- * @@ -694,6 +712,25 @@ private[spark] object JsonProtocol { AccumulableInfo(id, name, update, value) } + def executorMetricsFromJson(json: JValue): ExecutorMetrics = { + val metrics = new ExecutorMetrics + metrics.setHostname((json \ "Executor Hostname").extract[String]) + metrics.setPort(Utils.jsonOption(json \ "Executor Port").map(_.extract[Int])) + metrics.setTransportMetrics( + Utils.jsonOption((json \ "TransportMetrics")).map(transportMetrisFromJson)) + metrics + } + + def transportMetrisFromJson(json: JValue): TransportMetrics = { + val metrics = new TransportMetrics( + (json \ "TimeStamep").extract[Long], + (json \ "ClientOnheapSize").extract[Long], + (json \ "ClientDirectheapSize").extract[Long], + (json \ "ServerOnheapSize").extract[Long], + (json \ "ServerDirectheapSize").extract[Long]) + metrics + } + def taskMetricsFromJson(json: JValue): TaskMetrics = { if (json == JNothing) { return TaskMetrics.empty From 2501c8211be1dca9314d2e729263b67089d369a3 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 20 Aug 2015 15:54:30 +0800 Subject: [PATCH 07/28] limit history event log frequency --- .../scheduler/EventLoggingListener.scala | 63 +++++++++++++++++-- 1 file changed, 59 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 9102a29a3705d..70b69c8dae479 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,8 +20,11 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI +import akka.remote.transport.Transport +import org.apache.spark.executor.TransportMetrics + import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import com.google.common.base.Charsets import org.apache.hadoop.conf.Configuration @@ -91,7 +94,11 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) + private[scheduler] val logPath = getLogPath( + logBaseDir, appId, appAttemptId, compressionCodecName) + + private val latestMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] + private val modifiedMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] /** * Creates the log file in the configured log directory. @@ -152,8 +159,16 @@ private[spark] class EventLoggingListener( } } + private def logMetricsUpdateEvent() : Unit = { + modifiedMetrics.map(metrics => logEvent(metrics._2)) + latestMetrics.map(metrics => modifiedMetrics.update(metrics._1, metrics._2)) + } + // Events that do not trigger a flush - override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = logEvent(event) + override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { + logMetricsUpdateEvent() + logEvent(event) + } override def onTaskStart(event: SparkListenerTaskStart): Unit = logEvent(event) @@ -165,6 +180,7 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { + logMetricsUpdateEvent() logEvent(event, flushLogger = true) } @@ -191,11 +207,14 @@ private[spark] class EventLoggingListener( override def onApplicationEnd(event: SparkListenerApplicationEnd): Unit = { logEvent(event, flushLogger = true) } + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { logEvent(event, flushLogger = true) } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { + latestMetrics.remove(event.executorId) + modifiedMetrics.remove(event.executorId) logEvent(event, flushLogger = true) } @@ -204,7 +223,8 @@ private[spark] class EventLoggingListener( // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - logEvent(event, flushLogger = false) + latestMetrics.update(event.execId, event) + updateModifiedMetrics(event.execId) } /** @@ -226,6 +246,41 @@ private[spark] class EventLoggingListener( fileSystem.rename(new Path(logPath + IN_PROGRESS), target) } + private def updateModifiedMetrics(executorId: String): Unit = { + val toBeModifiedEvent = modifiedMetrics.get(executorId) + val latestEvent = latestMetrics.get(executorId) + if (toBeModifiedEvent.isEmpty) { + if (latestEvent.isDefined) modifiedMetrics.update(executorId, latestEvent.get) + } else { + val toBeModifiedMetrics = toBeModifiedEvent.get.executorMetrics.transportMetrics + if (toBeModifiedMetrics.isDefined) { + // latestEvent must has value + val latestTransMetrics = latestEvent.get.executorMetrics.transportMetrics.get + val toBeModTransMetrics = toBeModifiedMetrics.get + var timeStamp: Long = 0L + val (clientOnheapSize, serverOnheapSize) = + if (latestTransMetrics.clientOnheapSize + latestTransMetrics.serverOnheapSize > + toBeModTransMetrics.clientOnheapSize + toBeModTransMetrics.serverOnheapSize) { + timeStamp = latestTransMetrics.timeStamp + (latestTransMetrics.clientOnheapSize, latestTransMetrics.serverOnheapSize) + } else { + (toBeModTransMetrics.clientOnheapSize, toBeModTransMetrics.serverOnheapSize) + } + val (clientDirectheapSize, serverDirectheapSize) = + if (latestTransMetrics.clientDirectheapSize + latestTransMetrics.serverDirectheapSize > + toBeModTransMetrics.clientDirectheapSize + toBeModTransMetrics.serverDirectheapSize) { + timeStamp = latestTransMetrics.timeStamp + (latestTransMetrics.clientDirectheapSize, latestTransMetrics.serverDirectheapSize) + } else { + (toBeModTransMetrics.clientDirectheapSize, toBeModTransMetrics.serverDirectheapSize) + } + toBeModifiedEvent.get.executorMetrics.setTransportMetrics( + Some(TransportMetrics(timeStamp, clientOnheapSize, clientDirectheapSize, + serverOnheapSize, serverDirectheapSize))) + } + } + } + } private[spark] object EventLoggingListener extends Logging { From e0ae855333f8268342f127fdc47f3c3df584d17f Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 20 Aug 2015 16:20:26 +0800 Subject: [PATCH 08/28] add some comments for EventLoggingListener --- .../apache/spark/scheduler/EventLoggingListener.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 70b69c8dae479..be499f767b813 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -159,6 +159,9 @@ private[spark] class EventLoggingListener( } } + // We log the event both when stage submitted and stage completed, and after each logEvent call, + // replace the modifiedMetrics with the latestMetrics. In case the stages submit and complete + // time might be interleaved. So as to make the result the same with the running time. private def logMetricsUpdateEvent() : Unit = { modifiedMetrics.map(metrics => logEvent(metrics._2)) latestMetrics.map(metrics => modifiedMetrics.update(metrics._1, metrics._2)) @@ -246,6 +249,10 @@ private[spark] class EventLoggingListener( fileSystem.rename(new Path(logPath + IN_PROGRESS), target) } + /** + * According to the updated event to modify the maintained event's metrics + * @param executorId the executor whose metrics will be modified + */ private def updateModifiedMetrics(executorId: String): Unit = { val toBeModifiedEvent = modifiedMetrics.get(executorId) val latestEvent = latestMetrics.get(executorId) @@ -257,7 +264,8 @@ private[spark] class EventLoggingListener( // latestEvent must has value val latestTransMetrics = latestEvent.get.executorMetrics.transportMetrics.get val toBeModTransMetrics = toBeModifiedMetrics.get - var timeStamp: Long = 0L + var timeStamp: Long = toBeModTransMetrics.timeStamp + // the logic here should be the same with that for memoryListener val (clientOnheapSize, serverOnheapSize) = if (latestTransMetrics.clientOnheapSize + latestTransMetrics.serverOnheapSize > toBeModTransMetrics.clientOnheapSize + toBeModTransMetrics.serverOnheapSize) { From 424c17272707af1615bfafbd532c34c244d19020 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 20 Aug 2015 16:51:53 +0800 Subject: [PATCH 09/28] scala style fix --- .../org/apache/spark/executor/ExecutorMetrics.scala | 2 +- .../apache/spark/network/BlockTransferService.scala | 2 +- .../network/netty/NettyBlockTransferService.scala | 11 ++++++----- .../spark/network/nio/NioBlockTransferService.scala | 3 +++ .../scala/org/apache/spark/ui/memory/MemoryPage.scala | 3 --- .../scala/org/apache/spark/ui/memory/MemoryTab.scala | 2 +- .../org/apache/spark/ui/memory/MemoryTable.scala | 2 +- .../org/apache/spark/ui/storage/StorageTab.scala | 2 +- 8 files changed, 14 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 3850983243932..e4f7191298382 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -62,4 +62,4 @@ case class TransportMetrics( clientOnheapSize: Long, clientDirectheapSize: Long, serverOnheapSize: Long, - serverDirectheapSize: Long) \ No newline at end of file + serverDirectheapSize: Long) diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 9f0b4c7176af1..03b0fca4bcd74 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -41,7 +41,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo /** * Collect current executor memory metrics of transferService. */ - def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = {} + def getMemMetrics(executorMetrics: ExecutorMetrics): Unit /** * Tear down the transfer service. diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 65f1cee16fd31..f50ee2d6510ca 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -70,11 +70,12 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas()) val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas()) val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas()) - executorMetrics.setTransportMetrics(Some(TransportMetrics( - currentTime, clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize))) - logDebug(s"current Netty directHeapSize is $clientDirectHeapSize, heapSize is $clientOnHeapSize, " + - s"current server directHeapsize is $serverDirectHeapSize, server heapsize is $serverOnHeapSize, " + - s"executer id is ${SparkEnv.get.blockManager.blockManagerId.executorId}") + executorMetrics.setTransportMetrics(Some(TransportMetrics(currentTime, + clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize))) + logDebug(s"current Netty client directHeapSize is $clientDirectHeapSize, " + + s"client heapSize is $clientOnHeapSize, server directHeapsize is $serverDirectHeapSize, " + + s"server heapsize is $serverOnHeapSize, executer id is " + + s"${SparkEnv.get.blockManager.blockManagerId.executorId}") } private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index b2aec160635c7..eda94043b6b8c 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.nio import java.nio.ByteBuffer +import org.apache.spark.executor.ExecutorMetrics import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener @@ -56,6 +57,8 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa cm.id.host } + override def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = {} + /** * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch * local blocks or put local blocks. diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala index 62e4a5ea06ddd..b6dbd6cd53b5c 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala @@ -106,7 +106,4 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { UIUtils.headerSparkPage("Memory Usage", content, parent) } - - } - diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 74d85bc306661..8e55c068b7a52 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -151,4 +151,4 @@ class transportMemSize { } } -case class MemTime(memorySize: Long = 0L, timeStamp: Long = 0L) \ No newline at end of file +case class MemTime(memorySize: Long = 0L, timeStamp: Long = 0L) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala index 305ceee425f25..3972dd47f91b7 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -140,4 +140,4 @@ private[ui] class stagesTableBase( {s.name} } -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index ebc855e326534..22e2993b3b5bd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { +private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { val listener = parent.storageListener attachPage(new StoragePage(this)) From f21a8047e615c7dc17e5a823ebaabbbc84fffdb3 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Sat, 22 Aug 2015 00:45:34 +0800 Subject: [PATCH 10/28] remove executor port and fix test failure --- .../main/scala/org/apache/spark/executor/Executor.scala | 1 - .../org/apache/spark/executor/ExecutorMetrics.scala | 9 --------- .../scala/org/apache/spark/ui/memory/MemoryTab.scala | 2 +- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 2 -- .../scala/org/apache/spark/HeartbeatReceiverSuite.scala | 4 ++-- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 9 ++++++--- .../apache/spark/sql/execution/ui/SQLListenerSuite.scala | 8 ++++---- 7 files changed, 13 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index a5a38d4556300..1b3cb13f8e304 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -87,7 +87,6 @@ private[spark] class Executor( private val executorMetrics: ExecutorMetrics = new ExecutorMetrics executorMetrics.setHostname(Utils.localHostName) - executorMetrics.setPort(Some(env.blockTransferService.port)) // Create an RpcEndpoint for receiving RPCs from the driver private val executorEndpoint = env.rpcEnv.setupEndpoint( diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index e4f7191298382..a7d1da4bc6b61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -36,15 +36,6 @@ class ExecutorMetrics extends Serializable { def hostname: String = _hostname private[spark] def setHostname(value: String) = _hostname = value - /** - * Host's port the executor runs on - */ - private var _port: Option[Int] = _ - def port: Option[Int] = _port - private[spark] def setPort(value: Option[Int]) = _port = value - - def hostPort: String = hostname + ":" + port.getOrElse(0) - private var _transportMetrics: Option[TransportMetrics] = None def transportMetrics: Option[TransportMetrics] = _transportMetrics private[spark] def setTransportMetrics(value: Option[TransportMetrics]) = { diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 8e55c068b7a52..c423dcf0916d9 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -120,7 +120,7 @@ class MemoryUIInfo { case Some(transportMemSize) => transportInfo case _ => Some(new transportMemSize) } - executorAddress = execMetrics.hostPort + executorAddress = execMetrics.hostname if (execMetrics.transportMetrics.isDefined) { transportInfo.get.updateTransport(execMetrics.transportMetrics.get) } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2815cc44cc87b..f677ddd5e3c5a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -290,7 +290,6 @@ private[spark] object JsonProtocol { val transportMetrics = executorMetrics.transportMetrics.map( transportMetricsToJson).getOrElse(JNothing) ("Executor Hostname" -> executorMetrics.hostname) ~ - ("Executor Port" -> executorMetrics.port.map(new JInt(_)).getOrElse(JNothing)) ~ ("TransportMetrics" -> transportMetrics) } @@ -715,7 +714,6 @@ private[spark] object JsonProtocol { def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = new ExecutorMetrics metrics.setHostname((json \ "Executor Hostname").extract[String]) - metrics.setPort(Utils.jsonOption(json \ "Executor Port").map(_.extract[Int])) metrics.setTransportMetrics( Utils.jsonOption((json \ "TransportMetrics")).map(transportMetrisFromJson)) metrics diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index 4ea3ca0dcfc92..0523f5b1b4e36 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -27,7 +27,7 @@ import org.mockito.Mockito.{mock, spy, verify, when} import org.mockito.Matchers import org.mockito.Matchers._ -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEnv, RpcEndpointRef} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -218,7 +218,7 @@ class HeartbeatReceiverSuite assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), null, Matchers.eq( + Matchers.eq(executorId), Matchers.eq(null), Matchers.eq( Array(1L -> metrics)), Matchers.eq(blockManagerId)) } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index d05e90ecf0354..a5770b4d54196 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -83,8 +83,8 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") - val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", null, Seq( - (1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, + val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", new ExecutorMetrics, + Seq((1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)))) testEvent(stageSubmitted, stageSubmittedJsonString) @@ -1619,7 +1619,10 @@ class JsonProtocolSuite extends SparkFunSuite { |{ | "Event": "SparkListenerExecutorMetricsUpdate", | "Executor ID": "exec3", - | "Metrics Updated": [ + | "Executor Metrics Updated": { + | "Executor Hostname": null + | }, + | "Task Metrics Updated": [ | { | "Task ID": 1, | "Stage ID": 2, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index 80d1e88956949..0aab56e519b28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -108,7 +108,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { assert(listener.getExecutionMetrics(0).isEmpty) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", null, Seq( // (task id, stage id, stage attempt, metrics) (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), (1L, 0, 0, createTaskMetrics(accumulatorUpdates)) @@ -116,7 +116,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { assert(listener.getExecutionMetrics(0) === accumulatorUpdates.mapValues(_ * 2)) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", null, Seq( // (task id, stage id, stage attempt, metrics) (0L, 0, 0, createTaskMetrics(accumulatorUpdates)), (1L, 0, 0, createTaskMetrics(accumulatorUpdates.mapValues(_ * 2))) @@ -127,7 +127,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { // Retrying a stage should reset the metrics listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 1))) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", null, Seq( // (task id, stage id, stage attempt, metrics) (0L, 0, 1, createTaskMetrics(accumulatorUpdates)), (1L, 0, 1, createTaskMetrics(accumulatorUpdates)) @@ -167,7 +167,7 @@ class SQLListenerSuite extends SparkFunSuite with SharedSQLContext { // Summit a new stage listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(1, 0))) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", Seq( + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate("", null, Seq( // (task id, stage id, stage attempt, metrics) (0L, 1, 0, createTaskMetrics(accumulatorUpdates)), (1L, 1, 0, createTaskMetrics(accumulatorUpdates)) From 7b846a273cfe6a95f8db7369f96d9a210fa80fd1 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 9 Oct 2015 09:45:24 +0800 Subject: [PATCH 11/28] work with JavaConverters --- .../network/netty/NettyBlockTransferService.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 00d16db39aa79..75700cf85cc81 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -64,10 +64,11 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val currentTime = clock.getTimeMillis() val clientPooledAllocator = clientFactory.getPooledAllocator() val serverAllocator = server.getAllocator() - val clientDirectHeapSize: Long = sumOfMetrics(clientPooledAllocator.directArenas()) - val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas()) - val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas()) - val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas()) + val clientDirectHeapSize: Long = sumOfMetrics( + clientPooledAllocator.directArenas().asScala.toList) + val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas().asScala.toList) + val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) + val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas().asScala.toList) executorMetrics.setTransportMetrics(Some(TransportMetrics(currentTime, clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize))) logDebug(s"current Netty client directHeapSize is $clientDirectHeapSize, " + @@ -78,8 +79,8 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { arenaMetricList.map { Arena => - Arena.chunkLists().map { chunk => - chunk.iterator().map(_.chunkSize()).sum + Arena.chunkLists().asScala.map { chunk => + chunk.iterator().asScala.map(_.chunkSize()).sum }.sum }.sum } From 27b7da1b775b0c4101af3f1f6ee454668a49cdc1 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 2 Nov 2015 23:07:48 +0800 Subject: [PATCH 12/28] refine the code according to Imran's comments and the design doc --- .../org/apache/spark/executor/Executor.scala | 5 +- .../spark/executor/ExecutorMetrics.scala | 6 +- .../netty/NettyBlockTransferService.scala | 15 ++--- .../scheduler/EventLoggingListener.scala | 60 ++++++++--------- .../apache/spark/ui/memory/MemoryTab.scala | 65 +++++++++---------- .../apache/spark/ui/memory/MemoryTable.scala | 20 +++--- .../org/apache/spark/util/JsonProtocol.scala | 18 ++--- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- 8 files changed, 90 insertions(+), 101 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index ca6c277df08b3..21db8317075f6 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -437,7 +437,7 @@ private[spark] class Executor( metrics.updateAccumulators() if (isLocal) { - // JobProgressListener will hold an reference of it during + // JobProgressListener will hold a reference of it during // onExecutorMetricsUpdate(), then JobProgressListener can not see // the changes of metrics any more, so make a deep copy of it val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) @@ -452,6 +452,9 @@ private[spark] class Executor( env.blockTransferService.getMemMetrics(this.executorMetrics) val executorMetrics = if (isLocal) { + // JobProgressListener might hold a reference of it during onExecutorMetricsUpdate() + // in future, if then JobProgressListener can not see the changes of metrics any + // more, so make a deep copy of it here for future change. Utils.deserialize[ExecutorMetrics](Utils.serialize(this.executorMetrics)) } else { this.executorMetrics diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index a7d1da4bc6b61..f5e1b0cce0cb4 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -50,7 +50,5 @@ class ExecutorMetrics extends Serializable { @DeveloperApi case class TransportMetrics( timeStamp: Long, - clientOnheapSize: Long, - clientDirectheapSize: Long, - serverOnheapSize: Long, - serverDirectheapSize: Long) + onHeapSize: Long, + directSize: Long) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 75700cf85cc81..619719d6f2b65 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -22,7 +22,7 @@ import scala.concurrent.{Future, Promise} import io.netty.buffer._ -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf, SparkEnv} import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer @@ -32,7 +32,6 @@ import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.network.shuffle.protocol.UploadBlock import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.SparkEnv import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.{Clock, Utils, SystemClock} @@ -64,17 +63,17 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val currentTime = clock.getTimeMillis() val clientPooledAllocator = clientFactory.getPooledAllocator() val serverAllocator = server.getAllocator() - val clientDirectHeapSize: Long = sumOfMetrics( + val clientDirectSize: Long = sumOfMetrics( clientPooledAllocator.directArenas().asScala.toList) val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas().asScala.toList) - val serverDirectHeapSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) + val serverDirectSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas().asScala.toList) - executorMetrics.setTransportMetrics(Some(TransportMetrics(currentTime, - clientOnHeapSize, clientDirectHeapSize, serverOnHeapSize, serverDirectHeapSize))) - logDebug(s"current Netty client directHeapSize is $clientDirectHeapSize, " + - s"client heapSize is $clientOnHeapSize, server directHeapsize is $serverDirectHeapSize, " + + logDebug(s"Current Netty Client directSize is $clientDirectSize, " + + s"Client HeapSize is $clientOnHeapSize, server directHeapsize is $serverDirectSize, " + s"server heapsize is $serverOnHeapSize, executer id is " + s"${SparkEnv.get.blockManager.blockManagerId.executorId}") + executorMetrics.setTransportMetrics(Some(TransportMetrics(currentTime, + clientOnHeapSize + serverOnHeapSize, clientDirectSize + serverDirectSize))) } private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 606565da1d43c..833bbb9478a5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI -import akka.remote.transport.Transport import org.apache.spark.executor.TransportMetrics import scala.collection.mutable @@ -97,8 +96,9 @@ private[spark] class EventLoggingListener( private[scheduler] val logPath = getLogPath( logBaseDir, appId, appAttemptId, compressionCodecName) - private val latestMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] - private val modifiedMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] + private val executorIdToLatestMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] + private val executorIdToModifiedMaxMetrics = new + HashMap[String, SparkListenerExecutorMetricsUpdate] /** * Creates the log file in the configured log directory. @@ -161,17 +161,23 @@ private[spark] class EventLoggingListener( } } - // We log the event both when stage submitted and stage completed, and after each logEvent call, - // replace the modifiedMetrics with the latestMetrics. In case the stages submit and complete - // time might be interleaved. So as to make the result the same with the running time. - private def logMetricsUpdateEvent() : Unit = { - modifiedMetrics.map(metrics => logEvent(metrics._2)) - latestMetrics.map(metrics => modifiedMetrics.update(metrics._1, metrics._2)) + // When a stage is submitted and completed, we updated our executor memory metrics for that stage, + // and then log the metrics. Anytime we receive more executor metrics, we update our running set of + // {{executorIdToLatestMetrics}} and {{executorIdToModifiedMaxMetrics}}. Since stages submit and + // complete time might be interleaved, we maintain the latest and max metrics for each time segment. + // So, for each stage start and stage complete, we replace each item in + // {{executorIdToModifiedMaxMetrics}} with that in {{executorIdToLatestMetrics}}. + private def updateAndLogExecutorMemoryMetrics() : Unit = { + executorIdToModifiedMaxMetrics.foreach { case(_, metrics) => logEvent(metrics) } + executorIdToLatestMetrics.foreach {case(_, metrics) => logEvent(metrics) } + executorIdToLatestMetrics.foreach { case (executorId, metrics) => + executorIdToModifiedMaxMetrics.update(executorId, metrics) + } } // Events that do not trigger a flush override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { - logMetricsUpdateEvent() + updateAndLogExecutorMemoryMetrics() logEvent(event) } @@ -185,7 +191,7 @@ private[spark] class EventLoggingListener( // Events that trigger a flush override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { - logMetricsUpdateEvent() + updateAndLogExecutorMemoryMetrics() logEvent(event, flushLogger = true) } @@ -218,8 +224,8 @@ private[spark] class EventLoggingListener( } override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { - latestMetrics.remove(event.executorId) - modifiedMetrics.remove(event.executorId) + executorIdToLatestMetrics.remove(event.executorId) + executorIdToModifiedMaxMetrics.remove(event.executorId) logEvent(event, flushLogger = true) } @@ -228,7 +234,7 @@ private[spark] class EventLoggingListener( // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - latestMetrics.update(event.execId, event) + executorIdToLatestMetrics.update(event.execId, event) updateModifiedMetrics(event.execId) } @@ -258,10 +264,10 @@ private[spark] class EventLoggingListener( * @param executorId the executor whose metrics will be modified */ private def updateModifiedMetrics(executorId: String): Unit = { - val toBeModifiedEvent = modifiedMetrics.get(executorId) - val latestEvent = latestMetrics.get(executorId) + val toBeModifiedEvent = executorIdToModifiedMaxMetrics.get(executorId) + val latestEvent = executorIdToLatestMetrics.get(executorId) if (toBeModifiedEvent.isEmpty) { - if (latestEvent.isDefined) modifiedMetrics.update(executorId, latestEvent.get) + if (latestEvent.isDefined) executorIdToModifiedMaxMetrics.update(executorId, latestEvent.get) } else { val toBeModifiedMetrics = toBeModifiedEvent.get.executorMetrics.transportMetrics if (toBeModifiedMetrics.isDefined) { @@ -270,29 +276,23 @@ private[spark] class EventLoggingListener( val toBeModTransMetrics = toBeModifiedMetrics.get var timeStamp: Long = toBeModTransMetrics.timeStamp // the logic here should be the same with that for memoryListener - val (clientOnheapSize, serverOnheapSize) = - if (latestTransMetrics.clientOnheapSize + latestTransMetrics.serverOnheapSize > - toBeModTransMetrics.clientOnheapSize + toBeModTransMetrics.serverOnheapSize) { + val onHeapSize = if (latestTransMetrics.onHeapSize > toBeModTransMetrics.onHeapSize) { timeStamp = latestTransMetrics.timeStamp - (latestTransMetrics.clientOnheapSize, latestTransMetrics.serverOnheapSize) + latestTransMetrics.onHeapSize } else { - (toBeModTransMetrics.clientOnheapSize, toBeModTransMetrics.serverOnheapSize) + toBeModTransMetrics.onHeapSize } - val (clientDirectheapSize, serverDirectheapSize) = - if (latestTransMetrics.clientDirectheapSize + latestTransMetrics.serverDirectheapSize > - toBeModTransMetrics.clientDirectheapSize + toBeModTransMetrics.serverDirectheapSize) { + val directSize = if (latestTransMetrics.directSize > toBeModTransMetrics.directSize) { timeStamp = latestTransMetrics.timeStamp - (latestTransMetrics.clientDirectheapSize, latestTransMetrics.serverDirectheapSize) + latestTransMetrics.directSize } else { - (toBeModTransMetrics.clientDirectheapSize, toBeModTransMetrics.serverDirectheapSize) + toBeModTransMetrics.directSize } toBeModifiedEvent.get.executorMetrics.setTransportMetrics( - Some(TransportMetrics(timeStamp, clientOnheapSize, clientDirectheapSize, - serverOnheapSize, serverDirectheapSize))) + Some(TransportMetrics(timeStamp, onHeapSize, directSize))) } } } - } private[spark] object EventLoggingListener extends Logging { diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index c423dcf0916d9..51eac0e3e9088 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -41,11 +41,11 @@ class MemoryListener extends SparkListener { type ExecutorId = String val activeExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] val removedExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] - // latestExecIdToExecMetrics include all executors that is active and removed. + // latestExecIdToExecMetrics including all executors that is active and removed. // this may consume a lot of memory when executors are changing frequently, e.g. in dynamical // allocation mode. val latestExecIdToExecMetrics = new HashMap[ExecutorId, ExecutorMetrics] - // stagesIdToMem a map maintains all executors memory information of each stage, + // activeStagesToMem a map maintains all executors memory information of each stage, // the Map type is [(stageId, attemptId), Seq[(executorId, MemoryUIInfo)] val activeStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] val completedStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] @@ -55,10 +55,9 @@ class MemoryListener extends SparkListener { val executorMetrics = event.executorMetrics val memoryInfo = activeExecutorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) memoryInfo.updateExecutorMetrics(executorMetrics) - activeStagesToMem.map {stageToMem => - if (stageToMem._2.contains(executorId)) { - val memInfo = stageToMem._2.get(executorId).get - memInfo.updateExecutorMetrics(executorMetrics) + activeStagesToMem.foreach { case (_, stageMemMetrics) => + if(stageMemMetrics.contains(executorId)) { + stageMemMetrics.get(executorId).get.updateExecutorMetrics(executorMetrics) } } latestExecIdToExecMetrics.update(executorId, executorMetrics) @@ -84,21 +83,19 @@ class MemoryListener extends SparkListener { override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) val memInfoMap = new HashMap[ExecutorId, MemoryUIInfo] - activeExecutorIdToMem.map(idToMem => memInfoMap.update(idToMem._1, new MemoryUIInfo)) + activeExecutorIdToMem.foreach(idToMem => memInfoMap.update(idToMem._1, new MemoryUIInfo)) activeStagesToMem.update(stage, memInfoMap) } override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) - val memInfoMap = activeStagesToMem.get(stage) - if (memInfoMap.isDefined) { - activeExecutorIdToMem.map { idToMem => - val executorId = idToMem._1 - val memInfo = memInfoMap.get.getOrElse(executorId, new MemoryUIInfo) - if (latestExecIdToExecMetrics.contains(executorId)) { - memInfo.updateExecutorMetrics(latestExecIdToExecMetrics.get(executorId).get) + activeStagesToMem.get(stage).map { memInfoMap => + activeExecutorIdToMem.foreach { case (executorId, _) => + val memInfo = memInfoMap.getOrElse(executorId, new MemoryUIInfo) + latestExecIdToExecMetrics.get(executorId).foreach { prevExecutorMetrics => + memInfo.updateExecutorMetrics(prevExecutorMetrics) } - memInfoMap.get.update(executorId, memInfo) + memInfoMap.update(executorId, memInfo) } completedStagesToMem.put(stage, activeStagesToMem.remove(stage).get) } @@ -107,7 +104,7 @@ class MemoryListener extends SparkListener { class MemoryUIInfo { var executorAddress: String = _ - var transportInfo: Option[transportMemSize] = None + var transportInfo: Option[TransportMemSize] = None def this(execInfo: ExecutorInfo) = { this() @@ -115,38 +112,34 @@ class MemoryUIInfo { } def updateExecutorMetrics(execMetrics: ExecutorMetrics): Unit = { - if (execMetrics.transportMetrics.isDefined) { + execMetrics.transportMetrics.map { transPortMetrics => transportInfo = transportInfo match { case Some(transportMemSize) => transportInfo - case _ => Some(new transportMemSize) + case _ => Some(new TransportMemSize) } executorAddress = execMetrics.hostname - if (execMetrics.transportMetrics.isDefined) { - transportInfo.get.updateTransport(execMetrics.transportMetrics.get) - } + transportInfo.get.updateTransport(transPortMetrics) } } } -class transportMemSize { - var onheapSize: Long = _ - var directheapSize: Long = _ - var peakOnheapSizeTime: MemTime = new MemTime() - var peakDirectheapSizeTime: MemTime = new MemTime() +class TransportMemSize { + var onHeapSize: Long = _ + var directSize: Long = _ + var peakOnHeapSizeTime: MemTime = new MemTime() + var peakDirectSizeTime: MemTime = new MemTime() def updateTransport(transportMetrics: TransportMetrics): Unit = { - val updatedOnheapSize = transportMetrics.clientOnheapSize + - transportMetrics.serverOnheapSize - val updatedDirectheapSize = transportMetrics.clientDirectheapSize + - transportMetrics.serverDirectheapSize + val updatedOnHeapSize = transportMetrics.onHeapSize + val updatedDirectSize = transportMetrics.directSize val updateTime: Long = transportMetrics.timeStamp - onheapSize = updatedOnheapSize - directheapSize = updatedDirectheapSize - if (updatedOnheapSize >= peakOnheapSizeTime.memorySize) { - peakOnheapSizeTime = MemTime(updatedOnheapSize, updateTime) + onHeapSize = updatedOnHeapSize + directSize = updatedDirectSize + if (updatedOnHeapSize >= peakOnHeapSizeTime.memorySize) { + peakOnHeapSizeTime = MemTime(updatedOnHeapSize, updateTime) } - if (updatedDirectheapSize >= peakDirectheapSizeTime.memorySize) { - peakDirectheapSizeTime = MemTime(updatedDirectheapSize, updateTime) + if (updatedDirectSize >= peakDirectSizeTime.memorySize) { + peakDirectSizeTime = MemTime(updatedDirectSize, updateTime) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala index 3972dd47f91b7..24396b85f97dc 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -34,10 +34,10 @@ private[ui] class MemTableBase( protected def columns: Seq[Node] = { Executor ID Address - Net Memory (on-heap) - Net Memory (direct-heap) - Peak Net Memory (on-heap) / Happen Time - Peak Net Read (direct-heap) / Happen Time + Network Memory (on-heap) + Network Memory (direct-heap) + Peak Network Memory (on-heap) / Happen Time + Peak Network Read (direct-heap) / Happen Time } def toNodeSeq: Seq[Node] = { @@ -68,20 +68,20 @@ private[ui] class MemTableBase( {if (info._2.transportInfo.isDefined) { - {Utils.bytesToString(info._2.transportInfo.get.onheapSize)} + {Utils.bytesToString(info._2.transportInfo.get.onHeapSize)} - {Utils.bytesToString(info._2.transportInfo.get.directheapSize)} + {Utils.bytesToString(info._2.transportInfo.get.directSize)} - {Utils.bytesToString(info._2.transportInfo.get.peakOnheapSizeTime.memorySize)} + {Utils.bytesToString(info._2.transportInfo.get.peakOnHeapSizeTime.memorySize)} / - {UIUtils.formatDate(info._2.transportInfo.get.peakOnheapSizeTime.timeStamp)} + {UIUtils.formatDate(info._2.transportInfo.get.peakOnHeapSizeTime.timeStamp)} - {Utils.bytesToString(info._2.transportInfo.get.peakDirectheapSizeTime.memorySize)} + {Utils.bytesToString(info._2.transportInfo.get.peakDirectSizeTime.memorySize)} / - {UIUtils.formatDate(info._2.transportInfo.get.peakDirectheapSizeTime.timeStamp)} + {UIUtils.formatDate(info._2.transportInfo.get.peakDirectSizeTime.timeStamp)} } else { N/A diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 777c399c6f414..a8a6d3d83742d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -232,7 +232,7 @@ private[spark] object JsonProtocol { ("Event" -> Utils.getFormattedClassName(metricsUpdate)) ~ ("Executor ID" -> execId) ~ ("Executor Metrics Updated" -> executorMetricsToJson(executorMetrics)) ~ - ("Task Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => + ("Metrics Updated" -> taskMetrics.map { case (taskId, stageId, stageAttemptId, metrics) => ("Task ID" -> taskId) ~ ("Stage ID" -> stageId) ~ ("Stage Attempt ID" -> stageAttemptId) ~ @@ -296,10 +296,8 @@ private[spark] object JsonProtocol { def transportMetricsToJson(transportMetrics: TransportMetrics): JValue = { ("TimeStamep" -> transportMetrics.timeStamp) ~ - ("ClientOnheapSize" -> transportMetrics.clientOnheapSize) ~ - ("ClientDirectheapSize" -> transportMetrics.clientDirectheapSize) ~ - ("ServerOnheapSize" -> transportMetrics.serverOnheapSize) ~ - ("ServerDirectheapSize" -> transportMetrics.serverDirectheapSize) + ("OnHeapSize" -> transportMetrics.onHeapSize) ~ + ("DirectSize" -> transportMetrics.directSize) } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { @@ -722,17 +720,15 @@ private[spark] object JsonProtocol { val metrics = new ExecutorMetrics metrics.setHostname((json \ "Executor Hostname").extract[String]) metrics.setTransportMetrics( - Utils.jsonOption((json \ "TransportMetrics")).map(transportMetrisFromJson)) + Utils.jsonOption((json \ "TransportMetrics")).map(transportMetricsFromJson)) metrics } - def transportMetrisFromJson(json: JValue): TransportMetrics = { + def transportMetricsFromJson(json: JValue): TransportMetrics = { val metrics = new TransportMetrics( (json \ "TimeStamep").extract[Long], - (json \ "ClientOnheapSize").extract[Long], - (json \ "ClientDirectheapSize").extract[Long], - (json \ "ServerOnheapSize").extract[Long], - (json \ "ServerDirectheapSize").extract[Long]) + (json \ "OnHeapSize").extract[Long], + (json \ "DirectSize").extract[Long]) metrics } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 96c66bd89cf31..4746f8b5cf9d4 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -1677,7 +1677,7 @@ class JsonProtocolSuite extends SparkFunSuite { | "Executor Metrics Updated": { | "Executor Hostname": null | }, - | "Task Metrics Updated": [ + | "Metrics Updated": [ | { | "Task ID": 1, | "Stage ID": 2, From f2f0e643a32fadee1195824c525a276d348c4077 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 3 Nov 2015 09:26:52 +0800 Subject: [PATCH 13/28] fix scala style test --- .../spark/scheduler/EventLoggingListener.scala | 13 +++++++------ .../org/apache/spark/ui/memory/MemoryTab.scala | 2 +- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 833bbb9478a5c..33b2a777f1c10 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -161,12 +161,13 @@ private[spark] class EventLoggingListener( } } - // When a stage is submitted and completed, we updated our executor memory metrics for that stage, - // and then log the metrics. Anytime we receive more executor metrics, we update our running set of - // {{executorIdToLatestMetrics}} and {{executorIdToModifiedMaxMetrics}}. Since stages submit and - // complete time might be interleaved, we maintain the latest and max metrics for each time segment. - // So, for each stage start and stage complete, we replace each item in - // {{executorIdToModifiedMaxMetrics}} with that in {{executorIdToLatestMetrics}}. + // When a stage is submitted and completed, we updated our executor memory metrics for that + // stage, and then log the metrics. Anytime we receive more executor metrics, we update our + // running set of {{executorIdToLatestMetrics}} and {{executorIdToModifiedMaxMetrics}}. + // Since stages submit and complete time might be interleaved, we maintain the latest and + // max metrics for each time segment. So, for each stage start and stage complete, we + // replace each item in {{executorIdToModifiedMaxMetrics}} with that + // in {{executorIdToLatestMetrics}}. private def updateAndLogExecutorMemoryMetrics() : Unit = { executorIdToModifiedMaxMetrics.foreach { case(_, metrics) => logEvent(metrics) } executorIdToLatestMetrics.foreach {case(_, metrics) => logEvent(metrics) } diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 51eac0e3e9088..0db83fab23d94 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -56,7 +56,7 @@ class MemoryListener extends SparkListener { val memoryInfo = activeExecutorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) memoryInfo.updateExecutorMetrics(executorMetrics) activeStagesToMem.foreach { case (_, stageMemMetrics) => - if(stageMemMetrics.contains(executorId)) { + if (stageMemMetrics.contains(executorId)) { stageMemMetrics.get(executorId).get.updateExecutorMetrics(executorMetrics) } } From 5f7a99970ecaf517fadd07568aca1f95cd625f57 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 3 Nov 2015 10:17:05 +0800 Subject: [PATCH 14/28] capitalize class name --- .../main/scala/org/apache/spark/ui/memory/MemoryPage.scala | 4 ++-- .../main/scala/org/apache/spark/ui/memory/MemoryTable.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala index b6dbd6cd53b5c..b674f074fe62e 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryPage.scala @@ -44,9 +44,9 @@ private[ui] class MemoryPage(parent: MemoryTab) extends WebUIPage("") { val activeExecMemTable = new MemTableBase(activeMemInfoSorted, memoryListener) val removedExecMemTable = new MemTableBase(removedMemInfoSorted, memoryListener) - val completedStagesTable = new stagesTableBase( + val completedStagesTable = new StagesTableBase( completedStages, parent.basePath, progressListener) - val failedStagesTable = new stagesTableBase(failedStages, parent.basePath, progressListener) + val failedStagesTable = new StagesTableBase(failedStages, parent.basePath, progressListener) val summary: NodeSeq =
    diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala index 24396b85f97dc..4785dc7de9c61 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -93,7 +93,7 @@ private[ui] class MemTableBase( } } -private[ui] class stagesTableBase( +private[ui] class StagesTableBase( stageInfos: Seq[StageInfo], basePath: String, listener: JobProgressListener) { From 5ad7a6a35e4439b1d8345d0c32a89d5f40bc7b25 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 3 Nov 2015 11:05:34 +0800 Subject: [PATCH 15/28] change task metrics json format back to origin --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a8a6d3d83742d..c226e29217e46 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -638,7 +638,7 @@ private[spark] object JsonProtocol { def executorMetricsUpdateFromJson(json: JValue): SparkListenerExecutorMetricsUpdate = { val execInfo = (json \ "Executor ID").extract[String] val executorMetrics = executorMetricsFromJson(json \ "Executor Metrics Updated") - val taskMetrics = (json \ "Task Metrics Updated").extract[List[JValue]].map { json => + val taskMetrics = (json \ "Metrics Updated").extract[List[JValue]].map { json => val taskId = (json \ "Task ID").extract[Long] val stageId = (json \ "Stage ID").extract[Int] val stageAttemptId = (json \ "Stage Attempt ID").extract[Int] From 1dffa29ae4d81c7924bdc66a187bd9324fd2792f Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 17 Nov 2015 17:54:16 +0800 Subject: [PATCH 16/28] accroding to Imran's comment, refine the code --- .../spark/executor/ExecutorMetrics.scala | 23 +++++++--- .../netty/NettyBlockTransferService.scala | 12 ++--- .../scheduler/EventLoggingListener.scala | 44 +++++++++---------- .../apache/spark/ui/memory/MemoryTab.scala | 26 +++++------ .../apache/spark/ui/memory/MemoryTable.scala | 6 +-- .../org/apache/spark/util/JsonProtocol.scala | 10 ++--- 6 files changed, 62 insertions(+), 59 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index f5e1b0cce0cb4..34710007e5179 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -36,9 +36,9 @@ class ExecutorMetrics extends Serializable { def hostname: String = _hostname private[spark] def setHostname(value: String) = _hostname = value - private var _transportMetrics: Option[TransportMetrics] = None - def transportMetrics: Option[TransportMetrics] = _transportMetrics - private[spark] def setTransportMetrics(value: Option[TransportMetrics]) = { + private var _transportMetrics: TransportMetrics = new TransportMetrics + def transportMetrics: TransportMetrics = _transportMetrics + private[spark] def setTransportMetrics(value: TransportMetrics) = { _transportMetrics = value } } @@ -48,7 +48,16 @@ class ExecutorMetrics extends Serializable { * Metrics for network layer */ @DeveloperApi -case class TransportMetrics( - timeStamp: Long, - onHeapSize: Long, - directSize: Long) +class TransportMetrics ( + val timeStamp: Long = System.currentTimeMillis, + val onHeapSize: Long = 0L, + val offHeapSize: Long = 0L) + +object TransportMetrics { + def apply( + timeStamp: Long, + onHeapSize: Long, + offHeapSize: Long): TransportMetrics = { + new TransportMetrics(timeStamp, onHeapSize, offHeapSize) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 619719d6f2b65..96597738e1d3a 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -63,17 +63,17 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val currentTime = clock.getTimeMillis() val clientPooledAllocator = clientFactory.getPooledAllocator() val serverAllocator = server.getAllocator() - val clientDirectSize: Long = sumOfMetrics( + val clientOffHeapSize: Long = sumOfMetrics( clientPooledAllocator.directArenas().asScala.toList) val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas().asScala.toList) - val serverDirectSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) + val serverOffHeapSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas().asScala.toList) - logDebug(s"Current Netty Client directSize is $clientDirectSize, " + - s"Client HeapSize is $clientOnHeapSize, server directHeapsize is $serverDirectSize, " + + logDebug(s"Current Netty Client offHeapSize is $clientOffHeapSize, " + + s"Client HeapSize is $clientOnHeapSize, server directHeapsize is $serverOffHeapSize, " + s"server heapsize is $serverOnHeapSize, executer id is " + s"${SparkEnv.get.blockManager.blockManagerId.executorId}") - executorMetrics.setTransportMetrics(Some(TransportMetrics(currentTime, - clientOnHeapSize + serverOnHeapSize, clientDirectSize + serverDirectSize))) + executorMetrics.setTransportMetrics(TransportMetrics(currentTime, + clientOnHeapSize + serverOnHeapSize, clientOffHeapSize + serverOffHeapSize)) } private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 33b2a777f1c10..94da8b0c2e963 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,8 +20,6 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI -import org.apache.spark.executor.TransportMetrics - import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -34,6 +32,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.TransportMetrics import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} @@ -267,31 +266,30 @@ private[spark] class EventLoggingListener( private def updateModifiedMetrics(executorId: String): Unit = { val toBeModifiedEvent = executorIdToModifiedMaxMetrics.get(executorId) val latestEvent = executorIdToLatestMetrics.get(executorId) - if (toBeModifiedEvent.isEmpty) { - if (latestEvent.isDefined) executorIdToModifiedMaxMetrics.update(executorId, latestEvent.get) - } else { - val toBeModifiedMetrics = toBeModifiedEvent.get.executorMetrics.transportMetrics - if (toBeModifiedMetrics.isDefined) { + toBeModifiedEvent match { + case None => if (latestEvent.isDefined) executorIdToModifiedMaxMetrics.update( + executorId, latestEvent.get) + case Some(toBeModifiedEvent) => + val toBeModifiedMetrics = toBeModifiedEvent.executorMetrics.transportMetrics // latestEvent must has value - val latestTransMetrics = latestEvent.get.executorMetrics.transportMetrics.get - val toBeModTransMetrics = toBeModifiedMetrics.get + val latestTransMetrics = latestEvent.get.executorMetrics.transportMetrics + val toBeModTransMetrics = toBeModifiedMetrics var timeStamp: Long = toBeModTransMetrics.timeStamp // the logic here should be the same with that for memoryListener val onHeapSize = if (latestTransMetrics.onHeapSize > toBeModTransMetrics.onHeapSize) { - timeStamp = latestTransMetrics.timeStamp - latestTransMetrics.onHeapSize - } else { - toBeModTransMetrics.onHeapSize - } - val directSize = if (latestTransMetrics.directSize > toBeModTransMetrics.directSize) { - timeStamp = latestTransMetrics.timeStamp - latestTransMetrics.directSize - } else { - toBeModTransMetrics.directSize - } - toBeModifiedEvent.get.executorMetrics.setTransportMetrics( - Some(TransportMetrics(timeStamp, onHeapSize, directSize))) - } + timeStamp = latestTransMetrics.timeStamp + latestTransMetrics.onHeapSize + } else { + toBeModTransMetrics.onHeapSize + } + val offHeapSize = if (latestTransMetrics.offHeapSize > toBeModTransMetrics.offHeapSize) { + timeStamp = latestTransMetrics.timeStamp + latestTransMetrics.offHeapSize + } else { + toBeModTransMetrics.offHeapSize + } + toBeModifiedEvent.executorMetrics.setTransportMetrics( + TransportMetrics(timeStamp, onHeapSize, offHeapSize)) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 0db83fab23d94..05b3f0eff9022 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -112,36 +112,34 @@ class MemoryUIInfo { } def updateExecutorMetrics(execMetrics: ExecutorMetrics): Unit = { - execMetrics.transportMetrics.map { transPortMetrics => - transportInfo = transportInfo match { - case Some(transportMemSize) => transportInfo - case _ => Some(new TransportMemSize) - } - executorAddress = execMetrics.hostname - transportInfo.get.updateTransport(transPortMetrics) + transportInfo = transportInfo match { + case Some(transportMemSize) => transportInfo + case _ => Some(new TransportMemSize) } + executorAddress = execMetrics.hostname + transportInfo.get.updateTransport(execMetrics.transportMetrics) } } class TransportMemSize { var onHeapSize: Long = _ - var directSize: Long = _ + var offHeapSize: Long = _ var peakOnHeapSizeTime: MemTime = new MemTime() - var peakDirectSizeTime: MemTime = new MemTime() + var peakOffHeapSizeTime: MemTime = new MemTime() def updateTransport(transportMetrics: TransportMetrics): Unit = { val updatedOnHeapSize = transportMetrics.onHeapSize - val updatedDirectSize = transportMetrics.directSize + val updatedOffHeapSize = transportMetrics.offHeapSize val updateTime: Long = transportMetrics.timeStamp onHeapSize = updatedOnHeapSize - directSize = updatedDirectSize + offHeapSize = updatedOffHeapSize if (updatedOnHeapSize >= peakOnHeapSizeTime.memorySize) { peakOnHeapSizeTime = MemTime(updatedOnHeapSize, updateTime) } - if (updatedDirectSize >= peakDirectSizeTime.memorySize) { - peakDirectSizeTime = MemTime(updatedDirectSize, updateTime) + if (updatedOffHeapSize >= peakOffHeapSizeTime.memorySize) { + peakOffHeapSizeTime = MemTime(updatedOffHeapSize, updateTime) } } } -case class MemTime(memorySize: Long = 0L, timeStamp: Long = 0L) +case class MemTime(memorySize: Long = System.currentTimeMillis, timeStamp: Long = 0L) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala index 4785dc7de9c61..cdcb84ff715a4 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -71,7 +71,7 @@ private[ui] class MemTableBase( {Utils.bytesToString(info._2.transportInfo.get.onHeapSize)} - {Utils.bytesToString(info._2.transportInfo.get.directSize)} + {Utils.bytesToString(info._2.transportInfo.get.offHeapSize)} {Utils.bytesToString(info._2.transportInfo.get.peakOnHeapSizeTime.memorySize)} @@ -79,9 +79,9 @@ private[ui] class MemTableBase( {UIUtils.formatDate(info._2.transportInfo.get.peakOnHeapSizeTime.timeStamp)} - {Utils.bytesToString(info._2.transportInfo.get.peakDirectSizeTime.memorySize)} + {Utils.bytesToString(info._2.transportInfo.get.peakOffHeapSizeTime.memorySize)} / - {UIUtils.formatDate(info._2.transportInfo.get.peakDirectSizeTime.timeStamp)} + {UIUtils.formatDate(info._2.transportInfo.get.peakOffHeapSizeTime.timeStamp)} } else { N/A diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2b521e931ee06..028451aef4da6 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -288,8 +288,7 @@ private[spark] object JsonProtocol { } def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - val transportMetrics = executorMetrics.transportMetrics.map( - transportMetricsToJson).getOrElse(JNothing) + val transportMetrics = transportMetricsToJson(executorMetrics.transportMetrics) ("Executor Hostname" -> executorMetrics.hostname) ~ ("TransportMetrics" -> transportMetrics) } @@ -297,7 +296,7 @@ private[spark] object JsonProtocol { def transportMetricsToJson(transportMetrics: TransportMetrics): JValue = { ("TimeStamep" -> transportMetrics.timeStamp) ~ ("OnHeapSize" -> transportMetrics.onHeapSize) ~ - ("DirectSize" -> transportMetrics.directSize) + ("OffHeapSize" -> transportMetrics.offHeapSize) } def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { @@ -720,8 +719,7 @@ private[spark] object JsonProtocol { def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = new ExecutorMetrics metrics.setHostname((json \ "Executor Hostname").extract[String]) - metrics.setTransportMetrics( - Utils.jsonOption((json \ "TransportMetrics")).map(transportMetricsFromJson)) + metrics.setTransportMetrics(transportMetricsFromJson(json \ "TransportMetrics")) metrics } @@ -729,7 +727,7 @@ private[spark] object JsonProtocol { val metrics = new TransportMetrics( (json \ "TimeStamep").extract[Long], (json \ "OnHeapSize").extract[Long], - (json \ "DirectSize").extract[Long]) + (json \ "OffHeapSize").extract[Long]) metrics } From 75e63c366fcee3fed5cfe5c1066ab9727fd1f574 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 17 Nov 2015 17:55:25 +0800 Subject: [PATCH 17/28] add first test case --- .../spark/ui/memory/MemoryListenerSuite.scala | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala new file mode 100644 index 0000000000000..e5c1758778451 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.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.ui.memory + +import java.util.Properties + +import org.scalatest.Matchers + +import org.apache.spark._ +import org.apache.spark.{LocalSparkContext, SparkConf, Success} +import org.apache.spark.executor._ +import org.apache.spark.scheduler._ +import org.apache.spark.util.Utils + +class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { + private def createStageStartEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") + SparkListenerStageSubmitted(stageInfo) + } + + private def createStageEndEvent(stageId: Int, failed: Boolean = false) = { + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") + if (failed) { + stageInfo.failureReason = Some("Failed!") + } + SparkListenerStageCompleted(stageInfo) + } + + private def createExecutorMetricsUpdateEvent( + execId: String, + executorMetrics: ExecutorMetrics) = { + SparkListenerExecutorMetricsUpdate(execId, executorMetrics, null) + } + + test("test HashMap size for MemoryListener") { + val listener = new MemoryListener + val execId1 = "exec-1" + val execId2 = "exec-2" + + (1 to 2).foreach { i => + listener.onStageSubmitted(createStageStartEvent(i)) + listener.onStageCompleted(createStageEndEvent(i)) + } + // stages are all completed, no activeStages now + assert(listener.activeStagesToMem.isEmpty) + + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( + execId1, new ExecutorMetrics, null)) + // ExecutorMetrics is not related with Stages directly + assert(listener.activeStagesToMem.isEmpty) + + listener.onStageSubmitted(createStageStartEvent(3)) + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( + execId2, new ExecutorMetrics, null)) + // totally 2 executors updated their metrics + assert(listener.activeExecutorIdToMem.size == 2) + assert(listener.activeStagesToMem.size == 1) + listener.onStageCompleted(createStageEndEvent(3)) + + assert(listener.activeStagesToMem.isEmpty) + assert(listener.completedStagesToMem.size == 3) + assert(listener.activeExecutorIdToMem.size == listener.latestExecIdToExecMetrics.size) + assert(listener.removedExecutorIdToMem.isEmpty) + } + +} \ No newline at end of file From 0c1241ca4512032598371a18df1ea7ac8ef98b8a Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 18 Nov 2015 00:34:15 +0800 Subject: [PATCH 18/28] fix scala style --- .../spark/ui/memory/MemoryListenerSuite.scala | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index e5c1758778451..ac84ece4390e9 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -47,6 +47,17 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc SparkListenerExecutorMetricsUpdate(execId, executorMetrics, null) } + private def createExecutorMetrics( + hostname: String, + timeStamp: Long, + onHeapSize: Long, + offHeapSize: Long): ExecutorMetrics = { + val execMetrics = new ExecutorMetrics + execMetrics.setHostname(hostname) + execMetrics.setTransportMetrics(TransportMetrics(timeStamp, onHeapSize, offHeapSize)) + execMetrics + } + test("test HashMap size for MemoryListener") { val listener = new MemoryListener val execId1 = "exec-1" @@ -59,14 +70,14 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc // stages are all completed, no activeStages now assert(listener.activeStagesToMem.isEmpty) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( - execId1, new ExecutorMetrics, null)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, new ExecutorMetrics)) // ExecutorMetrics is not related with Stages directly assert(listener.activeStagesToMem.isEmpty) listener.onStageSubmitted(createStageStartEvent(3)) - listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate( - execId2, new ExecutorMetrics, null)) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId2, new ExecutorMetrics)) // totally 2 executors updated their metrics assert(listener.activeExecutorIdToMem.size == 2) assert(listener.activeStagesToMem.size == 1) @@ -77,5 +88,4 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc assert(listener.activeExecutorIdToMem.size == listener.latestExecIdToExecMetrics.size) assert(listener.removedExecutorIdToMem.isEmpty) } - -} \ No newline at end of file +} From c78628ee70dc20186c03253ec2eada430779d3bb Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 19 Nov 2015 10:52:37 +0800 Subject: [PATCH 19/28] add more test cases, with eventloging test left --- .../apache/spark/ui/memory/MemoryTab.scala | 8 +- .../spark/ui/memory/MemoryListenerSuite.scala | 175 +++++++++++++++++- 2 files changed, 172 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 05b3f0eff9022..e5095510c8d93 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -74,12 +74,6 @@ class MemoryListener extends SparkListener { removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) } - override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved): Unit = { - val executorId = event.blockManagerId.executorId - val info = activeExecutorIdToMem.remove(executorId) - removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) - } - override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) val memInfoMap = new HashMap[ExecutorId, MemoryUIInfo] @@ -142,4 +136,4 @@ class TransportMemSize { } } -case class MemTime(memorySize: Long = System.currentTimeMillis, timeStamp: Long = 0L) +case class MemTime(memorySize: Long = 0L, timeStamp: Long = System.currentTimeMillis) diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index ac84ece4390e9..26c5f813bf984 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.ui.memory -import java.util.Properties - import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor._ import org.apache.spark.scheduler._ -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.cluster._ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { private def createStageStartEvent(stageId: Int) = { @@ -88,4 +85,174 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc assert(listener.activeExecutorIdToMem.size == listener.latestExecIdToExecMetrics.size) assert(listener.removedExecutorIdToMem.isEmpty) } + + test("test first stage with no executor metrics update") { + val listener = new MemoryListener + val execId1 = "exec-1" + + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + + // stage 1, no metrics update + listener.onStageSubmitted(createStageStartEvent(1)) + listener.onStageCompleted(createStageEndEvent(1)) + + // stage 2, with one metrics update + listener.onStageSubmitted(createStageStartEvent(2)) + val execMetrics = createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics)) + listener.onStageCompleted(createStageEndEvent(2)) + + val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get + // no metrics for stage 1 since no metrics update for stage 1 + assert(mapForStage1.get(execId1).get.transportInfo == None) + val mapForStage2 = listener.completedStagesToMem.get((2,0)).get + assert(mapForStage2.size == 1) + val memInfo = mapForStage2.get(execId1).get + assert(memInfo.transportInfo.isDefined) + val transMetrics = memInfo.transportInfo.get + assert((20, 10, MemTime(20, 0), MemTime(10, 0)) == (transMetrics.onHeapSize, + transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) + + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) + } + + test("test multiple executors metrics updated in one stage") { + val listener = new MemoryListener + val execId1 = "exec-1" + + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + + // multiple metrics updated in one stage + listener.onStageSubmitted(createStageStartEvent(1)) + val execMetrics1 = createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics1)) + val execMetrics2 = createExecutorMetrics("host-1", 0L, 30, 5) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics2)) + val execMetrics3 = createExecutorMetrics("host-1", 0L, 15, 15) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics3)) + listener.onStageCompleted(createStageEndEvent(1)) + + val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get + val memInfo = mapForStage1.get(execId1).get + assert(memInfo.transportInfo.isDefined) + val transMetrics = memInfo.transportInfo.get + assert((15, 15, MemTime(30, 0), MemTime(15, 0)) == (transMetrics.onHeapSize, + transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) + + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) + } + + test("test stages use executor metrics updated in previous stages") { + val listener = new MemoryListener + val execId1 = "exec-1" + + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + + // multiple metrics updated in one stage + listener.onStageSubmitted(createStageStartEvent(1)) + val execMetrics1 = createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics1)) + val execMetrics2 = createExecutorMetrics("host-1", 0L, 30, 5) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics2)) + val execMetrics3 = createExecutorMetrics("host-1", 0L, 15, 15) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, execMetrics3)) + listener.onStageCompleted(createStageEndEvent(1)) + + // stage 2 and stage 3 don't get metrics + listener.onStageSubmitted(createStageStartEvent(2)) + listener.onStageCompleted(createStageEndEvent(2)) + listener.onStageSubmitted(createStageStartEvent(3)) + listener.onStageCompleted(createStageEndEvent(3)) + + // both stage 2 and stage 3 will use the metrics last updated in stage 1 + val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get + val memInfo2 = mapForStage2.get(execId1).get + assert(memInfo2.transportInfo.isDefined) + val transMetrics2 = memInfo2.transportInfo.get + assert((15, 15, MemTime(15, 0), MemTime(15, 0)) == (transMetrics2.onHeapSize, + transMetrics2.offHeapSize, + transMetrics2.peakOnHeapSizeTime, + transMetrics2.peakOffHeapSizeTime)) + + val mapForStage3 = listener.completedStagesToMem.get((3, 0)).get + val memInfo3 = mapForStage3.get(execId1).get + assert(memInfo3.transportInfo.isDefined) + val transMetrics3 = memInfo3.transportInfo.get + assert((15, 15, MemTime(15, 0), MemTime(15, 0)) == (transMetrics3.onHeapSize, + transMetrics3.offHeapSize, + transMetrics3.peakOnHeapSizeTime, + transMetrics3.peakOffHeapSizeTime)) + + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) + } + + test("test multiple executors") { + val listener = new MemoryListener + val execId1 = "exec-1" + val execId2 = "exec-2" + val execId3 = "exec-3" + + // two executors added first + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId2, new ExecutorInfo("host2", 1, Map.empty))) + + // two executors running in one stage and one executor is removed before stage complete + listener.onStageSubmitted(createStageStartEvent(1)) + val exec1Metrics = createExecutorMetrics("host-1", 1446336000L, 20, 10) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId1, exec1Metrics)) + val exec2Metrics = createExecutorMetrics("host-2", 1446337000L, 15, 5) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId2, exec2Metrics)) + // on more executor added when stage is running + listener.onExecutorAdded( + SparkListenerExecutorAdded(0L, execId3, new ExecutorInfo("host3", 1, Map.empty))) + val exec3Metrics = createExecutorMetrics("host-3", 1446338000L, 30, 15) + listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + execId3, exec3Metrics)) + // executor 2 removed before stage complete + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId2, "")) + listener.onStageCompleted(createStageEndEvent(1)) + + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) + listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId3, "")) + + // the completedStagesToMem will maintain the metrics of both the removed executors and new + // add executors + val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get + assert(mapForStage1.size == 3) + val memInfo1 = mapForStage1.get(execId1).get + val memInfo2 = mapForStage1.get(execId2).get + val memInfo3 = mapForStage1.get(execId3).get + val transMetrics1 = memInfo1.transportInfo.get + val transMetrics2 = memInfo2.transportInfo.get + val transMetrics3 = memInfo3.transportInfo.get + assert((20, 10, MemTime(20, 1446336000), MemTime(10, 1446336000)) == ( + transMetrics1.onHeapSize, + transMetrics1.offHeapSize, + transMetrics1.peakOnHeapSizeTime, + transMetrics1.peakOffHeapSizeTime)) + assert((15, 5, MemTime(15, 1446337000), MemTime(5, 1446337000)) == ( + transMetrics2.onHeapSize, + transMetrics2.offHeapSize, + transMetrics2.peakOnHeapSizeTime, + transMetrics2.peakOffHeapSizeTime)) + assert((30, 15, MemTime(30, 1446338000), MemTime(15, 1446338000)) == ( + transMetrics3.onHeapSize, + transMetrics3.offHeapSize, + transMetrics3.peakOnHeapSizeTime, + transMetrics3.peakOffHeapSizeTime)) + } } From a93bd962b2c9e6369a2257d70c1c378c68d77500 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 19 Nov 2015 11:20:11 +0800 Subject: [PATCH 20/28] scala style fix --- .../main/scala/org/apache/spark/executor/Executor.scala | 2 +- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 4 ++-- .../org/apache/spark/ui/memory/MemoryListenerSuite.scala | 8 ++++---- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 21db8317075f6..309d4382d776e 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -453,7 +453,7 @@ private[spark] class Executor( env.blockTransferService.getMemMetrics(this.executorMetrics) val executorMetrics = if (isLocal) { // JobProgressListener might hold a reference of it during onExecutorMetricsUpdate() - // in future, if then JobProgressListener can not see the changes of metrics any + // in future, if then JobProgressListener cannot see the changes of metrics any // more, so make a deep copy of it here for future change. Utils.deserialize[ExecutorMetrics](Utils.serialize(this.executorMetrics)) } else { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 028451aef4da6..afaa577731d99 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -294,7 +294,7 @@ private[spark] object JsonProtocol { } def transportMetricsToJson(transportMetrics: TransportMetrics): JValue = { - ("TimeStamep" -> transportMetrics.timeStamp) ~ + ("TimeStamp" -> transportMetrics.timeStamp) ~ ("OnHeapSize" -> transportMetrics.onHeapSize) ~ ("OffHeapSize" -> transportMetrics.offHeapSize) } @@ -725,7 +725,7 @@ private[spark] object JsonProtocol { def transportMetricsFromJson(json: JValue): TransportMetrics = { val metrics = new TransportMetrics( - (json \ "TimeStamep").extract[Long], + (json \ "TimeStamp").extract[Long], (json \ "OnHeapSize").extract[Long], (json \ "OffHeapSize").extract[Long]) metrics diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index 26c5f813bf984..e225ed4044ea6 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -107,7 +107,7 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get // no metrics for stage 1 since no metrics update for stage 1 assert(mapForStage1.get(execId1).get.transportInfo == None) - val mapForStage2 = listener.completedStagesToMem.get((2,0)).get + val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get assert(mapForStage2.size == 1) val memInfo = mapForStage2.get(execId1).get assert(memInfo.transportInfo.isDefined) @@ -118,7 +118,7 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) } - test("test multiple executors metrics updated in one stage") { + test("test multiple metrics updated in one stage") { val listener = new MemoryListener val execId1 = "exec-1" @@ -216,7 +216,7 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc val exec2Metrics = createExecutorMetrics("host-2", 1446337000L, 15, 5) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( execId2, exec2Metrics)) - // on more executor added when stage is running + // one more executor added when stage is running listener.onExecutorAdded( SparkListenerExecutorAdded(0L, execId3, new ExecutorInfo("host3", 1, Map.empty))) val exec3Metrics = createExecutorMetrics("host-3", 1446338000L, 30, 15) @@ -230,7 +230,7 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId3, "")) // the completedStagesToMem will maintain the metrics of both the removed executors and new - // add executors + // added executors val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get assert(mapForStage1.size == 3) val memInfo1 = mapForStage1.get(execId1).get From 89214f342d37daea4f27d08d08cee145e63f3eab Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 23 Nov 2015 17:23:16 +0800 Subject: [PATCH 21/28] fix test fail and add event logging unit test --- .../spark/executor/ExecutorMetrics.scala | 6 + .../scheduler/EventLoggingListener.scala | 26 ++- .../scheduler/EventLoggingListenerSuite.scala | 99 ++++++++++ .../spark/ui/memory/MemoryListenerSuite.scala | 178 +++++++++--------- .../apache/spark/util/JsonProtocolSuite.scala | 15 +- 5 files changed, 225 insertions(+), 99 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 34710007e5179..d6d4894e5d006 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -41,6 +41,12 @@ class ExecutorMetrics extends Serializable { private[spark] def setTransportMetrics(value: TransportMetrics) = { _transportMetrics = value } + + // for test only + def metricsDetails = { + (hostname, transportMetrics.timeStamp, transportMetrics.onHeapSize, + transportMetrics.offHeapSize) + } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 94da8b0c2e963..cc20bce58d2b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -32,7 +32,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SPARK_VERSION} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.executor.TransportMetrics +import org.apache.spark.executor.{ExecutorMetrics, TransportMetrics} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{JsonProtocol, Utils} @@ -169,10 +169,9 @@ private[spark] class EventLoggingListener( // in {{executorIdToLatestMetrics}}. private def updateAndLogExecutorMemoryMetrics() : Unit = { executorIdToModifiedMaxMetrics.foreach { case(_, metrics) => logEvent(metrics) } + // Clear the modified metrics map after each log action + executorIdToModifiedMaxMetrics.clear() executorIdToLatestMetrics.foreach {case(_, metrics) => logEvent(metrics) } - executorIdToLatestMetrics.foreach { case (executorId, metrics) => - executorIdToModifiedMaxMetrics.update(executorId, metrics) - } } // Events that do not trigger a flush @@ -234,8 +233,12 @@ private[spark] class EventLoggingListener( // No-op because logging every update would be overkill override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - executorIdToLatestMetrics.update(event.execId, event) - updateModifiedMetrics(event.execId) + // In order to avoid the logged event consumes too much storage size, taskMetrics would not + // be logged into event log file currently + val lightEvent = SparkListenerExecutorMetricsUpdate( + event.execId, event.executorMetrics, Seq.empty) + executorIdToLatestMetrics.update(lightEvent.execId, lightEvent) + updateModifiedMetrics(lightEvent.execId) } /** @@ -288,8 +291,15 @@ private[spark] class EventLoggingListener( } else { toBeModTransMetrics.offHeapSize } - toBeModifiedEvent.executorMetrics.setTransportMetrics( - TransportMetrics(timeStamp, onHeapSize, offHeapSize)) + + // We should maintain a new instance for each update to avoid side-effect + val modifiedExecMetrics = new ExecutorMetrics() + modifiedExecMetrics.setHostname(toBeModifiedEvent.executorMetrics.hostname) + modifiedExecMetrics.setTransportMetrics(TransportMetrics( + timeStamp, onHeapSize, offHeapSize)) + val modifiedEvent = SparkListenerExecutorMetricsUpdate( + toBeModifiedEvent.execId, modifiedExecMetrics, toBeModifiedEvent.taskMetrics) + executorIdToModifiedMaxMetrics.update(executorId, modifiedEvent) } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 5cb2d4225d281..a197724941e37 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -122,6 +122,105 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit "a fine:mind$dollar{bills}.1", None, Some("lz4"))) } + test("test event logger logging executor metrics") { + import org.apache.spark.scheduler.cluster._ + import org.apache.spark.ui.memory._ + val conf = EventLoggingListenerSuite.getLoggingConf(testDirPath) + val eventLogger = new EventLoggingListener("test-memListener", None, testDirPath.toUri(), conf) + val execId = "exec-1" + val hostName = "host-1" + + eventLogger.start() + eventLogger.onExecutorAdded(SparkListenerExecutorAdded( + 0L, execId, new ExecutorInfo(hostName, 1, Map.empty))) + + // stage 1 and stage 2 submitted + eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) + eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(hostName, 1L, 20, 10) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics1)) + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(hostName, 2L, 30, 10) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics2)) + // stage1 completed + eventLogger.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) + // stage3 submitted + eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(hostName, 3L, 30, 30) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics3)) + val execMetrics4 = MemoryListenerSuite.createExecutorMetrics(hostName, 4L, 20, 25) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics4)) + // stage 2 completed + eventLogger.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) + val execMetrics5 = MemoryListenerSuite.createExecutorMetrics(hostName, 5L, 15, 15) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics5)) + val execMetrics6 = MemoryListenerSuite.createExecutorMetrics(hostName, 6L, 25, 10) + eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId, execMetrics6)) + // stage 3 completed + eventLogger.onStageCompleted(MemoryListenerSuite.createStageEndEvent(3)) + + eventLogger.onExecutorRemoved(SparkListenerExecutorRemoved(7L, execId, "")) + + // Totally there are 15 logged events, including: + // 2 events of executor Added/Removed + // 6 events of stage Submitted/Completed + // 7 events of executorMetrics update (3 combined metrics and 4 original metrics) + assert(eventLogger.loggedEvents.size === 15) + eventLogger.stop() + + val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) + val lines = readLines(logData) + Utils.tryWithSafeFinally { + // totally there are 15 lines, including SparkListenerLogStart event and 14 other events + assert(lines.size === 16) + + // 4 executor metrics that is the latest metrics updated before stage submit and complete + val jsonMetrics = JsonProtocol.sparkEventFromJson(parse(lines(5))) + assert(Utils.getFormattedClassName(jsonMetrics) === Utils.getFormattedClassName( + SparkListenerExecutorMetricsUpdate)) + val jsonMetrics2 = jsonMetrics.asInstanceOf[SparkListenerExecutorMetricsUpdate] + assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics2.execId, jsonMetrics2 + .executorMetrics.metricsDetails)) + + val jsonMetrics4 = JsonProtocol.sparkEventFromJson(parse(lines(7))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + val jsonMetrics6 = JsonProtocol.sparkEventFromJson(parse(lines(10))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + val jsonMetrics8 = JsonProtocol.sparkEventFromJson(parse(lines(13))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics4.execId, jsonMetrics4 + .executorMetrics.metricsDetails)) + assert((execId, (hostName, 4L, 20, 25)) === (jsonMetrics6.execId, jsonMetrics6 + .executorMetrics.metricsDetails)) + assert((execId, (hostName, 6L, 25, 10)) === (jsonMetrics8.execId, jsonMetrics8 + .executorMetrics.metricsDetails)) + + // 3 executor metrics that is combined metrics that updated during each time segment + // There is no combined metrics before "jsonMetrics4" (lines(7)) because there is no + // metrics update between stage 1 complete and stage 3 submit. So only the last metrics + // update will be logged. + val jsonMetrics1 = JsonProtocol.sparkEventFromJson(parse(lines(4))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + val jsonMetrics5 = JsonProtocol.sparkEventFromJson(parse(lines(9))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + val jsonMetrics7 = JsonProtocol.sparkEventFromJson(parse(lines(12))) + .asInstanceOf[SparkListenerExecutorMetricsUpdate] + assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics1.execId, jsonMetrics1 + .executorMetrics.metricsDetails)) + assert((execId, (hostName, 3L, 30, 30)) === (jsonMetrics5.execId, jsonMetrics5 + .executorMetrics.metricsDetails)) + assert((execId, (hostName, 6L, 25, 15)) === (jsonMetrics7.execId, jsonMetrics7 + .executorMetrics.metricsDetails)) + } { + logData.close() + } + } + /* ----------------- * * Actual test logic * * ----------------- */ diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index e225ed4044ea6..2441433d17240 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -17,72 +17,40 @@ package org.apache.spark.ui.memory -import org.scalatest.Matchers - import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ -class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matchers { - private def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") - SparkListenerStageSubmitted(stageInfo) - } - - private def createStageEndEvent(stageId: Int, failed: Boolean = false) = { - val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, null, "") - if (failed) { - stageInfo.failureReason = Some("Failed!") - } - SparkListenerStageCompleted(stageInfo) - } - - private def createExecutorMetricsUpdateEvent( - execId: String, - executorMetrics: ExecutorMetrics) = { - SparkListenerExecutorMetricsUpdate(execId, executorMetrics, null) - } - - private def createExecutorMetrics( - hostname: String, - timeStamp: Long, - onHeapSize: Long, - offHeapSize: Long): ExecutorMetrics = { - val execMetrics = new ExecutorMetrics - execMetrics.setHostname(hostname) - execMetrics.setTransportMetrics(TransportMetrics(timeStamp, onHeapSize, offHeapSize)) - execMetrics - } - +class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext { test("test HashMap size for MemoryListener") { val listener = new MemoryListener val execId1 = "exec-1" val execId2 = "exec-2" (1 to 2).foreach { i => - listener.onStageSubmitted(createStageStartEvent(i)) - listener.onStageCompleted(createStageEndEvent(i)) + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(i)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(i)) } // stages are all completed, no activeStages now assert(listener.activeStagesToMem.isEmpty) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, new ExecutorMetrics)) // ExecutorMetrics is not related with Stages directly assert(listener.activeStagesToMem.isEmpty) - listener.onStageSubmitted(createStageStartEvent(3)) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId2, new ExecutorMetrics)) // totally 2 executors updated their metrics - assert(listener.activeExecutorIdToMem.size == 2) - assert(listener.activeStagesToMem.size == 1) - listener.onStageCompleted(createStageEndEvent(3)) + assert(listener.activeExecutorIdToMem.size === 2) + assert(listener.activeStagesToMem.size === 1) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(3)) assert(listener.activeStagesToMem.isEmpty) - assert(listener.completedStagesToMem.size == 3) - assert(listener.activeExecutorIdToMem.size == listener.latestExecIdToExecMetrics.size) + assert(listener.completedStagesToMem.size === 3) + assert(listener.activeExecutorIdToMem.size === listener.latestExecIdToExecMetrics.size) assert(listener.removedExecutorIdToMem.isEmpty) } @@ -94,25 +62,25 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) // stage 1, no metrics update - listener.onStageSubmitted(createStageStartEvent(1)) - listener.onStageCompleted(createStageEndEvent(1)) + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) // stage 2, with one metrics update - listener.onStageSubmitted(createStageStartEvent(2)) - val execMetrics = createExecutorMetrics("host-1", 0L, 20, 10) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) + val execMetrics = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics)) - listener.onStageCompleted(createStageEndEvent(2)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get // no metrics for stage 1 since no metrics update for stage 1 - assert(mapForStage1.get(execId1).get.transportInfo == None) + assert(mapForStage1.get(execId1).get.transportInfo === None) val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get - assert(mapForStage2.size == 1) + assert(mapForStage2.size === 1) val memInfo = mapForStage2.get(execId1).get assert(memInfo.transportInfo.isDefined) val transMetrics = memInfo.transportInfo.get - assert((20, 10, MemTime(20, 0), MemTime(10, 0)) == (transMetrics.onHeapSize, + assert((20, 10, MemTime(20, 0), MemTime(10, 0)) === (transMetrics.onHeapSize, transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) @@ -126,23 +94,23 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) // multiple metrics updated in one stage - listener.onStageSubmitted(createStageStartEvent(1)) - val execMetrics1 = createExecutorMetrics("host-1", 0L, 20, 10) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics1)) - val execMetrics2 = createExecutorMetrics("host-1", 0L, 30, 5) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 30, 5) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics2)) - val execMetrics3 = createExecutorMetrics("host-1", 0L, 15, 15) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 15, 15) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics3)) - listener.onStageCompleted(createStageEndEvent(1)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get val memInfo = mapForStage1.get(execId1).get assert(memInfo.transportInfo.isDefined) val transMetrics = memInfo.transportInfo.get - assert((15, 15, MemTime(30, 0), MemTime(15, 0)) == (transMetrics.onHeapSize, + assert((15, 15, MemTime(30, 0), MemTime(15, 0)) === (transMetrics.onHeapSize, transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) @@ -156,30 +124,30 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) // multiple metrics updated in one stage - listener.onStageSubmitted(createStageStartEvent(1)) - val execMetrics1 = createExecutorMetrics("host-1", 0L, 20, 10) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics1)) - val execMetrics2 = createExecutorMetrics("host-1", 0L, 30, 5) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 30, 5) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics2)) - val execMetrics3 = createExecutorMetrics("host-1", 0L, 15, 15) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 15, 15) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics3)) - listener.onStageCompleted(createStageEndEvent(1)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) // stage 2 and stage 3 don't get metrics - listener.onStageSubmitted(createStageStartEvent(2)) - listener.onStageCompleted(createStageEndEvent(2)) - listener.onStageSubmitted(createStageStartEvent(3)) - listener.onStageCompleted(createStageEndEvent(3)) + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(3)) // both stage 2 and stage 3 will use the metrics last updated in stage 1 val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get val memInfo2 = mapForStage2.get(execId1).get assert(memInfo2.transportInfo.isDefined) val transMetrics2 = memInfo2.transportInfo.get - assert((15, 15, MemTime(15, 0), MemTime(15, 0)) == (transMetrics2.onHeapSize, + assert((15, 15, MemTime(15, 0), MemTime(15, 0)) === (transMetrics2.onHeapSize, transMetrics2.offHeapSize, transMetrics2.peakOnHeapSizeTime, transMetrics2.peakOffHeapSizeTime)) @@ -188,7 +156,7 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc val memInfo3 = mapForStage3.get(execId1).get assert(memInfo3.transportInfo.isDefined) val transMetrics3 = memInfo3.transportInfo.get - assert((15, 15, MemTime(15, 0), MemTime(15, 0)) == (transMetrics3.onHeapSize, + assert((15, 15, MemTime(15, 0), MemTime(15, 0)) === (transMetrics3.onHeapSize, transMetrics3.offHeapSize, transMetrics3.peakOnHeapSizeTime, transMetrics3.peakOffHeapSizeTime)) @@ -208,23 +176,23 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc listener.onExecutorAdded( SparkListenerExecutorAdded(0L, execId2, new ExecutorInfo("host2", 1, Map.empty))) - // two executors running in one stage and one executor is removed before stage complete - listener.onStageSubmitted(createStageStartEvent(1)) - val exec1Metrics = createExecutorMetrics("host-1", 1446336000L, 20, 10) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + // three executors running in one stage and one executor is removed before stage complete + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) + val exec1Metrics = MemoryListenerSuite.createExecutorMetrics("host-1", 1446336000L, 20, 10) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, exec1Metrics)) - val exec2Metrics = createExecutorMetrics("host-2", 1446337000L, 15, 5) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val exec2Metrics = MemoryListenerSuite.createExecutorMetrics("host-2", 1446337000L, 15, 5) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId2, exec2Metrics)) - // one more executor added when stage is running + // one more executor added during the stage is running listener.onExecutorAdded( SparkListenerExecutorAdded(0L, execId3, new ExecutorInfo("host3", 1, Map.empty))) - val exec3Metrics = createExecutorMetrics("host-3", 1446338000L, 30, 15) - listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent( + val exec3Metrics = MemoryListenerSuite.createExecutorMetrics("host-3", 1446338000L, 30, 15) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId3, exec3Metrics)) // executor 2 removed before stage complete listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId2, "")) - listener.onStageCompleted(createStageEndEvent(1)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId3, "")) @@ -232,27 +200,59 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext with Matc // the completedStagesToMem will maintain the metrics of both the removed executors and new // added executors val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get - assert(mapForStage1.size == 3) + assert(mapForStage1.size === 3) val memInfo1 = mapForStage1.get(execId1).get val memInfo2 = mapForStage1.get(execId2).get val memInfo3 = mapForStage1.get(execId3).get val transMetrics1 = memInfo1.transportInfo.get val transMetrics2 = memInfo2.transportInfo.get val transMetrics3 = memInfo3.transportInfo.get - assert((20, 10, MemTime(20, 1446336000), MemTime(10, 1446336000)) == ( + assert((20, 10, MemTime(20, 1446336000), MemTime(10, 1446336000)) === ( transMetrics1.onHeapSize, transMetrics1.offHeapSize, transMetrics1.peakOnHeapSizeTime, transMetrics1.peakOffHeapSizeTime)) - assert((15, 5, MemTime(15, 1446337000), MemTime(5, 1446337000)) == ( + assert((15, 5, MemTime(15, 1446337000), MemTime(5, 1446337000)) === ( transMetrics2.onHeapSize, transMetrics2.offHeapSize, transMetrics2.peakOnHeapSizeTime, transMetrics2.peakOffHeapSizeTime)) - assert((30, 15, MemTime(30, 1446338000), MemTime(15, 1446338000)) == ( + assert((30, 15, MemTime(30, 1446338000), MemTime(15, 1446338000)) === ( transMetrics3.onHeapSize, transMetrics3.offHeapSize, transMetrics3.peakOnHeapSizeTime, transMetrics3.peakOffHeapSizeTime)) } } + +object MemoryListenerSuite { + def createStageStartEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") + SparkListenerStageSubmitted(stageInfo) + } + + def createStageEndEvent(stageId: Int, failed: Boolean = false) = { + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") + if (failed) { + stageInfo.failureReason = Some("Failed!") + } + SparkListenerStageCompleted(stageInfo) + } + + def createExecutorMetricsUpdateEvent( + execId: String, + executorMetrics: ExecutorMetrics) = { + SparkListenerExecutorMetricsUpdate(execId, executorMetrics, Seq.empty) + } + + def createExecutorMetrics( + hostname: String, + timeStamp: Long, + onHeapSize: Long, + offHeapSize: Long): ExecutorMetrics = { + val execMetrics = new ExecutorMetrics + execMetrics.setHostname(hostname) + execMetrics.setTransportMetrics(TransportMetrics(timeStamp, onHeapSize, offHeapSize)) + execMetrics + } +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0041d139c6b2f..16bfd211c24d3 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -83,7 +83,13 @@ class JsonProtocolSuite extends SparkFunSuite { val executorAdded = SparkListenerExecutorAdded(executorAddedTime, "exec1", new ExecutorInfo("Hostee.awesome.com", 11, logUrlMap)) val executorRemoved = SparkListenerExecutorRemoved(executorRemovedTime, "exec2", "test reason") - val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", new ExecutorMetrics, + val executorMetrics = { + val execMetrics = new ExecutorMetrics + execMetrics.setHostname("host-1") + execMetrics.setTransportMetrics(TransportMetrics(0L, 10, 10)) + execMetrics + } + val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", executorMetrics, Seq((1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)))) @@ -1676,7 +1682,12 @@ class JsonProtocolSuite extends SparkFunSuite { | "Event": "SparkListenerExecutorMetricsUpdate", | "Executor ID": "exec3", | "Executor Metrics Updated": { - | "Executor Hostname": null + | "Executor Hostname": "host-1", + | "TransportMetrics": { + | "TimeStamp": 0, + | "OnHeapSize": 10, + | "OffHeapSize": 10 + | } | }, | "Metrics Updated": [ | { From 1ed48c11df78e07a5f6d71de5a3d3b7fecf245b9 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 23 Nov 2015 17:32:33 +0800 Subject: [PATCH 22/28] scala syle --- .../scala/org/apache/spark/executor/ExecutorMetrics.scala | 2 +- .../org/apache/spark/ui/memory/MemoryListenerSuite.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index d6d4894e5d006..5df3f94ff7124 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -43,7 +43,7 @@ class ExecutorMetrics extends Serializable { } // for test only - def metricsDetails = { + def metricsDetails: (String, Long, Long, Long) = { (hostname, transportMetrics.timeStamp, transportMetrics.onHeapSize, transportMetrics.offHeapSize) } diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index 2441433d17240..1089705a1d9b6 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -226,12 +226,12 @@ class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext { } object MemoryListenerSuite { - def createStageStartEvent(stageId: Int) = { + def createStageStartEvent(stageId: Int): SparkListenerStageSubmitted = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") SparkListenerStageSubmitted(stageInfo) } - def createStageEndEvent(stageId: Int, failed: Boolean = false) = { + def createStageEndEvent(stageId: Int, failed: Boolean = false): SparkListenerStageCompleted = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") if (failed) { stageInfo.failureReason = Some("Failed!") @@ -241,7 +241,7 @@ object MemoryListenerSuite { def createExecutorMetricsUpdateEvent( execId: String, - executorMetrics: ExecutorMetrics) = { + executorMetrics: ExecutorMetrics): SparkListenerExecutorMetricsUpdate = { SparkListenerExecutorMetricsUpdate(execId, executorMetrics, Seq.empty) } @@ -255,4 +255,4 @@ object MemoryListenerSuite { execMetrics.setTransportMetrics(TransportMetrics(timeStamp, onHeapSize, offHeapSize)) execMetrics } -} \ No newline at end of file +} From b43807731228c4e5c672436a197e94341bf4ffd3 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 24 Nov 2015 09:25:42 +0800 Subject: [PATCH 23/28] roll back useless change --- .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index cc20bce58d2b0..5daec11e23034 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -92,8 +92,7 @@ private[spark] class EventLoggingListener( private[scheduler] val loggedEvents = new ArrayBuffer[JValue] // Visible for tests only. - private[scheduler] val logPath = getLogPath( - logBaseDir, appId, appAttemptId, compressionCodecName) + private[scheduler] val logPath = getLogPath(logBaseDir, appId, appAttemptId, compressionCodecName) private val executorIdToLatestMetrics = new HashMap[String, SparkListenerExecutorMetricsUpdate] private val executorIdToModifiedMaxMetrics = new From 4123ac7924275c534dd7849c108f71eebdf4b4a1 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 8 Dec 2015 11:41:27 +0800 Subject: [PATCH 24/28] modify the code according to Imran's comments, mainly with unit test --- .../org/apache/spark/executor/Executor.scala | 1 + .../spark/executor/ExecutorMetrics.scala | 28 +- .../spark/network/BlockTransferService.scala | 2 +- .../netty/NettyBlockTransferService.scala | 8 +- .../scheduler/EventLoggingListener.scala | 34 ++- .../apache/spark/ui/memory/MemoryTab.scala | 5 +- .../org/apache/spark/util/JsonProtocol.scala | 5 + .../scheduler/EventLoggingListenerSuite.scala | 141 ++++++---- .../spark/ui/memory/MemoryListenerSuite.scala | 265 +++++++----------- .../apache/spark/util/JsonProtocolSuite.scala | 19 ++ 10 files changed, 265 insertions(+), 243 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 309d4382d776e..0937e440da1de 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -87,6 +87,7 @@ private[spark] class Executor( private val executorMetrics: ExecutorMetrics = new ExecutorMetrics executorMetrics.setHostname(Utils.localHostName) + executorMetrics.setPort(env.rpcEnv.address.port) // Create an RpcEndpoint for receiving RPCs from the driver private val executorEndpoint = env.rpcEnv.setupEndpoint( diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 5df3f94ff7124..a7afcf5169009 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -32,10 +32,19 @@ class ExecutorMetrics extends Serializable { /** * Host's name the executor runs on */ - private var _hostname: String = _ + private var _hostname: String = "" def hostname: String = _hostname private[spark] def setHostname(value: String) = _hostname = value + /** + * Host's port the executor runs on + */ + private var _port: Int = _ + def port: Int = _port + private[spark] def setPort(value: Int) = _port = value + + private[spark] def hostPort: String = hostname + ":" + port + private var _transportMetrics: TransportMetrics = new TransportMetrics def transportMetrics: TransportMetrics = _transportMetrics private[spark] def setTransportMetrics(value: TransportMetrics) = { @@ -49,6 +58,19 @@ class ExecutorMetrics extends Serializable { } } +object ExecutorMetrics extends Serializable { + def apply( + hostName: String, + port: Int, + transportMetrics: TransportMetrics): ExecutorMetrics = { + val execMetrics = new ExecutorMetrics + execMetrics.setHostname(hostName) + execMetrics.setPort(port) + execMetrics.setTransportMetrics(transportMetrics) + execMetrics + } +} + /** * :: DeveloperApi :: * Metrics for network layer @@ -57,9 +79,9 @@ class ExecutorMetrics extends Serializable { class TransportMetrics ( val timeStamp: Long = System.currentTimeMillis, val onHeapSize: Long = 0L, - val offHeapSize: Long = 0L) + val offHeapSize: Long = 0L) extends Serializable -object TransportMetrics { +object TransportMetrics extends Serializable { def apply( timeStamp: Long, onHeapSize: Long, diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 03b0fca4bcd74..7524915f74f3b 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -41,7 +41,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo /** * Collect current executor memory metrics of transferService. */ - def getMemMetrics(executorMetrics: ExecutorMetrics): Unit + private[spark] def getMemMetrics(executorMetrics: ExecutorMetrics): Unit /** * Tear down the transfer service. diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index b73335546115a..37a3fbd671685 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -59,7 +59,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage clock = newClock } - override def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = { + private[spark] override def getMemMetrics(executorMetrics: ExecutorMetrics): Unit = { val currentTime = clock.getTimeMillis() val clientPooledAllocator = clientFactory.getPooledAllocator() val serverAllocator = server.getAllocator() @@ -68,9 +68,9 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas().asScala.toList) val serverOffHeapSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas().asScala.toList) - logDebug(s"Current Netty Client offHeapSize is $clientOffHeapSize, " + - s"Client HeapSize is $clientOnHeapSize, server directHeapsize is $serverOffHeapSize, " + - s"server heapsize is $serverOnHeapSize, executer id is " + + logDebug(s"Current Netty Client offheap size is $clientOffHeapSize, " + + s"Client heap size is $clientOnHeapSize, Server offheap size is $serverOffHeapSize, " + + s"server heap size is $serverOnHeapSize, executor id is " + s"${SparkEnv.get.blockManager.blockManagerId.executorId}") executorMetrics.setTransportMetrics(TransportMetrics(currentTime, clientOnHeapSize + serverOnHeapSize, clientOffHeapSize + serverOffHeapSize)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 5daec11e23034..558b8a8271f9a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -230,14 +230,14 @@ private[spark] class EventLoggingListener( // No-op because logging every update would be overkill override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = {} - // No-op because logging every update would be overkill + // Track executor metrics for logging on stage start and end override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { - // In order to avoid the logged event consumes too much storage size, taskMetrics would not - // be logged into event log file currently - val lightEvent = SparkListenerExecutorMetricsUpdate( + // We only track the executor metrics in each stage, so we drop the task metrics as they are + // quite verbose + val eventWithoutTaskMetrics = SparkListenerExecutorMetricsUpdate( event.execId, event.executorMetrics, Seq.empty) - executorIdToLatestMetrics.update(lightEvent.execId, lightEvent) - updateModifiedMetrics(lightEvent.execId) + executorIdToLatestMetrics(eventWithoutTaskMetrics.execId) = eventWithoutTaskMetrics + updateModifiedMetrics(eventWithoutTaskMetrics) } /** @@ -263,18 +263,17 @@ private[spark] class EventLoggingListener( /** * According to the updated event to modify the maintained event's metrics - * @param executorId the executor whose metrics will be modified + * @param latestEvent the latest event received that used to update the maintained metric */ - private def updateModifiedMetrics(executorId: String): Unit = { + private def updateModifiedMetrics(latestEvent: SparkListenerExecutorMetricsUpdate): Unit = { + val executorId = latestEvent.execId val toBeModifiedEvent = executorIdToModifiedMaxMetrics.get(executorId) - val latestEvent = executorIdToLatestMetrics.get(executorId) toBeModifiedEvent match { - case None => if (latestEvent.isDefined) executorIdToModifiedMaxMetrics.update( - executorId, latestEvent.get) + case None => + executorIdToModifiedMaxMetrics(executorId) = latestEvent case Some(toBeModifiedEvent) => val toBeModifiedMetrics = toBeModifiedEvent.executorMetrics.transportMetrics - // latestEvent must has value - val latestTransMetrics = latestEvent.get.executorMetrics.transportMetrics + val latestTransMetrics = latestEvent.executorMetrics.transportMetrics val toBeModTransMetrics = toBeModifiedMetrics var timeStamp: Long = toBeModTransMetrics.timeStamp // the logic here should be the same with that for memoryListener @@ -292,13 +291,12 @@ private[spark] class EventLoggingListener( } // We should maintain a new instance for each update to avoid side-effect - val modifiedExecMetrics = new ExecutorMetrics() - modifiedExecMetrics.setHostname(toBeModifiedEvent.executorMetrics.hostname) - modifiedExecMetrics.setTransportMetrics(TransportMetrics( - timeStamp, onHeapSize, offHeapSize)) + val modifiedExecMetrics = ExecutorMetrics(toBeModifiedEvent.executorMetrics.hostname, + toBeModifiedEvent.executorMetrics.port, + TransportMetrics(timeStamp, onHeapSize, offHeapSize)) val modifiedEvent = SparkListenerExecutorMetricsUpdate( toBeModifiedEvent.execId, modifiedExecMetrics, toBeModifiedEvent.taskMetrics) - executorIdToModifiedMaxMetrics.update(executorId, modifiedEvent) + executorIdToModifiedMaxMetrics(executorId) = modifiedEvent } } } diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index e5095510c8d93..91fa2770f7145 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -96,6 +96,7 @@ class MemoryListener extends SparkListener { } } +@DeveloperApi class MemoryUIInfo { var executorAddress: String = _ var transportInfo: Option[TransportMemSize] = None @@ -110,11 +111,12 @@ class MemoryUIInfo { case Some(transportMemSize) => transportInfo case _ => Some(new TransportMemSize) } - executorAddress = execMetrics.hostname + executorAddress = execMetrics.hostPort transportInfo.get.updateTransport(execMetrics.transportMetrics) } } +@DeveloperApi class TransportMemSize { var onHeapSize: Long = _ var offHeapSize: Long = _ @@ -136,4 +138,5 @@ class TransportMemSize { } } +@DeveloperApi case class MemTime(memorySize: Long = 0L, timeStamp: Long = System.currentTimeMillis) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 9d397ac8a2030..7d910872fe825 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -290,6 +290,7 @@ private[spark] object JsonProtocol { def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { val transportMetrics = transportMetricsToJson(executorMetrics.transportMetrics) ("Executor Hostname" -> executorMetrics.hostname) ~ + ("Executor Port" -> executorMetrics.port) ~ ("TransportMetrics" -> transportMetrics) } @@ -724,7 +725,11 @@ private[spark] object JsonProtocol { def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = new ExecutorMetrics + if (json == JNothing) { + return metrics + } metrics.setHostname((json \ "Executor Hostname").extract[String]) + metrics.setPort((json \ "Executor Port").extract[Int]) metrics.setTransportMetrics(transportMetricsFromJson(json \ "TransportMetrics")) metrics } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index a197724941e37..cb1187ede91e8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -84,15 +84,15 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } - test("End-to-end event logging") { - testApplicationEventLogging() - } - - test("End-to-end event logging with compression") { - CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => - testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) - } - } +// test("End-to-end event logging") { +// testApplicationEventLogging() +// } +// +// test("End-to-end event logging with compression") { +// CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => +// testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) +// } +// } test("Log overwriting") { val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) @@ -129,6 +129,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener("test-memListener", None, testDirPath.toUri(), conf) val execId = "exec-1" val hostName = "host-1" + val port = 80 eventLogger.start() eventLogger.onExecutorAdded(SparkListenerExecutorAdded( @@ -137,28 +138,28 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // stage 1 and stage 2 submitted eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) - val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(hostName, 1L, 20, 10) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 1L, 20, 10) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics1)) - val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(hostName, 2L, 30, 10) + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 2L, 30, 10) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics2)) // stage1 completed eventLogger.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) // stage3 submitted eventLogger.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) - val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(hostName, 3L, 30, 30) + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 3L, 30, 30) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics3)) - val execMetrics4 = MemoryListenerSuite.createExecutorMetrics(hostName, 4L, 20, 25) + val execMetrics4 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 4L, 20, 25) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics4)) // stage 2 completed eventLogger.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) - val execMetrics5 = MemoryListenerSuite.createExecutorMetrics(hostName, 5L, 15, 15) + val execMetrics5 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 5L, 15, 15) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics5)) - val execMetrics6 = MemoryListenerSuite.createExecutorMetrics(hostName, 6L, 25, 10) + val execMetrics6 = MemoryListenerSuite.createExecutorMetrics(hostName, port, 6L, 25, 10) eventLogger.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId, execMetrics6)) // stage 3 completed @@ -176,49 +177,79 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) val lines = readLines(logData) Utils.tryWithSafeFinally { - // totally there are 15 lines, including SparkListenerLogStart event and 14 other events + // totally there are 16 lines, including SparkListenerLogStart event and 15 other events assert(lines.size === 16) - // 4 executor metrics that is the latest metrics updated before stage submit and complete - val jsonMetrics = JsonProtocol.sparkEventFromJson(parse(lines(5))) - assert(Utils.getFormattedClassName(jsonMetrics) === Utils.getFormattedClassName( - SparkListenerExecutorMetricsUpdate)) - val jsonMetrics2 = jsonMetrics.asInstanceOf[SparkListenerExecutorMetricsUpdate] - assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics2.execId, jsonMetrics2 - .executorMetrics.metricsDetails)) - - val jsonMetrics4 = JsonProtocol.sparkEventFromJson(parse(lines(7))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - val jsonMetrics6 = JsonProtocol.sparkEventFromJson(parse(lines(10))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - val jsonMetrics8 = JsonProtocol.sparkEventFromJson(parse(lines(13))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics4.execId, jsonMetrics4 - .executorMetrics.metricsDetails)) - assert((execId, (hostName, 4L, 20, 25)) === (jsonMetrics6.execId, jsonMetrics6 - .executorMetrics.metricsDetails)) - assert((execId, (hostName, 6L, 25, 10)) === (jsonMetrics8.execId, jsonMetrics8 - .executorMetrics.metricsDetails)) - - // 3 executor metrics that is combined metrics that updated during each time segment - // There is no combined metrics before "jsonMetrics4" (lines(7)) because there is no - // metrics update between stage 1 complete and stage 3 submit. So only the last metrics - // update will be logged. - val jsonMetrics1 = JsonProtocol.sparkEventFromJson(parse(lines(4))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - val jsonMetrics5 = JsonProtocol.sparkEventFromJson(parse(lines(9))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - val jsonMetrics7 = JsonProtocol.sparkEventFromJson(parse(lines(12))) - .asInstanceOf[SparkListenerExecutorMetricsUpdate] - assert((execId, (hostName, 2L, 30, 10)) === (jsonMetrics1.execId, jsonMetrics1 - .executorMetrics.metricsDetails)) - assert((execId, (hostName, 3L, 30, 30)) === (jsonMetrics5.execId, jsonMetrics5 - .executorMetrics.metricsDetails)) - assert((execId, (hostName, 6L, 25, 15)) === (jsonMetrics7.execId, jsonMetrics7 - .executorMetrics.metricsDetails)) + val listenerBus = new LiveListenerBus + val memoryListener = new MemoryListener + listenerBus.addListener(memoryListener) + + val sparkEvents: Seq[SparkListenerEvent] = lines.map { line => + val event = JsonProtocol.sparkEventFromJson(parse(line)) + listenerBus.postToAll(event) + event + } + + // Make sure there always an original {{SparkListenerExecutorMetricsUpdate}} event updated + // before each stage complete. + val latestMetricsStage1 = getLatestExecutorMetricsBeforeStageEnd(sparkEvents, 1).get + val latestMetricsStage2 = getLatestExecutorMetricsBeforeStageEnd(sparkEvents, 2).get + val latestMetricsStage3 = getLatestExecutorMetricsBeforeStageEnd(sparkEvents, 3).get + assertMetrics(execId, (hostName, 2L, 30, 10), latestMetricsStage1) + assertMetrics(execId, (hostName, 4L, 20, 25), latestMetricsStage2) + assertMetrics(execId, (hostName, 6L, 25, 10), latestMetricsStage3) + + // Following is an integration test with [[org.apache.spark.ui.memory.MemoryListener]], make + // sure the events logged in history file can work correctly. + System.out.println(memoryListener) + val mapForStage1 = memoryListener.completedStagesToMem((1, 0)) + val transMetrics1 = mapForStage1(execId).transportInfo.get + MemoryListenerSuite.assertTransMetrics( + 30, 10, MemTime(30, 2L), MemTime(10, 2L), transMetrics1) + val mapForStage2 = memoryListener.completedStagesToMem((2, 0)) + val transMetrics2 = mapForStage2(execId).transportInfo.get + MemoryListenerSuite.assertTransMetrics( + 20, 25, MemTime(30, 3L), MemTime(30, 3L), transMetrics2) + val mapForStage3 = memoryListener.completedStagesToMem((3, 0)) + val transMetrics3 = mapForStage3(execId).transportInfo.get + MemoryListenerSuite.assertTransMetrics( + 25, 10, MemTime(30, 3L), MemTime(30, 3L), transMetrics3) } { logData.close() } + + def getLatestExecutorMetricsBeforeStageEnd( + events: Seq[SparkListenerEvent], + stageId: Int): Option[SparkListenerExecutorMetricsUpdate] = { + val itr = events.iterator + var latestMetrics: Option[SparkListenerExecutorMetricsUpdate] = None + var isStageSubmitted: Boolean = false + while(itr.hasNext) { + val event = itr.next() + event match { + case ss: SparkListenerStageSubmitted if ss.stageInfo.stageId == stageId => + isStageSubmitted = true + case sc: SparkListenerStageCompleted if sc.stageInfo.stageId == stageId => + return latestMetrics + case emu: SparkListenerExecutorMetricsUpdate if isStageSubmitted => + latestMetrics = Some(emu) + case _ => // Do nothing for other events + } + } + latestMetrics + } + + def assertMetrics( + execId: String, + metricsDetails: (String, Long, Long, Long), + event: SparkListenerExecutorMetricsUpdate): Unit = { + val eventDetails = event.executorMetrics.metricsDetails + assert(execId === event.execId) + assert(metricsDetails._1 === eventDetails._1) + assert(metricsDetails._2 === eventDetails._2) + assert(metricsDetails._3 === eventDetails._3) + assert(metricsDetails._4 === eventDetails._4) + } } /* ----------------- * @@ -255,7 +286,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Verify file contains exactly the two events logged val logData = EventLoggingListener.openEventLog(new Path(eventLogger.logPath), fileSystem) - try { + Utils.tryWithSafeFinally { val lines = readLines(logData) val logStart = SparkListenerLogStart(SPARK_VERSION) assert(lines.size === 3) @@ -265,7 +296,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(JsonProtocol.sparkEventFromJson(parse(lines(0))) === logStart) assert(JsonProtocol.sparkEventFromJson(parse(lines(1))) === applicationStart) assert(JsonProtocol.sparkEventFromJson(parse(lines(2))) === applicationEnd) - } finally { + } { logData.close() } } diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index 1089705a1d9b6..9b4f6ff631e9f 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -22,220 +22,153 @@ import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ -class MemoryListenerSuite extends SparkFunSuite with LocalSparkContext { - test("test HashMap size for MemoryListener") { - val listener = new MemoryListener - val execId1 = "exec-1" - val execId2 = "exec-2" - - (1 to 2).foreach { i => - listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(i)) - listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(i)) - } - // stages are all completed, no activeStages now - assert(listener.activeStagesToMem.isEmpty) - - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId1, new ExecutorMetrics)) - // ExecutorMetrics is not related with Stages directly - assert(listener.activeStagesToMem.isEmpty) +class MemoryListenerSuite extends SparkFunSuite { - listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId2, new ExecutorMetrics)) - // totally 2 executors updated their metrics - assert(listener.activeExecutorIdToMem.size === 2) - assert(listener.activeStagesToMem.size === 1) - listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(3)) - - assert(listener.activeStagesToMem.isEmpty) - assert(listener.completedStagesToMem.size === 3) - assert(listener.activeExecutorIdToMem.size === listener.latestExecIdToExecMetrics.size) - assert(listener.removedExecutorIdToMem.isEmpty) - } - - test("test first stage with no executor metrics update") { + test("test stages use executor metrics updated in previous stages") { val listener = new MemoryListener val execId1 = "exec-1" + val host1 = "host-1" listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + SparkListenerExecutorAdded(1L, execId1, new ExecutorInfo(host1, 1, Map.empty))) // stage 1, no metrics update listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) - // stage 2, with one metrics update + // multiple metrics updated in stage 2 listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) - val execMetrics = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId1, execMetrics)) - listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) - - val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get - // no metrics for stage 1 since no metrics update for stage 1 - assert(mapForStage1.get(execId1).get.transportInfo === None) - val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get - assert(mapForStage2.size === 1) - val memInfo = mapForStage2.get(execId1).get - assert(memInfo.transportInfo.isDefined) - val transMetrics = memInfo.transportInfo.get - assert((20, 10, MemTime(20, 0), MemTime(10, 0)) === (transMetrics.onHeapSize, - transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) - - listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) - } - - test("test multiple metrics updated in one stage") { - val listener = new MemoryListener - val execId1 = "exec-1" - - listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) - - // multiple metrics updated in one stage - listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) - val execMetrics1 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId1, execMetrics1)) - val execMetrics2 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 30, 5) - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId1, execMetrics2)) - val execMetrics3 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 15, 15) - listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( - execId1, execMetrics3)) - listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) - - val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get - val memInfo = mapForStage1.get(execId1).get - assert(memInfo.transportInfo.isDefined) - val transMetrics = memInfo.transportInfo.get - assert((15, 15, MemTime(30, 0), MemTime(15, 0)) === (transMetrics.onHeapSize, - transMetrics.offHeapSize, transMetrics.peakOnHeapSizeTime, transMetrics.peakOffHeapSizeTime)) - - listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) - } - - test("test stages use executor metrics updated in previous stages") { - val listener = new MemoryListener - val execId1 = "exec-1" - - listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) - - // multiple metrics updated in one stage - listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) - val execMetrics1 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 20, 10) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 2L, 20, 10) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics1)) - val execMetrics2 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 30, 5) + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 3L, 30, 5) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics2)) - val execMetrics3 = MemoryListenerSuite.createExecutorMetrics("host-1", 0L, 15, 15) + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 4L, 15, 15) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics3)) - listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) - - // stage 2 and stage 3 don't get metrics - listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) + + // stage 3 and stage 4 don't get metrics listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(3)) listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(3)) + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(4)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(4)) + + // no metrics for stage 1 since no metrics updated for stage 1 + val mapForStage1 = listener.completedStagesToMem((1, 0)) + assert(mapForStage1.get(execId1).get.transportInfo === None) + + // metrics is with aggregated value for stage 2 when there are more than one metrics updated + val mapForStage2 = listener.completedStagesToMem((2, 0)) + val transMetrics2 = mapForStage2(execId1).transportInfo.get + MemoryListenerSuite.assertTransMetrics( + 15, 15, MemTime(30, 3), MemTime(15, 4), transMetrics2) - // both stage 2 and stage 3 will use the metrics last updated in stage 1 - val mapForStage2 = listener.completedStagesToMem.get((2, 0)).get - val memInfo2 = mapForStage2.get(execId1).get - assert(memInfo2.transportInfo.isDefined) - val transMetrics2 = memInfo2.transportInfo.get - assert((15, 15, MemTime(15, 0), MemTime(15, 0)) === (transMetrics2.onHeapSize, - transMetrics2.offHeapSize, - transMetrics2.peakOnHeapSizeTime, - transMetrics2.peakOffHeapSizeTime)) - - val mapForStage3 = listener.completedStagesToMem.get((3, 0)).get - val memInfo3 = mapForStage3.get(execId1).get + // both stage 3 and stage 4 will use the metrics last updated in stage 2 + val mapForStage3 = listener.completedStagesToMem((3, 0)) + val memInfo3 = mapForStage3(execId1) assert(memInfo3.transportInfo.isDefined) val transMetrics3 = memInfo3.transportInfo.get - assert((15, 15, MemTime(15, 0), MemTime(15, 0)) === (transMetrics3.onHeapSize, - transMetrics3.offHeapSize, - transMetrics3.peakOnHeapSizeTime, - transMetrics3.peakOffHeapSizeTime)) + MemoryListenerSuite.assertTransMetrics( + 15, 15, MemTime(15, 4), MemTime(15, 4), transMetrics3) + + val mapForStage4 = listener.completedStagesToMem((4, 0)) + val memInfo4 = mapForStage4(execId1) + assert(memInfo4.transportInfo.isDefined) + val transMetrics4 = memInfo4.transportInfo.get + MemoryListenerSuite.assertTransMetrics( + 15, 15, MemTime(15, 4), MemTime(15, 4), transMetrics4) listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) } - test("test multiple executors") { + test("test multiple executors with multiple stages") { val listener = new MemoryListener - val execId1 = "exec-1" - val execId2 = "exec-2" - val execId3 = "exec-3" + val (execId1, execId2, execId3) = ("exec-1", "exec-2", "exec-3") + val (host1, host2, host3) = ("host-1", "host-2", "host-3") + val (port1, port2, port3) = (80, 80, 80) // two executors added first listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId1, new ExecutorInfo("host1", 1, Map.empty))) + SparkListenerExecutorAdded(1L, execId1, new ExecutorInfo(host1, 1, Map.empty))) listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId2, new ExecutorInfo("host2", 1, Map.empty))) + SparkListenerExecutorAdded(2L, execId2, new ExecutorInfo(host2, 1, Map.empty))) // three executors running in one stage and one executor is removed before stage complete listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(1)) - val exec1Metrics = MemoryListenerSuite.createExecutorMetrics("host-1", 1446336000L, 20, 10) + val exec1Metrics = MemoryListenerSuite.createExecutorMetrics(host1, port1, 3L, 20, 10) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, exec1Metrics)) - val exec2Metrics = MemoryListenerSuite.createExecutorMetrics("host-2", 1446337000L, 15, 5) + val exec2Metrics = MemoryListenerSuite.createExecutorMetrics(host2, port2, 4L, 15, 5) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId2, exec2Metrics)) // one more executor added during the stage is running listener.onExecutorAdded( - SparkListenerExecutorAdded(0L, execId3, new ExecutorInfo("host3", 1, Map.empty))) - val exec3Metrics = MemoryListenerSuite.createExecutorMetrics("host-3", 1446338000L, 30, 15) + SparkListenerExecutorAdded(0L, execId3, new ExecutorInfo(host3, 1, Map.empty))) + val exec3Metrics = MemoryListenerSuite.createExecutorMetrics(host3, port3, 5L, 30, 15) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId3, exec3Metrics)) + + assert(listener.activeExecutorIdToMem.size === 3) + assert(listener.removedExecutorIdToMem.isEmpty) + // executor 2 removed before stage complete - listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId2, "")) + listener.onExecutorRemoved(SparkListenerExecutorRemoved(6L, execId2, "")) listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(1)) + (2 to 3).foreach { i => + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(i)) + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(i)) + } - listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId1, "")) - listener.onExecutorRemoved(SparkListenerExecutorRemoved(0L, execId3, "")) + // stages are all completed, no activeStages now + assert(listener.activeStagesToMem.isEmpty) + + listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(4)) + listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( + execId2, new ExecutorMetrics)) + + assert(listener.activeExecutorIdToMem.size === 3) + assert(listener.activeStagesToMem.size === 1) + + listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(4)) - // the completedStagesToMem will maintain the metrics of both the removed executors and new - // added executors - val mapForStage1 = listener.completedStagesToMem.get((1, 0)).get + assert(listener.activeStagesToMem.isEmpty) + assert(listener.completedStagesToMem.size === 4) + assert(listener.activeExecutorIdToMem.size === listener.latestExecIdToExecMetrics.size) + assert(listener.removedExecutorIdToMem.size === 1) + + listener.onExecutorRemoved(SparkListenerExecutorRemoved(7L, execId1, "")) + listener.onExecutorRemoved(SparkListenerExecutorRemoved(8L, execId3, "")) + + assert(listener.removedExecutorIdToMem.size === 3) + + // the {{completedStagesToMem}} will maintain the metrics of both the removed executors and + // new added executors + val mapForStage1 = listener.completedStagesToMem((1, 0)) assert(mapForStage1.size === 3) - val memInfo1 = mapForStage1.get(execId1).get - val memInfo2 = mapForStage1.get(execId2).get - val memInfo3 = mapForStage1.get(execId3).get - val transMetrics1 = memInfo1.transportInfo.get - val transMetrics2 = memInfo2.transportInfo.get - val transMetrics3 = memInfo3.transportInfo.get - assert((20, 10, MemTime(20, 1446336000), MemTime(10, 1446336000)) === ( - transMetrics1.onHeapSize, - transMetrics1.offHeapSize, - transMetrics1.peakOnHeapSizeTime, - transMetrics1.peakOffHeapSizeTime)) - assert((15, 5, MemTime(15, 1446337000), MemTime(5, 1446337000)) === ( - transMetrics2.onHeapSize, - transMetrics2.offHeapSize, - transMetrics2.peakOnHeapSizeTime, - transMetrics2.peakOffHeapSizeTime)) - assert((30, 15, MemTime(30, 1446338000), MemTime(15, 1446338000)) === ( - transMetrics3.onHeapSize, - transMetrics3.offHeapSize, - transMetrics3.peakOnHeapSizeTime, - transMetrics3.peakOffHeapSizeTime)) + + val transMetrics1 = mapForStage1(execId1).transportInfo.get + val transMetrics2 = mapForStage1(execId2).transportInfo.get + val transMetrics3 = mapForStage1(execId3).transportInfo.get + + MemoryListenerSuite.assertTransMetrics( + 20, 10, MemTime(20, 3), MemTime(10, 3), transMetrics1) + MemoryListenerSuite.assertTransMetrics( + 15, 5, MemTime(15, 4), MemTime(5, 4), transMetrics2) + MemoryListenerSuite.assertTransMetrics( + 30, 15, MemTime(30, 5), MemTime(15, 5), transMetrics3) } } -object MemoryListenerSuite { +object MemoryListenerSuite extends SparkFunSuite { def createStageStartEvent(stageId: Int): SparkListenerStageSubmitted = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") SparkListenerStageSubmitted(stageInfo) } - def createStageEndEvent(stageId: Int, failed: Boolean = false): SparkListenerStageCompleted = { + def createStageEndEvent(stageId: Int): SparkListenerStageCompleted = { val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, Seq.empty, Seq.empty, "") - if (failed) { - stageInfo.failureReason = Some("Failed!") - } SparkListenerStageCompleted(stageInfo) } @@ -247,12 +180,22 @@ object MemoryListenerSuite { def createExecutorMetrics( hostname: String, + port: Int, timeStamp: Long, onHeapSize: Long, offHeapSize: Long): ExecutorMetrics = { - val execMetrics = new ExecutorMetrics - execMetrics.setHostname(hostname) - execMetrics.setTransportMetrics(TransportMetrics(timeStamp, onHeapSize, offHeapSize)) - execMetrics + ExecutorMetrics(hostname, port, TransportMetrics(timeStamp, onHeapSize, offHeapSize)) + } + + def assertTransMetrics( + onHeapSize: Long, + offHeapSize: Long, + peakOnHeapSizeTime: MemTime, + peakOffHeapSizTime: MemTime, + transMemSize: TransportMemSize): Unit = { + assert(onHeapSize === transMemSize.onHeapSize) + assert(offHeapSize === transMemSize.offHeapSize) + assert(peakOnHeapSizeTime === transMemSize.peakOnHeapSizeTime) + assert(peakOffHeapSizTime === transMemSize.peakOffHeapSizeTime) } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index e6c0b707602f2..d5aea550b69a1 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -86,6 +86,7 @@ class JsonProtocolSuite extends SparkFunSuite { val executorMetrics = { val execMetrics = new ExecutorMetrics execMetrics.setHostname("host-1") + execMetrics.setPort(80) execMetrics.setTransportMetrics(TransportMetrics(0L, 10, 10)) execMetrics } @@ -387,6 +388,23 @@ class JsonProtocolSuite extends SparkFunSuite { assert(false === oldInfo.internal) } + test("ExecutorMetrics backward compatibility") { + // ExecutorMetrics is newly added + val executorMetricsUpdate = SparkListenerExecutorMetricsUpdate("exec3", new ExecutorMetrics, + Seq((1L, 2, 3, makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, + hasHadoopInput = true, hasOutput = true)))) + assert(executorMetricsUpdate.executorMetrics != null) + assert(executorMetricsUpdate.executorMetrics.transportMetrics != null) + val newJson = JsonProtocol.executorMetricsUpdateToJson(executorMetricsUpdate) + val oldJson = newJson.removeField { case (field, _) => field == "Executor Metrics Updated"} + val newMetrics = JsonProtocol.executorMetricsUpdateFromJson(oldJson) + assert(newMetrics.executorMetrics.hostname === "") + assert(newMetrics.executorMetrics.port === 0) + assert(newMetrics.executorMetrics.transportMetrics.onHeapSize === 0L) + assert(newMetrics.executorMetrics.transportMetrics.offHeapSize === 0L) + assert(newMetrics.executorMetrics.transportMetrics.timeStamp != 0L) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -1708,6 +1726,7 @@ class JsonProtocolSuite extends SparkFunSuite { | "Executor ID": "exec3", | "Executor Metrics Updated": { | "Executor Hostname": "host-1", + | "Executor Port": 80, | "TransportMetrics": { | "TimeStamp": 0, | "OnHeapSize": 10, From 2ce9fd931f1271b0d4e0fa111f8adc271140a8d3 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 8 Dec 2015 13:29:28 +0800 Subject: [PATCH 25/28] fix scala style --- .../scheduler/EventLoggingListenerSuite.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index cb1187ede91e8..40400cff725e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -84,15 +84,15 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } } -// test("End-to-end event logging") { -// testApplicationEventLogging() -// } -// -// test("End-to-end event logging with compression") { -// CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => -// testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) -// } -// } + test("End-to-end event logging") { + testApplicationEventLogging() + } + + test("End-to-end event logging with compression") { + CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => + testApplicationEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) + } + } test("Log overwriting") { val logUri = EventLoggingListener.getLogPath(testDir.toURI, "test", None) @@ -201,7 +201,6 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Following is an integration test with [[org.apache.spark.ui.memory.MemoryListener]], make // sure the events logged in history file can work correctly. - System.out.println(memoryListener) val mapForStage1 = memoryListener.completedStagesToMem((1, 0)) val transMetrics1 = mapForStage1(execId).transportInfo.get MemoryListenerSuite.assertTransMetrics( From 4b3dbe4d41cb06f7ee9b48e816e147ca46a0dea3 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 9 Dec 2015 15:16:29 +0800 Subject: [PATCH 26/28] change port to option and some bug fixes --- .../org/apache/spark/executor/Executor.scala | 4 +- .../spark/executor/ExecutorMetrics.scala | 16 +++-- .../apache/spark/ui/memory/MemoryTab.scala | 66 ++++++++++++++----- .../apache/spark/ui/memory/MemoryTable.scala | 4 +- .../spark/ui/memory/StageMemoryPage.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 6 +- .../scheduler/EventLoggingListenerSuite.scala | 2 +- .../spark/ui/memory/MemoryListenerSuite.scala | 13 ++-- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- 9 files changed, 78 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index d63746348e76a..c63981e07b837 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -87,7 +87,9 @@ private[spark] class Executor( private val executorMetrics: ExecutorMetrics = new ExecutorMetrics executorMetrics.setHostname(Utils.localHostName) - executorMetrics.setPort(env.rpcEnv.address.port) + if (env.rpcEnv.address != null) { + executorMetrics.setPort(Some(env.rpcEnv.address.port)) + } // Whether to load classes in user jars before those in Spark jars private val userClassPathFirst = conf.getBoolean("spark.executor.userClassPathFirst", false) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index a7afcf5169009..9fee863783624 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -39,11 +39,17 @@ class ExecutorMetrics extends Serializable { /** * Host's port the executor runs on */ - private var _port: Int = _ - def port: Int = _port - private[spark] def setPort(value: Int) = _port = value + private var _port: Option[Int] = None + def port: Option[Int] = _port + private[spark] def setPort(value: Option[Int]) = _port = value - private[spark] def hostPort: String = hostname + ":" + port + private[spark] def hostPort: String = { + val hp = port match { + case None => hostname + case value => hostname + ":" + value + } + hp + } private var _transportMetrics: TransportMetrics = new TransportMetrics def transportMetrics: TransportMetrics = _transportMetrics @@ -61,7 +67,7 @@ class ExecutorMetrics extends Serializable { object ExecutorMetrics extends Serializable { def apply( hostName: String, - port: Int, + port: Option[Int], transportMetrics: TransportMetrics): ExecutorMetrics = { val execMetrics = new ExecutorMetrics execMetrics.setHostname(hostName) diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index 91fa2770f7145..d3e7a64a4d279 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -40,27 +40,32 @@ private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory" class MemoryListener extends SparkListener { type ExecutorId = String val activeExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] + // TODO There might be plenty of removed executors (e.g. Dynamic Allocation Mode), This may use + // too much memory. val removedExecutorIdToMem = new HashMap[ExecutorId, MemoryUIInfo] - // latestExecIdToExecMetrics including all executors that is active and removed. - // this may consume a lot of memory when executors are changing frequently, e.g. in dynamical - // allocation mode. + // A map that maintains the latest metrics of each active executor val latestExecIdToExecMetrics = new HashMap[ExecutorId, ExecutorMetrics] // activeStagesToMem a map maintains all executors memory information of each stage, // the Map type is [(stageId, attemptId), Seq[(executorId, MemoryUIInfo)] val activeStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] + // TODO We need to get conf of the retained stages so that we don't need to handle all the + // stages since there might be too many completed stages. val completedStagesToMem = new HashMap[(Int, Int), HashMap[ExecutorId, MemoryUIInfo]] override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate): Unit = { val executorId = event.execId val executorMetrics = event.executorMetrics val memoryInfo = activeExecutorIdToMem.getOrElseUpdate(executorId, new MemoryUIInfo) - memoryInfo.updateExecutorMetrics(executorMetrics) + memoryInfo.updateMemUiInfo(executorMetrics) activeStagesToMem.foreach { case (_, stageMemMetrics) => - if (stageMemMetrics.contains(executorId)) { - stageMemMetrics.get(executorId).get.updateExecutorMetrics(executorMetrics) + // If executor is added in the stage running time, we also update the metrics for the + // executor in {{activeStagesToMem}} + if (!stageMemMetrics.contains(executorId)) { + stageMemMetrics(executorId) = new MemoryUIInfo } + stageMemMetrics(executorId).updateMemUiInfo(executorMetrics) } - latestExecIdToExecMetrics.update(executorId, executorMetrics) + latestExecIdToExecMetrics(executorId) = executorMetrics } override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { @@ -71,28 +76,39 @@ class MemoryListener extends SparkListener { override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { val executorId = event.executorId val info = activeExecutorIdToMem.remove(executorId) + latestExecIdToExecMetrics.remove(executorId) removedExecutorIdToMem.getOrElseUpdate(executorId, info.getOrElse(new MemoryUIInfo)) } override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) val memInfoMap = new HashMap[ExecutorId, MemoryUIInfo] - activeExecutorIdToMem.foreach(idToMem => memInfoMap.update(idToMem._1, new MemoryUIInfo)) - activeStagesToMem.update(stage, memInfoMap) + activeExecutorIdToMem.map { case (id, _) => + memInfoMap(id) = new MemoryUIInfo + val latestExecMetrics = latestExecIdToExecMetrics.get(id) + latestExecMetrics match { + case None => // Do nothing + case Some(metrics) => + memInfoMap(id).updateMemUiInfo(metrics) + } + } + activeStagesToMem(stage) = memInfoMap } override def onStageCompleted(event: SparkListenerStageCompleted): Unit = { val stage = (event.stageInfo.stageId, event.stageInfo.attemptId) + // We need to refresh {{activeStagesToMem}} with {{activeExecutorIdToMem}} in case the + // executor is added in the stage running time and no {{SparkListenerExecutorMetricsUpdate}} + // event is updated in this stage. activeStagesToMem.get(stage).map { memInfoMap => - activeExecutorIdToMem.foreach { case (executorId, _) => - val memInfo = memInfoMap.getOrElse(executorId, new MemoryUIInfo) - latestExecIdToExecMetrics.get(executorId).foreach { prevExecutorMetrics => - memInfo.updateExecutorMetrics(prevExecutorMetrics) + activeExecutorIdToMem.foreach { case (executorId, memUiInfo) => + if (!memInfoMap.contains(executorId)) { + memInfoMap(executorId) = new MemoryUIInfo + memInfoMap(executorId).copyMemUiInfo(memUiInfo) } - memInfoMap.update(executorId, memInfo) } - completedStagesToMem.put(stage, activeStagesToMem.remove(stage).get) } + completedStagesToMem.put(stage, activeStagesToMem.remove(stage).get) } } @@ -106,13 +122,18 @@ class MemoryUIInfo { executorAddress = execInfo.executorHost } - def updateExecutorMetrics(execMetrics: ExecutorMetrics): Unit = { + def updateMemUiInfo(execMetrics: ExecutorMetrics): Unit = { transportInfo = transportInfo match { case Some(transportMemSize) => transportInfo case _ => Some(new TransportMemSize) } executorAddress = execMetrics.hostPort - transportInfo.get.updateTransport(execMetrics.transportMetrics) + transportInfo.get.updateTransMemSize(execMetrics.transportMetrics) + } + + def copyMemUiInfo(memUiInfo: MemoryUIInfo): Unit = { + executorAddress = memUiInfo.executorAddress + transportInfo.foreach(_.copyTransMemSize(memUiInfo.transportInfo.get)) } } @@ -123,7 +144,7 @@ class TransportMemSize { var peakOnHeapSizeTime: MemTime = new MemTime() var peakOffHeapSizeTime: MemTime = new MemTime() - def updateTransport(transportMetrics: TransportMetrics): Unit = { + def updateTransMemSize(transportMetrics: TransportMetrics): Unit = { val updatedOnHeapSize = transportMetrics.onHeapSize val updatedOffHeapSize = transportMetrics.offHeapSize val updateTime: Long = transportMetrics.timeStamp @@ -136,6 +157,15 @@ class TransportMemSize { peakOffHeapSizeTime = MemTime(updatedOffHeapSize, updateTime) } } + + def copyTransMemSize(transMemSize: TransportMemSize): Unit = { + onHeapSize = transMemSize.onHeapSize + offHeapSize = transMemSize.offHeapSize + peakOnHeapSizeTime = MemTime(transMemSize.peakOnHeapSizeTime.memorySize, + transMemSize.peakOnHeapSizeTime.timeStamp) + peakOffHeapSizeTime = MemTime(transMemSize.peakOffHeapSizeTime.memorySize, + transMemSize.peakOffHeapSizeTime.timeStamp) + } } @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala index cdcb84ff715a4..11a1f0dc321ef 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTable.scala @@ -35,9 +35,9 @@ private[ui] class MemTableBase( Executor ID Address Network Memory (on-heap) - Network Memory (direct-heap) + Network Memory (off-heap) Peak Network Memory (on-heap) / Happen Time - Peak Network Read (direct-heap) / Happen Time + Peak Network Read (off-heap) / Happen Time } def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala b/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala index ca32d52a256ef..b5059dce2a522 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/StageMemoryPage.scala @@ -39,7 +39,7 @@ private[ui] class StageMemoryPage(parent: MemoryTab) extends WebUIPage("stage") val finishedStageToMem = memoryListener.completedStagesToMem val content = if (finishedStageToMem.get(stage).isDefined) { - val executorIdToMem = finishedStageToMem.get(stage).get.toSeq.sortBy(_._1) + val executorIdToMem = finishedStageToMem(stage).toSeq.sortBy(_._1) val execMemTable = new MemTableBase(executorIdToMem, memoryListener)

    Executors ({executorIdToMem.size})

    ++ execMemTable.toNodeSeq diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b30234492f103..b228846f2a62d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -24,7 +24,7 @@ import scala.collection.Map import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.DefaultScalaModule -import org.json4s.DefaultFormats +import org.json4s.{JsonAST, DefaultFormats} import org.json4s.JsonDSL._ import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ @@ -296,7 +296,7 @@ private[spark] object JsonProtocol { def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { val transportMetrics = transportMetricsToJson(executorMetrics.transportMetrics) ("Executor Hostname" -> executorMetrics.hostname) ~ - ("Executor Port" -> executorMetrics.port) ~ + ("Executor Port" -> executorMetrics.port.map(new JInt(_)).getOrElse(JNothing)) ~ ("TransportMetrics" -> transportMetrics) } @@ -732,7 +732,7 @@ private[spark] object JsonProtocol { return metrics } metrics.setHostname((json \ "Executor Hostname").extract[String]) - metrics.setPort((json \ "Executor Port").extract[Int]) + metrics.setPort(Utils.jsonOption(json \ "Executor Port").map(_.extract[Int])) metrics.setTransportMetrics(transportMetricsFromJson(json \ "TransportMetrics")) metrics } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 40400cff725e0..920c8dc567aba 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -129,7 +129,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val eventLogger = new EventLoggingListener("test-memListener", None, testDirPath.toUri(), conf) val execId = "exec-1" val hostName = "host-1" - val port = 80 + val port: Option[Int] = Some(80) eventLogger.start() eventLogger.onExecutorAdded(SparkListenerExecutorAdded( diff --git a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala index 9b4f6ff631e9f..d6ed2d9493cc2 100644 --- a/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/memory/MemoryListenerSuite.scala @@ -28,6 +28,7 @@ class MemoryListenerSuite extends SparkFunSuite { val listener = new MemoryListener val execId1 = "exec-1" val host1 = "host-1" + val port: Option[Int] = Some(80) listener.onExecutorAdded( SparkListenerExecutorAdded(1L, execId1, new ExecutorInfo(host1, 1, Map.empty))) @@ -38,13 +39,13 @@ class MemoryListenerSuite extends SparkFunSuite { // multiple metrics updated in stage 2 listener.onStageSubmitted(MemoryListenerSuite.createStageStartEvent(2)) - val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 2L, 20, 10) + val execMetrics1 = MemoryListenerSuite.createExecutorMetrics(host1, port, 2L, 20, 10) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics1)) - val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 3L, 30, 5) + val execMetrics2 = MemoryListenerSuite.createExecutorMetrics(host1, port, 3L, 30, 5) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics2)) - val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(host1, 80, 4L, 15, 15) + val execMetrics3 = MemoryListenerSuite.createExecutorMetrics(host1, port, 4L, 15, 15) listener.onExecutorMetricsUpdate(MemoryListenerSuite.createExecutorMetricsUpdateEvent( execId1, execMetrics3)) listener.onStageCompleted(MemoryListenerSuite.createStageEndEvent(2)) @@ -87,7 +88,8 @@ class MemoryListenerSuite extends SparkFunSuite { val listener = new MemoryListener val (execId1, execId2, execId3) = ("exec-1", "exec-2", "exec-3") val (host1, host2, host3) = ("host-1", "host-2", "host-3") - val (port1, port2, port3) = (80, 80, 80) + val (port1, port2, port3): (Option[Int], Option[Int], Option[Int]) = + (Some(80), Some(80), Some(80)) // two executors added first listener.onExecutorAdded( @@ -135,7 +137,6 @@ class MemoryListenerSuite extends SparkFunSuite { assert(listener.activeStagesToMem.isEmpty) assert(listener.completedStagesToMem.size === 4) - assert(listener.activeExecutorIdToMem.size === listener.latestExecIdToExecMetrics.size) assert(listener.removedExecutorIdToMem.size === 1) listener.onExecutorRemoved(SparkListenerExecutorRemoved(7L, execId1, "")) @@ -180,7 +181,7 @@ object MemoryListenerSuite extends SparkFunSuite { def createExecutorMetrics( hostname: String, - port: Int, + port: Option[Int], timeStamp: Long, onHeapSize: Long, offHeapSize: Long): ExecutorMetrics = { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2bf8a8e398d9d..fa3310c03b119 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -86,7 +86,7 @@ class JsonProtocolSuite extends SparkFunSuite { val executorMetrics = { val execMetrics = new ExecutorMetrics execMetrics.setHostname("host-1") - execMetrics.setPort(80) + execMetrics.setPort(Some(80)) execMetrics.setTransportMetrics(TransportMetrics(0L, 10, 10)) execMetrics } @@ -398,7 +398,7 @@ class JsonProtocolSuite extends SparkFunSuite { val oldJson = newJson.removeField { case (field, _) => field == "Executor Metrics Updated"} val newMetrics = JsonProtocol.executorMetricsUpdateFromJson(oldJson) assert(newMetrics.executorMetrics.hostname === "") - assert(newMetrics.executorMetrics.port === 0) + assert(newMetrics.executorMetrics.port === None) assert(newMetrics.executorMetrics.transportMetrics.onHeapSize === 0L) assert(newMetrics.executorMetrics.transportMetrics.offHeapSize === 0L) assert(newMetrics.executorMetrics.transportMetrics.timeStamp != 0L) From 0ea7cabc0ebd0aafbca63560837cc2f9f0c933bc Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 12 Jan 2016 17:32:13 +0800 Subject: [PATCH 27/28] address comments of code refinement --- .../org/apache/spark/executor/ExecutorMetrics.scala | 8 +------- .../network/netty/NettyBlockTransferService.scala | 13 +++++++------ .../org/apache/spark/scheduler/TaskScheduler.scala | 7 +++++-- .../spark/scheduler/EventLoggingListenerSuite.scala | 10 +++++----- 4 files changed, 18 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 9fee863783624..d63209f52b877 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -46,7 +46,7 @@ class ExecutorMetrics extends Serializable { private[spark] def hostPort: String = { val hp = port match { case None => hostname - case value => hostname + ":" + value + case value => hostname + ":" + value.get } hp } @@ -56,12 +56,6 @@ class ExecutorMetrics extends Serializable { private[spark] def setTransportMetrics(value: TransportMetrics) = { _transportMetrics = value } - - // for test only - def metricsDetails: (String, Long, Long, Long) = { - (hostname, transportMetrics.timeStamp, transportMetrics.onHeapSize, - transportMetrics.offHeapSize) - } } object ExecutorMetrics extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 36ca259be4a68..e3f221cdbc977 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -66,11 +66,10 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val currentTime = clock.getTimeMillis() val clientPooledAllocator = clientFactory.getPooledAllocator() val serverAllocator = server.getAllocator() - val clientOffHeapSize: Long = sumOfMetrics( - clientPooledAllocator.directArenas().asScala.toList) - val clientOnHeapSize: Long = sumOfMetrics(clientPooledAllocator.heapArenas().asScala.toList) - val serverOffHeapSize: Long = sumOfMetrics(serverAllocator.directArenas().asScala.toList) - val serverOnHeapSize: Long = sumOfMetrics(serverAllocator.heapArenas().asScala.toList) + val clientOffHeapSize: Long = sumOfMetrics(convToScala(clientPooledAllocator.directArenas())) + val clientOnHeapSize: Long = sumOfMetrics(convToScala(clientPooledAllocator.heapArenas())) + val serverOffHeapSize: Long = sumOfMetrics(convToScala(serverAllocator.directArenas())) + val serverOnHeapSize: Long = sumOfMetrics(convToScala(serverAllocator.heapArenas())) logDebug(s"Current Netty Client offheap size is $clientOffHeapSize, " + s"Client heap size is $clientOnHeapSize, Server offheap size is $serverOffHeapSize, " + s"server heap size is $serverOnHeapSize, executor id is " + @@ -79,7 +78,9 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage clientOnHeapSize + serverOnHeapSize, clientOffHeapSize + serverOffHeapSize)) } - private def sumOfMetrics(arenaMetricList: List[PoolArenaMetric]): Long = { + private def convToScala = (x: java.util.List[PoolArenaMetric]) => x.asScala + + private def sumOfMetrics(arenaMetricList: Seq[PoolArenaMetric]): Long = { arenaMetricList.map { Arena => Arena.chunkLists().asScala.map { chunk => chunk.iterator().asScala.map(_.chunkSize()).sum diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index b833fc1bfbd85..f4b0a027c36f1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -65,8 +65,11 @@ private[spark] trait TaskScheduler { * alive. Return true if the driver knows about the given block manager. Otherwise, return false, * indicating that the block manager should re-register. */ - def executorHeartbeatReceived(execId: String, executorMetrics: ExecutorMetrics, - taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean + def executorHeartbeatReceived( + execId: String, + executorMetrics: ExecutorMetrics, + taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean /** * Get an application ID associated with the job. diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 920c8dc567aba..fa446552aae81 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -242,12 +242,12 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit execId: String, metricsDetails: (String, Long, Long, Long), event: SparkListenerExecutorMetricsUpdate): Unit = { - val eventDetails = event.executorMetrics.metricsDetails + val execMetrics = event.executorMetrics assert(execId === event.execId) - assert(metricsDetails._1 === eventDetails._1) - assert(metricsDetails._2 === eventDetails._2) - assert(metricsDetails._3 === eventDetails._3) - assert(metricsDetails._4 === eventDetails._4) + assert(metricsDetails._1 === execMetrics.hostname) + assert(metricsDetails._2 === execMetrics.transportMetrics.timeStamp) + assert(metricsDetails._3 === execMetrics.transportMetrics.onHeapSize) + assert(metricsDetails._4 === execMetrics.transportMetrics.offHeapSize) } } From 87f8172fbc1e219cea18e80996b6b0fd12b141de Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 12 Jan 2016 22:19:27 +0800 Subject: [PATCH 28/28] fix import ordering error --- .../spark/network/netty/NettyBlockTransferService.scala | 4 ++-- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 1 - .../src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala | 4 ++-- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 858189897348d..781481c909d32 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -25,7 +25,7 @@ import scala.concurrent.{Future, Promise} import io.netty.buffer._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv} -import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} +import org.apache.spark.executor.{ExecutorMetrics, TransportMetrics} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap, TransportClientFactory} @@ -36,7 +36,7 @@ import org.apache.spark.network.shuffle.protocol.UploadBlock import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} -import org.apache.spark.util.{Clock, Utils, SystemClock} +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * A BlockTransferService that uses Netty to fetch a set of blocks at at time. diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 41f30d13c9709..8730a53065424 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -31,7 +31,6 @@ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} import org.apache.spark.ui.jobs.{JobProgressListener, JobsTab, StagesTab} import org.apache.spark.ui.memory.{MemoryListener, MemoryTab} -import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.ui.scope.RDDOperationGraphListener import org.apache.spark.ui.storage.{StorageListener, StorageTab} import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala index d3e7a64a4d279..cc23334860a5a 100644 --- a/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/memory/MemoryTab.scala @@ -20,10 +20,10 @@ package org.apache.spark.ui.memory import scala.collection.mutable.HashMap import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.{TransportMetrics, ExecutorMetrics} +import org.apache.spark.executor.{ExecutorMetrics, TransportMetrics} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.ui.{SparkUITab, SparkUI} +import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class MemoryTab(parent: SparkUI) extends SparkUITab(parent, "memory") { val memoryListener = parent.memoryListener